From 5d90345d807cb1481324cb334f6046e602bb76a1 Mon Sep 17 00:00:00 2001 From: loi Date: Wed, 27 Nov 2024 17:31:53 -0800 Subject: [PATCH 1/3] FIREFLY-1608: Switch to Redis for Caching and Messaging - Migrate the existing replicating cache to Redis. - Transition from the current messaging system to Redis pub/sub for improved communication. (Mostly completed.) - Support local Redis server for single instance Firefly. - Write tests to cover all functionalities and ensure everything is working correctly. FIREFLY-1611: Refactor CacheManager: Simplify and Update Cache Usage - Remove replication features - Eliminate the 'sharing' requirement - Rename or consolidate the existing cache buckets to better reflect the current usage of Ehcache. --- buildScript/dependencies.gradle | 7 +- config/ehcache.xml | 99 +------- config/test/app-test.prop | 2 +- .../caltech/ipac/astro/net/TargetNetwork.java | 2 +- .../ipac/firefly/core/RedisService.java | 180 +++++++++++++++ .../edu/caltech/ipac/firefly/core/Util.java | 75 +++++++ .../firefly/data/sofia/DataExtractUtil.java | 8 +- .../ipac/firefly/messaging/Message.java | 33 ++- .../ipac/firefly/messaging/Messenger.java | 212 ++++++------------ .../ipac/firefly/server/RequestOwner.java | 7 +- .../firefly/server/cache/DistribMapCache.java | 67 ++++++ .../server/cache/DistributedCache.java | 140 ++++++++++++ .../firefly/server/cache/EhcacheProvider.java | 95 ++++---- ...{KeyBasedCache.java => LocalMapCache.java} | 16 +- .../firefly/server/cache/PrivateCache.java | 62 ----- .../ipac/firefly/server/cache/UserCache.java | 22 +- .../ipac/firefly/server/db/BaseDbAdapter.java | 12 +- .../ipac/firefly/server/db/DbInstance.java | 2 +- .../ipac/firefly/server/db/DuckDbAdapter.java | 3 +- .../firefly/server/db/EmbeddedDbUtil.java | 48 +--- .../ipac/firefly/server/db/HsqlDbAdapter.java | 2 +- .../firefly/server/db/spring/JdbcFactory.java | 6 +- .../server/events/CacheEventWorker.java | 6 +- .../server/events/ReplicatedQueueList.java | 2 +- .../server/persistence/GuestHistoryCache.java | 4 +- .../firefly/server/persistence/QueryIBE.java | 3 +- .../server/query/BaseFileInfoProcessor.java | 4 +- .../server/query/FileGroupsProcessor.java | 3 +- .../server/query/IpacTablePartProcessor.java | 7 +- .../server/query/JsonStringProcessor.java | 5 +- .../server/query/TableResultSearch.java | 2 +- .../server/query/TemplateGenerator.java | 2 +- .../server/query/UserCatalogQuery.java | 3 +- .../server/servlets/AnyFileDownload.java | 4 +- .../server/servlets/AnyFileUpload.java | 4 +- .../server/servlets/MultiPartHandler.java | 3 +- .../server/servlets/MultipartDataUtil.java | 3 +- .../firefly/server/servlets/ServerStatus.java | 8 +- .../firefly/server/visualize/FitsCacher.java | 4 +- .../server/visualize/PlotServUtils.java | 7 +- .../server/visualize/VisServerOps.java | 6 +- .../imageretrieve/LocalFileRetriever.java | 3 +- .../ipac/table/io/TableParseHandler.java | 2 +- .../edu/caltech/ipac/util/StringUtils.java | 6 +- .../edu/caltech/ipac/util/cache/Cache.java | 18 -- .../caltech/ipac/util/cache/CacheManager.java | 137 +++++++---- .../ipac/util/download/CacheHelper.java | 2 +- .../edu/caltech/ipac/firefly/ConfigTest.java | 35 ++- .../ipac/firefly/core/RedisServiceTest.java | 106 +++++++++ .../ipac/firefly/messaging/MessengerTest.java | 47 ++-- .../caltech/ipac/table/DuckDbAdapterTest.java | 2 +- .../ipac/table/EmbeddedDbUtilTest.java | 15 +- .../cache/CachePeerProviderFactoryTest.java | 18 +- .../caltech/ipac/util/cache/CacheTest.java | 120 ++++++++++ 54 files changed, 1111 insertions(+), 580 deletions(-) create mode 100644 src/firefly/java/edu/caltech/ipac/firefly/core/RedisService.java create mode 100644 src/firefly/java/edu/caltech/ipac/firefly/core/Util.java create mode 100644 src/firefly/java/edu/caltech/ipac/firefly/server/cache/DistribMapCache.java create mode 100644 src/firefly/java/edu/caltech/ipac/firefly/server/cache/DistributedCache.java rename src/firefly/java/edu/caltech/ipac/firefly/server/cache/{KeyBasedCache.java => LocalMapCache.java} (81%) delete mode 100644 src/firefly/java/edu/caltech/ipac/firefly/server/cache/PrivateCache.java create mode 100644 src/firefly/test/edu/caltech/ipac/firefly/core/RedisServiceTest.java create mode 100644 src/firefly/test/edu/caltech/ipac/util/cache/CacheTest.java diff --git a/buildScript/dependencies.gradle b/buildScript/dependencies.gradle index ca5d6c4ab9..b82a7659eb 100644 --- a/buildScript/dependencies.gradle +++ b/buildScript/dependencies.gradle @@ -52,8 +52,8 @@ dependencies { implementation ('com.googlecode.json-simple:json-simple:1.1.1') { transitive = false } - // jedis - implementation ('redis.clients:jedis:3.1.0') { exclude group: 'org.slf4j' } + // jedis; support redis versions: 5.0 to 7.2 Family of releases + implementation ('redis.clients:jedis:4.4.8') { exclude group: 'org.slf4j' } // ehcache. exclude core because it contains duplicated older CacheManager implementation ('net.sf.ehcache:ehcache:2.7.4') @@ -98,6 +98,9 @@ dependencies { // duckdb - SQL support for parquet, csv, and tsv files implementation 'org.duckdb:duckdb_jdbc:1.1.3' + + // embedded redis server; version ~ 6.2 + implementation 'com.github.codemonstur:embedded-redis:1.4.3' } diff --git a/config/ehcache.xml b/config/ehcache.xml index bc21a18aa2..9cc67d78ac 100644 --- a/config/ehcache.xml +++ b/config/ehcache.xml @@ -1,5 +1,5 @@ + xsi:noNamespaceSchemaLocation="ehcache.xsd" name="localCache"> - - - - - + overflowToDisk="false" + /> - - - - - - - - - - - - - - - - - - + /> \ No newline at end of file diff --git a/config/test/app-test.prop b/config/test/app-test.prop index 1b4ac6fb69..da69799503 100644 --- a/config/test/app-test.prop +++ b/config/test/app-test.prop @@ -23,5 +23,5 @@ workspace.protocol.webdav=edu.caltech.ipac.firefly.server.WebDAVWorkspaceManager # workspace.protocol.webdav=edu.caltech.ipac.firefly.server.ws.WebdavImpl visualize.fits.Security=false -CacheManager.disabled=true +#CacheManager.disabled=true diff --git a/src/firefly/java/edu/caltech/ipac/astro/net/TargetNetwork.java b/src/firefly/java/edu/caltech/ipac/astro/net/TargetNetwork.java index d929b97201..f5b2e800c9 100644 --- a/src/firefly/java/edu/caltech/ipac/astro/net/TargetNetwork.java +++ b/src/firefly/java/edu/caltech/ipac/astro/net/TargetNetwork.java @@ -18,7 +18,7 @@ public class TargetNetwork { public final static int TWO_MONTHS= 60 * 86400; - private final static Cache objCache= CacheManager.getCache(Cache.TYPE_PERM_SMALL); + private final static Cache objCache= CacheManager.getLocal(); public static ResolvedWorldPt resolveToWorldPt(String objName, Resolver resolver) { if (resolver==null || resolver==UNKNOWN || resolver==NONE) { diff --git a/src/firefly/java/edu/caltech/ipac/firefly/core/RedisService.java b/src/firefly/java/edu/caltech/ipac/firefly/core/RedisService.java new file mode 100644 index 0000000000..6383a3d612 --- /dev/null +++ b/src/firefly/java/edu/caltech/ipac/firefly/core/RedisService.java @@ -0,0 +1,180 @@ +/* + * License information at https://github.com/Caltech-IPAC/firefly/blob/master/License.txt + */ + +package edu.caltech.ipac.firefly.core; + +import edu.caltech.ipac.firefly.server.util.Logger; +import edu.caltech.ipac.util.AppProperties; +import redis.clients.jedis.Jedis; +import redis.clients.jedis.JedisPool; +import redis.clients.jedis.JedisPoolConfig; +import redis.clients.jedis.Protocol; +import redis.embedded.RedisServer; + +import java.io.IOException; +import java.net.ConnectException; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.time.Duration; +import java.time.temporal.ChronoUnit; +import java.util.Arrays; +import java.util.LinkedHashMap; +import java.util.Map; + +import static edu.caltech.ipac.firefly.core.RedisService.Status.ONLINE; +import static edu.caltech.ipac.firefly.core.RedisService.Status.OFFLINE; + + +/** + * RedisService provides a connection management interface for interacting with a Redis server. + * This service is responsible for establishing and managing the Redis connection, + * checking the status of the connection, and ensuring that the connection is healthy. + * + *

The class abstracts the complexity of connecting to Redis, allowing other components of + * the application to interact with Redis without worrying about low-level connection details.

+ * + *

Common use cases include checking the availability of the Redis server, monitoring + * connection health, and ensuring proper connectivity before performing Redis operations.

+ * + * Date: 2024-11-18 + * + * @author loi + * @version $Id: $ + */ +public class RedisService { + public enum Status {ONLINE, FAIL_TO_CONNECT, OFFLINE}; + public static final String REDIS_HOST = "redis.host"; + public static final String MAX_POOL_SIZE = "redis.max.poolsize"; + private static final int REDIS_PORT = AppProperties.getIntProperty("redis.port", 6379); + private static final String MAX_MEM = AppProperties.getProperty("redis.max.mem", "128M"); + private static final String DB_DIR = AppProperties.getProperty("redis.db.dir", System.getProperty("java.io.tmpdir")); + private static final String REDIS_PASSWORD = getRedisPassword(); + private static final Logger.LoggerImpl LOG = Logger.getLogger(); + + // message broker.. Jedis + private static final String redisHost = AppProperties.getProperty(REDIS_HOST, "localhost"); + public static final int maxPoolSize = AppProperties.getIntProperty(MAX_POOL_SIZE, 100); + private static JedisPool jedisPool; + private static Status status = Status.OFFLINE; + + private static String getRedisPassword() { + String passwd = System.getenv("REDIS_PASSWORD"); + if (passwd == null) passwd = AppProperties.getProperty("REDIS_PASSWORD"); + return passwd; + } + + static { + connect(); + } + + static JedisPool createJedisPool() { + try { + JedisPoolConfig pconfig = new JedisPoolConfig(); + pconfig.setTestOnBorrow(true); + pconfig.setMaxTotal(maxPoolSize); + pconfig.setBlockWhenExhausted(true); // wait; if needed + pconfig.setMaxWait(Duration.of(5, ChronoUnit.SECONDS)); + JedisPool pool = new JedisPool(pconfig, redisHost, REDIS_PORT, Protocol.DEFAULT_TIMEOUT, REDIS_PASSWORD); + pool.getResource().close(); // test connection + return pool; + } catch(Exception ignored) {} + return null; + } + + static void startLocal() { + try { + RedisServer redisServer = RedisServer.newRedisServer() + .port(REDIS_PORT) + .setting("maxmemory %s".formatted(MAX_MEM)) + .setting("dir %s".formatted(DB_DIR)) // Directory where redis database files are stored + .setting("dbfilename redis.rdb") // RDB file name + .build(); + redisServer.start(); + connect(); + } catch (IOException ignored) {} + } + + public static void connect() throws RuntimeException { + if (jedisPool != null && !jedisPool.isClosed()) { + status = Status.ONLINE; + return; + } + + jedisPool = createJedisPool(); + if (jedisPool == null && redisHost.equals("localhost")) { + // can't connect; will start up embedded version if localhost + startLocal(); + jedisPool = createJedisPool(); + } + if (jedisPool == null) { + LOG.error("Unable to connect to Redis at " + redisHost + ":" + REDIS_PORT); + status = Status.FAIL_TO_CONNECT; + } else { + status = ONLINE; + } + } + + public static void disconnect() { + status = OFFLINE; + if (jedisPool != null) { + jedisPool.close(); + jedisPool = null; + } + } + + public static Status getStatus() { return status; } + + public static Map getStats() { + + LinkedHashMap stats = new LinkedHashMap<>(); + stats.put("status", getRedisHostPortDesc()); + if (!isOffline()) { + String passwd = ""; + try { + if (REDIS_PASSWORD != null) { + passwd = new String(MessageDigest.getInstance("MD5").digest(REDIS_PASSWORD.getBytes())); + } + } catch (NoSuchAlgorithmException e) {/* ignore */} + try (Jedis redis = getConnection()) { + + stats.put("active conn", jedisPool.getNumActive()); + stats.put("idle conn", jedisPool.getNumIdle()); + stats.put("max conn", maxPoolSize); + stats.put("max-wait", jedisPool.getMaxBorrowWaitTimeMillis()); + stats.put("avg-wait", jedisPool.getMeanBorrowWaitTimeMillis()); + stats.put("cache-size", redis.dbSize()); + stats.put("DB_DIR", DB_DIR); + stats.put("MAX_MEM", MAX_MEM); + stats.put("password", passwd); + + Arrays.stream(redis.info().split("\r\n")).forEach((s) -> { + if (!s.startsWith("#")) { + String[] kv = s.split(":"); + stats.put(kv[0], kv[1]); + } + }); + } catch (Exception ignored) {} + } + return stats; + } + + public static String getRedisHostPortDesc() { + return redisHost + ":" + REDIS_PORT + " ("+ getStatus() + ")"; + } + + public static boolean isOffline() { + return !getStatus().equals(ONLINE); + } + + public static int getConnectionCount() { + return isOffline() ? -1 : jedisPool.getNumActive(); + } + + public static Jedis getConnection() throws Exception { + if(!isOffline()) { + return jedisPool.getResource(); + } + throw new ConnectException("Unable to connect to Redis at " + REDIS_HOST + ":" + REDIS_PORT); + } +} diff --git a/src/firefly/java/edu/caltech/ipac/firefly/core/Util.java b/src/firefly/java/edu/caltech/ipac/firefly/core/Util.java new file mode 100644 index 0000000000..f23567824b --- /dev/null +++ b/src/firefly/java/edu/caltech/ipac/firefly/core/Util.java @@ -0,0 +1,75 @@ +/* + * License information at https://github.com/Caltech-IPAC/firefly/blob/master/License.txt + */ + +package edu.caltech.ipac.firefly.core; + +import edu.caltech.ipac.firefly.server.util.Logger; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.util.Base64; + +/** + * Date: 11/19/24 + * + * @author loi + * @version : $ + */ +public class Util { + private static final Logger.LoggerImpl logger = Logger.getLogger(); + + + /** + * Serializes a Java object into a Base64-encoded string. + *

+ * This method converts the given object into a byte stream, + * encodes the byte stream into a Base64 string, and returns the result. + * The object must implement {@link java.io.Serializable} for this method to work. + *

+ * @param obj the object to serialize; must implement {@link java.io.Serializable} + * @return a Base64-encoded string representing the serialized object, or null + */ + public static String serialize(Object obj) { + if (obj == null) return null; + try { + ByteArrayOutputStream bstream = new ByteArrayOutputStream(); + ObjectOutputStream ostream = new ObjectOutputStream(bstream); + ostream.writeObject(obj); + ostream.flush(); + byte[] bytes = bstream.toByteArray(); + return Base64.getEncoder().encodeToString(bytes); + } catch (Exception e) { + logger.warn(e); + return null; + } + } + + /** + * Deserializes a Base64-encoded string into a Java object. + *

+ * This method decodes the provided Base64 string into a byte stream, + * then reconstructs the original object using Java's object serialization mechanism. + *

+ * @param base64 the Base64-encoded string representing the serialized object + * @return the deserialized Java object, or null. + */ + public static Object deserialize(String base64) { + try { + if (base64 == null) return null; + byte[] bytes = Base64.getDecoder().decode(base64); + ByteArrayInputStream bstream = new ByteArrayInputStream(bytes); + ObjectInputStream ostream = new ObjectInputStream(bstream); + return ostream.readObject(); + } catch (Exception e) { + logger.warn(e); + return null; + } + } + + + + +} diff --git a/src/firefly/java/edu/caltech/ipac/firefly/data/sofia/DataExtractUtil.java b/src/firefly/java/edu/caltech/ipac/firefly/data/sofia/DataExtractUtil.java index 46f4d064cd..2a7518c0cc 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/data/sofia/DataExtractUtil.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/data/sofia/DataExtractUtil.java @@ -3,16 +3,12 @@ import edu.caltech.ipac.firefly.data.ServerRequest; import edu.caltech.ipac.firefly.server.ServerCommandAccess; import edu.caltech.ipac.firefly.server.ServerContext; -import edu.caltech.ipac.firefly.server.SrvParam; -import edu.caltech.ipac.firefly.server.cache.UserCache; -import edu.caltech.ipac.firefly.server.query.EmbeddedDbProcessor; import edu.caltech.ipac.firefly.server.util.multipart.UploadFileInfo; import edu.caltech.ipac.table.DataGroup; import edu.caltech.ipac.util.cache.Cache; +import edu.caltech.ipac.util.cache.CacheManager; import edu.caltech.ipac.util.cache.StringKey; -import javax.servlet.http.HttpServletRequest; -import javax.servlet.http.HttpServletResponse; import java.io.File; /** @@ -36,7 +32,7 @@ public abstract class DataExtractUtil { */ public File getSourceFile(ServerRequest req) { String fileKey = req.getParam("file"); - Cache sessionCache = UserCache.getInstance(); + Cache sessionCache = CacheManager.getUserCache(); File sourceFile = ServerContext.convertToFile(fileKey); if (sourceFile == null || !sourceFile.canRead()) { UploadFileInfo tmp = (UploadFileInfo) (sessionCache.get(new StringKey(fileKey))); diff --git a/src/firefly/java/edu/caltech/ipac/firefly/messaging/Message.java b/src/firefly/java/edu/caltech/ipac/firefly/messaging/Message.java index e60938250d..b0e4e93ef2 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/messaging/Message.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/messaging/Message.java @@ -4,12 +4,16 @@ package edu.caltech.ipac.firefly.messaging; +import edu.caltech.ipac.firefly.core.background.JobInfo; +import edu.caltech.ipac.firefly.core.background.JobManager; import edu.caltech.ipac.firefly.data.ServerEvent.Scope; +import edu.caltech.ipac.firefly.server.ServerContext; +import edu.caltech.ipac.util.AppProperties; /** * The message object sent or received by Messenger. As implemented, Messenger * uses JSON when serialize/deserialize is needed. However, this is an internal - * implementation and it can be changed without breaking Messenger/Message contract. + * implementation, and it can be changed without breaking Messenger/Message contract. * * Date: 2019-03-15 * @@ -17,7 +21,7 @@ * @version $Id: $ */ public class Message { - private JsonHelper helper = new JsonHelper(); + protected JsonHelper helper = new JsonHelper(); public MsgHeader getHeader() { Scope scope = Scope.valueOf(helper.getValue(Scope.SELF.name(), "header", "scope")); @@ -54,4 +58,29 @@ public static Message parse(String json) { public String toJson() { return helper.toJson(); } + +//==================================================================== +// Predefined messages +//==================================================================== + + public static final class JobCompleted extends Message { + public static final String TOPIC = "JobCompleted"; + static final String FROM = AppProperties.getProperty("mail.smtp.from", "donotreply@ipac.caltech.edu"); + JobInfo job; + public JobCompleted(JobInfo jobInfo) { + job = jobInfo; + setHeader(Scope.CHANNEL, jobInfo.getOwner(), TOPIC, FROM); + helper.setValue(JobManager.toJsonObject(jobInfo), "jobInfo"); + var user = ServerContext.getRequestOwner().getUserInfo(); + if (user != null) { + helper.setValue(user.getName(), "user", "name"); + helper.setValue(user.getEmail(), "user", "email"); + helper.setValue(user.getLoginName(), "user", "loginName"); + } + var ssoAdpt = ServerContext.getRequestOwner().getSsoAdapter(); + if (ssoAdpt != null) { + helper.setValue(ssoAdpt.getAuthToken(), "user", "authToken"); + } + } + } } \ No newline at end of file diff --git a/src/firefly/java/edu/caltech/ipac/firefly/messaging/Messenger.java b/src/firefly/java/edu/caltech/ipac/firefly/messaging/Messenger.java index f02fda0690..77c2fc108a 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/messaging/Messenger.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/messaging/Messenger.java @@ -4,25 +4,18 @@ package edu.caltech.ipac.firefly.messaging; +import edu.caltech.ipac.firefly.core.RedisService; import edu.caltech.ipac.firefly.server.util.Logger; -import edu.caltech.ipac.util.AppProperties; import redis.clients.jedis.Jedis; -import redis.clients.jedis.JedisPool; -import redis.clients.jedis.JedisPoolConfig; import redis.clients.jedis.JedisPubSub; -import redis.clients.jedis.Protocol; -import java.security.MessageDigest; -import java.security.NoSuchAlgorithmException; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; /** * An implementation of publish-subscribe messaging pattern based on Jedis client and Redis backend. - * This class abstract the use of threads and it ensures that there is only 1 thread used per topic. - * + * This class abstract the use of threads, and it ensures that there is only 1 thread used per topic. * Topic refer here is equivalent to jedis 'channel'. This is so it will not be confused with channel * used in Scope. * @@ -33,117 +26,38 @@ * @version $Id: $ */ public class Messenger { - public enum Status {CONNECTED, FAIL_TO_CONNECT, NOT_USING}; - private static final String REDIS_HOST = AppProperties.getProperty("redis.host", "127.0.0.1"); - private static final int REDIS_PORT = AppProperties.getIntProperty("redis.port", 6379); - private static final int MAX_POOL_SIZE = AppProperties.getIntProperty("redis.max.poolsize", 25); - private static final String REDIS_PASSWORD = getRedisPassword(); - private static final Logger.LoggerImpl LOG = Logger.getLogger(); - - // message broker.. Jedis - private static JedisPool jedisPool; - private static Status status = Status.NOT_USING; + private static Logger.LoggerImpl LOG = Logger.getLogger(); // to limit one thread per topic private static ConcurrentHashMap pubSubHandlers = new ConcurrentHashMap<>(); - private static String getRedisPassword() { - String passwd = System.getenv("REDIS_PASSWORD"); - if (passwd == null) passwd = AppProperties.getProperty("REDIS_PASSWORD"); - return passwd; - } - - static boolean init() { - - if (jedisPool == null) { - try { - JedisPoolConfig pconfig = new JedisPoolConfig(); - pconfig.setTestOnBorrow(true); - pconfig.setMaxTotal(MAX_POOL_SIZE); - pconfig.setBlockWhenExhausted(true); // wait.. if needed - jedisPool = new JedisPool(pconfig, REDIS_HOST, REDIS_PORT, Protocol.DEFAULT_TIMEOUT, REDIS_PASSWORD); - jedisPool.getResource().close(); - status = Status.CONNECTED; - } catch (Exception ex) { - LOG.error("Unable to connect to Redis at " + REDIS_HOST + ":" + REDIS_PORT); - jedisPool = null; - status = Status.FAIL_TO_CONNECT; - return false; - } - } - return true; - } - - public static Status getStatus() { return status; } - - public static String getStats() { - - if (getStatus()==Status.NOT_USING) return "Messenger is offline"; - JsonHelper stats = new JsonHelper(); - if (!init()) { - return stats.setValue("Messenger is offline").toJson(); - } else { - String passwd = ""; - try { - if (REDIS_PASSWORD != null) { - passwd = new String(MessageDigest.getInstance("MD5").digest(REDIS_PASSWORD.getBytes())); - } - } catch (NoSuchAlgorithmException e) {/* ignore */} - - return stats.setValue(jedisPool.getNumActive(), "active") - .setValue(jedisPool.getNumIdle(), "idle") - .setValue(MAX_POOL_SIZE, "max") - .setValue(jedisPool.getMaxBorrowWaitTimeMillis(), "max-wait") - .setValue(jedisPool.getMeanBorrowWaitTimeMillis(), "avg-wait") - .setValue(passwd, "password") - .toJson(); - } - } - - public static String getRedisHostPortDesc() { - return REDIS_HOST + ":" + REDIS_PORT + " ("+ getStatus() + ")"; - } - - public static boolean isOffline() { - if (!init()) return true; - - try (Jedis ignored = jedisPool.getResource()) { // test connection - return false; - } catch (Exception e) { - return true; - } - } - - public static int getConnectionCount() { - if (!init()) return -1; - return jedisPool.getNumActive(); - } - /** * @param topic the topic to subscribe to * @param subscriber the subscriber to receive the messages * @return the given subscriber. Useful for functional programming. */ public static Subscriber subscribe(String topic, Subscriber subscriber) { - if (!init()) return null; - if (pubSubHandlers.containsKey(topic)) { + LOG.trace("Add subscriber to existing topic: " + topic); SubscriberHandle pubSub = pubSubHandlers.get(topic); pubSub.addSubscriber(subscriber); } else { + LOG.trace("Add subscriber to new topic: " + topic); SubscriberHandle pubSub = new SubscriberHandle(topic); - pubSub.addSubscriber(subscriber); pubSubHandlers.put(topic, pubSub); + pubSub.addSubscriber(subscriber); } return subscriber; } + public static int getSubscribedTopics() { + return pubSubHandlers.size(); + } + /** * @param subscriber the subscriber to remove */ public static void unSubscribe(Subscriber subscriber) { - if (!init()) return; - pubSubHandlers.values().stream() .filter(hdl -> hdl.subscribers.contains(subscriber)) .forEach(hdl -> hdl.removeSubscriber(subscriber)); @@ -155,9 +69,7 @@ public static void unSubscribe(Subscriber subscriber) { * @param msg message to send */ public static void publish(String topic, Message msg) { - if (!init()) return; - - try (Jedis jedis = jedisPool.getResource()) { + try (Jedis jedis = RedisService.getConnection()) { jedis.publish(topic, msg.toJson()); } catch (Exception e) { LOG.error(e.getMessage()); @@ -165,26 +77,44 @@ public static void publish(String topic, Message msg) { } /** - * Send the given message. A topic will derived from the msg's headers + * Send the given message. The message's subject is used as the topic. * @param msg the message to send */ public static void publish(Message msg) { // some firefly's specific logic here... - String topic = msg.getHeader().getScope().name(); + String topic = msg.getHeader().getSubject(); publish(topic, msg); } /** - * Internal handler class used to manage the one to many relationship of Messenger's subscriber and + * Internal handler class used to manage the one-to-many relationship of Messenger's subscriber and * Jedis's subscriber */ static class SubscriberHandle { - private CopyOnWriteArrayList subscribers = new CopyOnWriteArrayList<>(); - private String topic; - ExecutorService executor; - private JedisPubSub jPubSub; + private final CopyOnWriteArrayList subscribers = new CopyOnWriteArrayList<>(); + private final String topic; + private final AtomicInteger retries = new AtomicInteger(5); + JedisPubSub jPubSub = new JedisPubSub() { + public void onMessage(String channel, String message) { + Message msg = Message.parse(message); + subscribers.forEach((sub) -> { + try { + sub.onMessage(msg); + } catch (Exception e) { + LOG.warn("Error while processing message: " + e.getMessage()); + } + }); + } + public void onSubscribe(String channel, int subscribedChannels) { + LOG.info("Subscribed to topic: " + topic); + } + public void onUnsubscribe(String channel, int subscribedChannels) { + LOG.info("Unsubscribed from topic: " + channel); + } + + }; SubscriberHandle(String topic) { this.topic = topic; @@ -192,59 +122,43 @@ static class SubscriberHandle { void addSubscriber(Subscriber sub) { subscribers.add(sub); - if (subscribers.size() == 1) { - // first subscriber.. need to connect to redis - doSubscribe(); - } + subscribe(); } void removeSubscriber(Subscriber sub) { subscribers.remove(sub); - if (subscribers.size() == 0) { - // no more subscrber.. disconnect from redis - cleanup(); + if (subscribers.isEmpty()) { + // no more subscriber; disconnect from redis + jPubSub.unsubscribe(); } } - void doSubscribe() { - jPubSub = new JedisPubSub() { - public void onMessage(String channel, String message) { - Message msg = Message.parse(message); - subscribers.forEach((sub) -> { - try { - sub.onMessage(msg); - } catch (Exception e) { - LOG.warn("Error while processing message: " + e.getMessage()); + void subscribe() { + if (subscribers.size() > 1 || jPubSub.isSubscribed()) return; // already subscribed; do nothing. + + Thread subscriberThread = new Thread(() -> { + LOG.trace("start subscribing to topic: " + topic); + try (Jedis jedis = RedisService.getConnection()) { + jedis.subscribe(jPubSub, topic); // Blocks here + } catch (Exception e) { + LOG.error(e, "Error while subscribing to topic: " + topic); + if (!subscribers.isEmpty()) { + if (retries.decrementAndGet() != 0) { + try { + LOG.info("Retry subscribing to %s after 1s wait".formatted(topic)); + Thread.sleep(1_000); + subscribe(); + } catch (InterruptedException ignored) {} + } else { + LOG.info("Gave up subscribing after a connection failure to the topic: " + topic); + pubSubHandlers.remove(topic); // give up and clear topic cache } - }); - } - }; - - executor = Executors.newSingleThreadExecutor(); - executor.submit(() -> { - try { - while (subscribers.size() > 0) { - if (Messenger.init()) { - try (Jedis jedis = jedisPool.getResource()) { - jedis.subscribe(jPubSub, topic); - } catch (Exception e) { - // quietly ignores to avoid excessive error logs. - } - } - - Thread.sleep(5000); // if disconnected and there's still subscribers to this topic, attempt to reconnect after a brief pause. } - } catch (InterruptedException e) { - cleanup(); } + LOG.trace("exiting subscribing to topic: " + topic); }); - } - - void cleanup() { - if (jPubSub != null) jPubSub.unsubscribe(); - jPubSub = null; - if (executor != null) executor.shutdown(); - executor = null; + // Start the subscriber in a separate thread + subscriberThread.start(); } } diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/RequestOwner.java b/src/firefly/java/edu/caltech/ipac/firefly/server/RequestOwner.java index f090303e7b..ff2a77d679 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/RequestOwner.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/RequestOwner.java @@ -7,7 +7,6 @@ import edu.caltech.ipac.firefly.server.cache.UserCache; import edu.caltech.ipac.firefly.server.events.FluxAction; import edu.caltech.ipac.firefly.server.events.ServerEventManager; -import edu.caltech.ipac.firefly.server.network.HttpServiceInput; import edu.caltech.ipac.firefly.server.security.SsoAdapter; import edu.caltech.ipac.firefly.server.util.Logger; import edu.caltech.ipac.firefly.server.ws.WorkspaceFactory; @@ -39,6 +38,7 @@ public class RequestOwner implements Cloneable { private static final Logger.LoggerImpl LOG = Logger.getLogger(); public static String USER_KEY = "usrkey"; + public static int USER_KEY_EXPIRY = 3600 * 24 * 7 * 2; // 2 weeks public static final String SET_USERINFO_ACTION = "app_data.setUserInfo"; private static boolean ignoreAuth = AppProperties.getBooleanProperty("ignore.auth", false); private RequestAgent requestAgent; @@ -171,7 +171,7 @@ public UserInfo getUserInfo() { } if (userInfo == null) { - Cache cache = CacheManager.getCache(Cache.TYPE_PERM_SMALL); + Cache cache = CacheManager.getLocal(); userInfo = (UserInfo) cache.get(new StringKey(getUserKey())); if (userInfo == null) { userInfo = UserInfo.newGuestUser(); @@ -238,7 +238,6 @@ private String newUserKey() { throw new RuntimeException("Unable to generate a new userKey after 1000 tries."); } } while (UserCache.exists(new StringKey(userKey))); - UserCache.create(new StringKey(userKey)); return userKey; } @@ -247,7 +246,7 @@ private void updateUserKey(UserInfo userInfo) { String cVal = requestAgent.getCookieVal(USER_KEY, ""); if (!userKey.equals(cVal)) { Cookie cookie = new Cookie(USER_KEY, userKey); - cookie.setMaxAge(3600 * 24 * 7 * 2); // to live for two weeks + cookie.setMaxAge(USER_KEY_EXPIRY); // to live for two weeks cookie.setPath(requestAgent.getContextPath()); requestAgent.sendCookie(cookie); } diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/cache/DistribMapCache.java b/src/firefly/java/edu/caltech/ipac/firefly/server/cache/DistribMapCache.java new file mode 100644 index 0000000000..aa25d47d76 --- /dev/null +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/cache/DistribMapCache.java @@ -0,0 +1,67 @@ +/* + * License information at https://github.com/Caltech-IPAC/firefly/blob/master/License.txt + */ +package edu.caltech.ipac.firefly.server.cache; + +import redis.clients.jedis.Jedis; +import java.util.ArrayList; +import java.util.List; + +/** + * Like {@link DistributedCache} but specifically designed for managing Redis Maps. + *

+ * Date: Nov 18, 2024 + *

+ * @author loi + * @version $Id: EhcacheImpl.java,v 1.8 2009/12/16 21:43:25 loi Exp $ + */ +public class DistribMapCache extends DistributedCache { + String mapKey; + long lifespanInSecs; + + public DistribMapCache(String mapKey) { + this(mapKey, 0); + } + + public DistribMapCache(String mapKey, long lifespanInSecs) { + this.mapKey = mapKey; + this.lifespanInSecs = lifespanInSecs; + } + +//==================================================================== +// override for Redis Map implementation +//==================================================================== + + String get(Jedis redis, String key) { + return redis.hget(mapKey, key); + } + + void del(Jedis redis, String key) { + redis.hdel(mapKey, key); + } + + void set(Jedis redis, String key, String value) { + boolean exists = redis.exists(mapKey); + redis.hset(mapKey, key, value); + if (!exists && lifespanInSecs > 0) { + redis.expire(mapKey, lifespanInSecs); // set only when creating a new map; setting here instead of hset to accommodate older version of redis. + } + } + + void setex(Jedis redis, String key, String value, long lifespanInSecs) { + throw new IllegalArgumentException("Cannot set expiry on individual key. Do it as Map"); + } + + List keys(Jedis redis) { + return new ArrayList<>(redis.hkeys(mapKey)); + } + + boolean exists(Jedis redis, String key) { + return redis.hexists(mapKey, key); + } + + int size(Jedis redis) { + return (int) redis.hlen(mapKey); + } +} + diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/cache/DistributedCache.java b/src/firefly/java/edu/caltech/ipac/firefly/server/cache/DistributedCache.java new file mode 100644 index 0000000000..68ba4cdacb --- /dev/null +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/cache/DistributedCache.java @@ -0,0 +1,140 @@ +/* + * License information at https://github.com/Caltech-IPAC/firefly/blob/master/License.txt + */ +package edu.caltech.ipac.firefly.server.cache; + +import edu.caltech.ipac.firefly.core.RedisService; +import edu.caltech.ipac.firefly.server.util.Logger; +import edu.caltech.ipac.util.cache.Cache; +import edu.caltech.ipac.util.cache.CacheKey; +import redis.clients.jedis.Jedis; + +import java.util.List; + +import static edu.caltech.ipac.firefly.core.Util.deserialize; +import static edu.caltech.ipac.firefly.core.Util.serialize; + +/** + * This class provides an implementation of a distributed cache using Redis. + *

+ * While Redis supports various data structures such as lists, sets, and maps, + * this implementation is designed to focus solely on string-based storage. + * In some cases, the stored strings may represent JSON-formatted data to encapsulate + * more complex data structures. + *

+ * For storage of plain objects beyond simple strings, this implementation + * uses Base64 encoding. Objects are serialized into Base64-encoded strings + * for storage and are deserialized back into objects when retrieved from the cache. + * This approach ensures compatibility with Redis's string data type while + * maintaining flexibility for handling diverse data types. + *

+ * Date: Nov 18, 2024 + * @author loi + * @version $Id: EhcacheImpl.java,v 1.8 2009/12/16 21:43:25 loi Exp $ + */ +public class DistributedCache implements Cache { + static final Logger.LoggerImpl LOG = Logger.getLogger(); + private static final String BASE64 = "BASE64::"; + + public void put(CacheKey key, Object value) { + put(key, value, 0); + } + + public void put(CacheKey key, Object value, int lifespanInSecs) { + String keystr = key.getUniqueString(); + try(Jedis redis = RedisService.getConnection()) { + if (redis != null) { + if (value == null) { + del(redis, keystr); + } else { + if (lifespanInSecs > 0) { + setex(redis, keystr, v2set(value), lifespanInSecs); + } else { + set(redis, keystr, v2set(value)); + } + } + } + } catch (Exception ex) { LOG.error(ex); } + } + + public Object get(CacheKey key) { + try(Jedis redis = RedisService.getConnection()) { + return v2get( get(redis, key.getUniqueString()) ); + } catch (Exception ex) { LOG.error(ex); } + return null; + } + + public boolean isCached(CacheKey key) { + try(Jedis redis = RedisService.getConnection()) { + return exists(redis, key.getUniqueString()); + } catch (Exception ex) { LOG.error(ex); } + return false; + } + + public List getKeys() { + try(Jedis redis = RedisService.getConnection()) { + return keys(redis); + } catch (Exception ex) { LOG.error(ex); } + return null; + } + + public int getSize() { + try(Jedis redis = RedisService.getConnection()) { + return size(redis); + } catch (Exception ex) { LOG.error(ex); } + return -1; + } + +//==================================================================== +// Implementation of redis string; override for map, list, and set. +//==================================================================== + + String get(Jedis redis, String key) { + return redis.get(key); + } + + void del(Jedis redis, String key) { + redis.del(key); + } + + void set(Jedis redis, String key, String value) { + redis.set(key, value); + } + + void setex(Jedis redis, String key, String value, long lifespanInSecs) { + redis.setex(key, lifespanInSecs, value); + } + + List keys(Jedis redis) { + return redis.keys("*").stream().toList(); + } + + boolean exists(Jedis redis, String key) { + return redis.exists(key); + } + + int size(Jedis redis) { + return Math.toIntExact(redis.dbSize()); + } + +//==================================================================== +// Utility functions +//==================================================================== + + static String v2set(Object object) { + if (object instanceof String v) { + return v; + } else { + return BASE64 + serialize(object); + } + } + + static Object v2get(Object object) { + if (object instanceof String v) { + return v.startsWith(BASE64) ? deserialize(v.substring(BASE64.length())) : v; + } else { + return object; // this should not happen. + } + } + +} diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/cache/EhcacheProvider.java b/src/firefly/java/edu/caltech/ipac/firefly/server/cache/EhcacheProvider.java index 368d0094c9..04e79d1f8a 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/cache/EhcacheProvider.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/cache/EhcacheProvider.java @@ -19,19 +19,33 @@ import java.io.File; import java.net.MalformedURLException; import java.net.URL; +import java.util.List; import java.util.concurrent.TimeUnit; +import static edu.caltech.ipac.util.cache.Cache.*; +import static edu.caltech.ipac.util.cache.CacheManager.EmptyCache; + /** - * Date: Jul 17, 2008 - * + * 11/21/2024 + * With the move to containerized deployments, each App Server instance will typically host only one application. + * Additionally, we are no longer replicating cache buckets. These changes allow us to simplify the caching strategy. + * It is no longer necessary to differentiate between cache buckets shared across applications within the same App Server + * and those that are not. Instead, we can assume a single shared cache for all applications. + *

+ * Created: Jul 17, 2008 * @author loi * @version $Id: EhcacheProvider.java,v 1.28 2012/09/25 23:50:59 loi Exp $ */ public class EhcacheProvider implements Cache.Provider { + /* + * ehcache buckets defined in ehcache.xml configuration file. + */ + public static final String VIS_SHARED_MEM = "VIS_SHARED_MEM"; + public static final String PERM_SMALL = "PERM_SMALL"; + private static final Logger.LoggerImpl _log= Logger.getLogger(); - private static final net.sf.ehcache.CacheManager manager; - private static net.sf.ehcache.CacheManager sharedManager; + private static net.sf.ehcache.CacheManager manager = null; private static final boolean enableJMX = AppProperties.getBooleanProperty("ehcache.jmx.monitor", true); private static long curConfModTime = 0; @@ -56,37 +70,22 @@ public class EhcacheProvider implements Cache.Provider { _log.info("cache manager config file: " + url); } - _log.info("loading ehcache config file: " + url); - - manager = net.sf.ehcache.CacheManager.newInstance(url); - - // Due to the shared nature of this file, we only want to pick up the latest version. // Latest is based on when ehcache.xml is modified. - if (f.lastModified() > curConfModTime) { + if (url != null && (manager == null || f.lastModified() > curConfModTime)) { curConfModTime = f.lastModified(); + _log.info("loading ehcache config file: " + url); - File sharedConfig = ServerContext.getConfigFile("shared_ehcache.xml"); File ignoreSizeOf = ServerContext.getConfigFile("ignore_sizeof.txt"); System.setProperty("net.sf.ehcache.sizeof.filter", ignoreSizeOf.getAbsolutePath()); - // Two 2 tries to start cache manager: - // 1. The first time will only work in the single app deployment such the firefly standalone version. - // Ehcahce is not deployed in the tomcat lib directory. - // - // 2. In the typical multi app production case there will be an exception, because ehcache is in the tomcat lib - // directory and has a different class loader. Then the cache manager will start without - // the sizeofEngine override. To use the sizeofEngine wrapper in multi app production case it would need to - // be a jar that is placed in the tomcat lib directory alone with EHcache. - String sizeEngName="unknown"; try { - sizeEngName= ObjectSizeEngineWrapper.class.getName(); - System.setProperty("net.sf.ehcache.sizeofengine.shared.VIS_SHARED_MEM", sizeEngName); - sharedManager = CacheManager.create(sharedConfig.getAbsolutePath()); + String sizeEngName= ObjectSizeEngineWrapper.class.getName(); + System.setProperty("net.sf.ehcache.sizeofengine.localCache.VIS_SHARED_MEM", sizeEngName); + manager = net.sf.ehcache.CacheManager.newInstance(url); } catch (RuntimeException e) { - System.clearProperty("net.sf.ehcache.sizeofengine.shared.VIS_SHARED_MEM"); - _log.warn("failed to VIS_SHARED_MEM sizeof engine: "+sizeEngName, "Using default"); - sharedManager = CacheManager.create(sharedConfig.getAbsolutePath()); + _log.error(e, "unable to create net.sf.ehcache.CacheManager"); + throw new CacheException("unable to create net.sf.ehcache.CacheManager"); } float pctVisSharedMemSize = AppProperties.getFloatProperty("pct.vis.shared.mem.size", 0F); @@ -96,18 +95,17 @@ public class EhcacheProvider implements Cache.Provider { if (StringUtils.isEmpty(sharedMemSize)) { sharedMemSize = String.format("%dM", (int)(Runtime.getRuntime().maxMemory() * pctVisSharedMemSize/1024/1024)); } - sharedManager.getCache(Cache.TYPE_VIS_SHARED_MEM).getCacheConfiguration().setMaxBytesLocalHeap(sharedMemSize); + manager.getCache(VIS_SHARED_MEM).getCacheConfiguration().setMaxBytesLocalHeap(sharedMemSize); // setup cleanup task int ttiSecs = AppProperties.getIntProperty("vis.shared.tti.secs", DEF_TTI_SEC); // defaults to expire after 60 mins of inactivity. - sharedManager.getCache(Cache.TYPE_VIS_SHARED_MEM).getCacheConfiguration().setTimeToIdleSeconds(ttiSecs); + manager.getCache(VIS_SHARED_MEM).getCacheConfiguration().setTimeToIdleSeconds(ttiSecs); ServerContext.SCHEDULE_TASK_EXEC.scheduleAtFixedRate( - () -> sharedManager.getCache(Cache.TYPE_VIS_SHARED_MEM).getKeysWithExpiryCheck(), // forces expiry check + () -> manager.getCache(VIS_SHARED_MEM).getKeysWithExpiryCheck(), // forces expiry check EVICT_CHECK_INTVL, EVICT_CHECK_INTVL, TimeUnit.MINUTES); // check every EVICT_CHECK_INTVL minutes } - _log.info("shared cache manager config file: " + sharedConfig); } @@ -121,14 +119,10 @@ public class EhcacheProvider implements Cache.Provider { public Cache getCache(String type) { Ehcache ehcache = getEhcacheManager(type).getCache(type); if (ehcache == null) { - throw new IllegalArgumentException("Unknow cache type. Make sure cache type '" + + throw new IllegalArgumentException("Unknown cache type. Make sure cache type '" + type + "' is defined in your ehcache.xml file"); } - if (type.equals(Cache.TYPE_PERM_FILE) || type.equals(Cache.TYPE_TEMP_FILE)) { - return new FileCache(ehcache); - } else { - return new EhcacheImpl(ehcache); - } + return new EhcacheImpl(ehcache); } public void shutdown() { @@ -141,16 +135,8 @@ public CacheManager getEhcacheManager() { return manager; } - public CacheManager getSharedManager() { - return sharedManager; - } - private net.sf.ehcache.CacheManager getEhcacheManager(String name) { - if (name != null && name.equals(Cache.TYPE_VIS_SHARED_MEM)) { - return sharedManager; - } else { - return manager; - } + return manager; } @@ -158,15 +144,15 @@ private net.sf.ehcache.CacheManager getEhcacheManager(String name) { // inner classes //==================================================================== - static class FileCache extends EhcacheImpl { + public static class FileCache implements Cache { + private final Cache cache; - public FileCache(Ehcache cache) { - super(cache); + public FileCache(Cache cache) { + this.cache = cache; } - @Override public Object get(CacheKey key) { - Object o = super.get(key); + Object o = cache.get(key); if (o==null) return null; else if (o instanceof File) o= exist(key,(File)o); @@ -175,10 +161,16 @@ public Object get(CacheKey key) { return o; } + // pass to delegate + public void put(CacheKey key, Object value) {cache.put(key, value);} + public void put(CacheKey key, Object value, int lifespanInSecs) {cache.put(key, value, lifespanInSecs);} + public boolean isCached(CacheKey key) {return cache.isCached(key);} + public int getSize() {return cache.getSize();} + public List getKeys() {return cache.getKeys();} private Object exist(CacheKey key, File f) { if (f != null && !f.exists()){ - super.cache.remove(key); + cache.put(key, null); // this will remove it f= null; } return f; @@ -191,7 +183,6 @@ private Object exist(CacheKey key, FileInfo f) { } return retval; } - } static class LoggingEventListener implements CacheEventListener { diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/cache/KeyBasedCache.java b/src/firefly/java/edu/caltech/ipac/firefly/server/cache/LocalMapCache.java similarity index 81% rename from src/firefly/java/edu/caltech/ipac/firefly/server/cache/KeyBasedCache.java rename to src/firefly/java/edu/caltech/ipac/firefly/server/cache/LocalMapCache.java index 3a65319050..c6d0d3d975 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/cache/KeyBasedCache.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/cache/LocalMapCache.java @@ -3,7 +3,6 @@ */ package edu.caltech.ipac.firefly.server.cache; -import edu.caltech.ipac.firefly.server.ServerContext; import edu.caltech.ipac.util.cache.Cache; import edu.caltech.ipac.util.cache.CacheKey; import edu.caltech.ipac.util.cache.CacheManager; @@ -21,14 +20,14 @@ * @author loi * @version $Id: UserCache.java,v 1.5 2009/03/23 23:55:16 loi Exp $ */ -public class KeyBasedCache implements Cache { +public class LocalMapCache implements Cache { private Cache cache; private StringKey uniqueKey; - public KeyBasedCache(String key) { + public LocalMapCache(String key) { uniqueKey = new StringKey(key); - cache = CacheManager.getCache(Cache.TYPE_HTTP_SESSION); + cache = CacheManager.getLocal(); } public StringKey getUserKey() { @@ -74,15 +73,6 @@ public int getSize() { return getSessionMap().size(); } - public static boolean exists(StringKey userKey) { - Cache cache = CacheManager.getCache(Cache.TYPE_HTTP_SESSION); - return cache.isCached(userKey); - } - - public static void create(StringKey userKey) { - Cache cache = CacheManager.getCache(Cache.TYPE_HTTP_SESSION); - cache.put(userKey, null); - } //==================================================================== // //==================================================================== diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/cache/PrivateCache.java b/src/firefly/java/edu/caltech/ipac/firefly/server/cache/PrivateCache.java deleted file mode 100644 index 9d669b3cec..0000000000 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/cache/PrivateCache.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * License information at https://github.com/Caltech-IPAC/firefly/blob/master/License.txt - */ -package edu.caltech.ipac.firefly.server.cache; - -import edu.caltech.ipac.util.cache.Cache; -import edu.caltech.ipac.util.cache.CacheKey; -import edu.caltech.ipac.util.cache.StringKey; - -import java.util.List; - -/** - * this is a cache created from a public cache with a private key. - * all objects stored in this cache will have the private key appended to it. - * access to this cache require the knowledge of the private key. - * this is useful when storing user's specific information without session constraint. - * the expiration or eviction policy is based on the backed cache, and not based on session. - * the private key in this case is the user's key. - * - * @author loi - * @version $Id: UserCache.java,v 1.5 2009/03/23 23:55:16 loi Exp $ - */ -public class PrivateCache implements Cache { - - private Cache cache; - private StringKey privateKey; - - public PrivateCache(String privateKey, Cache cache) { - this.cache = cache; - this.privateKey = new StringKey(privateKey); - } - - public void put(CacheKey key, Object value) { - cache.put(getPrivateKey(key), value); - } - - public void put(CacheKey key, Object value, int lifespanInSecs) { - cache.put(getPrivateKey(key), value, lifespanInSecs); - } - - public Object get(CacheKey key) { - return cache.get(getPrivateKey(key)); - } - - public boolean isCached(CacheKey key) { - return cache.isCached(getPrivateKey(key)); - } - - public List getKeys() { - throw new UnsupportedOperationException( - "For performance reason, this operation is not supported."); - } - - public int getSize() { - throw new UnsupportedOperationException( - "For performance reason, this operation is not supported."); - } - - private StringKey getPrivateKey(CacheKey key) { - return new StringKey(privateKey).appendToKey(key); - } -} diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/cache/UserCache.java b/src/firefly/java/edu/caltech/ipac/firefly/server/cache/UserCache.java index cc9d3e2932..d851f14488 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/cache/UserCache.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/cache/UserCache.java @@ -3,18 +3,13 @@ */ package edu.caltech.ipac.firefly.server.cache; +import edu.caltech.ipac.firefly.core.RedisService; import edu.caltech.ipac.firefly.server.ServerContext; import edu.caltech.ipac.util.cache.Cache; -import edu.caltech.ipac.util.cache.CacheKey; -import edu.caltech.ipac.util.cache.CacheManager; import edu.caltech.ipac.util.cache.StringKey; +import redis.clients.jedis.Jedis; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; +import static edu.caltech.ipac.firefly.server.RequestOwner.USER_KEY_EXPIRY; /** * Date: Jul 21, 2008 @@ -22,13 +17,20 @@ * @author loi * @version $Id: UserCache.java,v 1.5 2009/03/23 23:55:16 loi Exp $ */ -public class UserCache extends KeyBasedCache { +public class UserCache extends DistribMapCache { public static Cache getInstance(){ return new UserCache(); } private UserCache() { - super(ServerContext.getRequestOwner().getUserKey()); + super(ServerContext.getRequestOwner().getUserKey(), USER_KEY_EXPIRY); + } + + public static boolean exists(StringKey userKey) { + try(Jedis redis = RedisService.getConnection()) { + return redis.exists(userKey.getUniqueString()); + } catch (Exception ex) { LOG.error(ex); } + return false; } } diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/db/BaseDbAdapter.java b/src/firefly/java/edu/caltech/ipac/firefly/server/db/BaseDbAdapter.java index 9005007a35..3d8cf83e68 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/db/BaseDbAdapter.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/db/BaseDbAdapter.java @@ -3,6 +3,7 @@ */ package edu.caltech.ipac.firefly.server.db; +import edu.caltech.ipac.firefly.core.Util; import edu.caltech.ipac.firefly.data.FileInfo; import edu.caltech.ipac.firefly.data.SortInfo; import edu.caltech.ipac.firefly.data.TableServerRequest; @@ -65,6 +66,7 @@ abstract public class BaseDbAdapter implements DbAdapter { //==================================================================== public BaseDbAdapter(File dbFile) { this.dbFile = dbFile; } + public File getDbFile() { return dbFile; } public DbInstance getDbInstance() { @@ -730,7 +732,7 @@ protected Object[] getDdFrom(DataType dt, int colIdx) { dt.getRef(), dt.getMaxValue(), dt.getMinValue(), - serialize(dt.getLinkInfos()), // index(21) is used in HsqlDbAdapter. if it changes, update. + Util.serialize(dt.getLinkInfos()), // index(21) is used in HsqlDbAdapter. if it changes, update. dt.getDataOptions(), dt.getArraySize(), dt.getCellRenderer(), @@ -908,10 +910,10 @@ protected Object[] getAuxFrom(DataGroup dg) { return new Object[] { dg.getTitle(), dg.size(), - serialize(dg.getGroupInfos()), - serialize(dg.getLinkInfos()), - serialize(dg.getParamInfos()), - serialize(dg.getResourceInfos()) + Util.serialize(dg.getGroupInfos()), + Util.serialize(dg.getLinkInfos()), + Util.serialize(dg.getParamInfos()), + Util.serialize(dg.getResourceInfos()) }; } diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/db/DbInstance.java b/src/firefly/java/edu/caltech/ipac/firefly/server/db/DbInstance.java index 7df8dee851..f4710aaaa0 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/db/DbInstance.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/db/DbInstance.java @@ -109,7 +109,7 @@ public String getDbUrl() { public void setPooled(boolean pooled) { isPooled = pooled;} public boolean testConn(Connection conn) { try (Statement stmt = conn.createStatement()) { - stmt.execute("SELECT 1 FROM (VALUES (0))"); // works with all DMBS + stmt.execute("SELECT 1"); // works with all DMBS return true; } catch (SQLException e) { return false; } }; diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/db/DuckDbAdapter.java b/src/firefly/java/edu/caltech/ipac/firefly/server/db/DuckDbAdapter.java index 47560a86cf..cf80c612e1 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/db/DuckDbAdapter.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/db/DuckDbAdapter.java @@ -3,6 +3,7 @@ */ package edu.caltech.ipac.firefly.server.db; +import edu.caltech.ipac.firefly.core.Util; import edu.caltech.ipac.firefly.data.TableServerRequest; import edu.caltech.ipac.firefly.server.ServerContext; import edu.caltech.ipac.firefly.server.db.spring.JdbcFactory; @@ -188,7 +189,7 @@ protected int createDataTbl(DataGroup dg, String tblName) throws DataAccessExcep List aryIdx = colIdxWithArrayData(colsAry); for (int r = 0; r < totalRows; r++) { Object[] row = dg.get(r).getData(); - aryIdx.forEach(idx -> row[idx] = serialize(row[idx])); // serialize array data if necessary + aryIdx.forEach(idx -> row[idx] = Util.serialize(row[idx])); // serialize array data if necessary addRow(appender, row, r); } appender.flush(); diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/db/EmbeddedDbUtil.java b/src/firefly/java/edu/caltech/ipac/firefly/server/db/EmbeddedDbUtil.java index c3af3f1c76..ab30369b14 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/db/EmbeddedDbUtil.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/db/EmbeddedDbUtil.java @@ -3,6 +3,7 @@ */ package edu.caltech.ipac.firefly.server.db; +import edu.caltech.ipac.firefly.core.Util; import edu.caltech.ipac.firefly.data.ServerEvent; import edu.caltech.ipac.firefly.data.ServerRequest; import edu.caltech.ipac.firefly.data.TableServerRequest; @@ -26,14 +27,9 @@ import org.springframework.jdbc.core.JdbcTemplate; import javax.validation.constraints.NotNull; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; import java.sql.*; import java.time.LocalDate; import java.time.LocalDateTime; -import java.time.ZoneOffset; import java.util.*; import java.util.Date; import java.util.stream.Collectors; @@ -54,6 +50,10 @@ import static java.time.ZoneOffset.UTC; /** + * Using duckdb appender greatly improve performance when ingesting large volume of data. + * But, direct BLOB support is not available. Therefore, we will serialize Java object + * into base64 string for storage. + * * @author loi * @version $Id: DbInstance.java,v 1.3 2012/03/15 20:35:40 loi Exp $ */ @@ -158,7 +158,7 @@ private static Object convertToType(Class clz, ResultSet rs, int idx, boolean is if (isAry) { if (val instanceof Array) return val; // we will assume the data type matches - return deserialize(val.toString()); // handles base64 encoded Java serialized objects + return Util.deserialize(val.toString()); // handles base64 encoded Java serialized objects } else if (clz == String.class) { if (val instanceof Blob b) { return new String(b.getBytes(1, (int) b.length()), UTF_8); // handles binary UTF-8 encoded string @@ -412,42 +412,12 @@ public static void dbToDataType(DataType dtype, ResultSet rs) { // But, direct BLOB support is not available. Therefore, we will serialize Java object // into base64 string for storage. //==================================================================== - - public static String serialize(Object obj) { - if (obj == null) return null; - try { - ByteArrayOutputStream bstream = new ByteArrayOutputStream(); - ObjectOutputStream ostream = new ObjectOutputStream(bstream); - ostream.writeObject(obj); - ostream.flush(); - byte[] bytes = bstream.toByteArray(); - return Base64.getEncoder().encodeToString(bytes); - } catch (Exception e) { - logger.warn(e); - return null; - } - } - public static Object deserialize(ResultSet rs, String cname) { - return getSafe(() -> deserialize(rs.getString(cname))); + return getSafe(() -> Util.deserialize(rs.getString(cname))); } public static Object deserialize(ResultSet rs, int cidx) { - return getSafe(() -> deserialize(rs.getString(cidx))); + return getSafe(() -> Util.deserialize(rs.getString(cidx))); } - - public static Object deserialize(String base64) { - try { - if (base64 == null) return null; - byte[] bytes = Base64.getDecoder().decode(base64); - ByteArrayInputStream bstream = new ByteArrayInputStream(bytes); - ObjectInputStream ostream = new ObjectInputStream(bstream); - return ostream.readObject(); - } catch (Exception e) { - logger.warn(e); - return null; - } - } - //==================================================================== // privates functions //==================================================================== @@ -533,7 +503,7 @@ public void setValues(PreparedStatement ps, int i) throws SQLException { for (int cidx = 0; cidx < cols.length; cidx++) { Object v = data.getData(cols[cidx].getKeyName(), ridx); if (cIsAry.get(cidx)) { - v = serialize(v); + v = Util.serialize(v); } ps.setObject(cidx+1, v); } diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/db/HsqlDbAdapter.java b/src/firefly/java/edu/caltech/ipac/firefly/server/db/HsqlDbAdapter.java index 2f879b44b0..744d078309 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/db/HsqlDbAdapter.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/db/HsqlDbAdapter.java @@ -138,7 +138,7 @@ public DataAccessException handleSqlExp(String msg, Exception e) { } } if (e instanceof DataAccessException dax) { - return new DataAccessException(msg, dax.getCause()); + return new DataAccessException(msg, dax.getCause() == null ? dax : dax.getCause()); // when the cause is a DataAccessException, don't need to interpret. } return new DataAccessException(msg, e); } diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/db/spring/JdbcFactory.java b/src/firefly/java/edu/caltech/ipac/firefly/server/db/spring/JdbcFactory.java index 554f739d2c..c2801d296d 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/db/spring/JdbcFactory.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/db/spring/JdbcFactory.java @@ -22,7 +22,7 @@ import java.util.Map; import java.util.Properties; -import static edu.caltech.ipac.util.StringUtils.getWith; +import static edu.caltech.ipac.util.StringUtils.getIf; /** * Date: Oct 7, 2008 @@ -40,7 +40,7 @@ public class JdbcFactory { * @return */ public static JdbcTemplate getTemplate(DbInstance dbInstance) { - DataSource datasource = getWith( () -> getDataSource(dbInstance), (ds) -> ds != null,3); // return null if failed after 3 tries + DataSource datasource = getIf( () -> getDataSource(dbInstance), (ds) -> ds != null,3); // return null if failed after 3 tries return datasource == null ? null : new JdbcTemplate(datasource); } @@ -65,7 +65,7 @@ public static TransactionTemplate getTransactionTemplate(DataSource dataSource) * @return */ public static SimpleJdbcTemplate getSimpleTemplate(DbInstance dbInstance) { - DataSource datasource = getWith( () -> getDataSource(dbInstance), (ds) -> ds != null,3); // return null if failed after 3 tries + DataSource datasource = getIf( () -> getDataSource(dbInstance), (ds) -> ds != null,3); // return null if failed after 3 tries return datasource == null ? null : new SimpleJdbcTemplate(datasource); } diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/events/CacheEventWorker.java b/src/firefly/java/edu/caltech/ipac/firefly/server/events/CacheEventWorker.java index c8dc69b27f..95c1a93f3c 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/events/CacheEventWorker.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/events/CacheEventWorker.java @@ -20,9 +20,6 @@ import net.sf.ehcache.event.CacheEventListener; import net.sf.ehcache.event.NotificationScope; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; - import static edu.caltech.ipac.firefly.server.ServerContext.SHORT_TASK_EXEC; /** @@ -30,8 +27,7 @@ */ public class CacheEventWorker implements ServerEventManager.EventWorker { - private static final String EVENT_SENDING_CACHE= Cache.TYPE_PERM_SMALL; - private static final Cache cache= CacheManager.getCache(EVENT_SENDING_CACHE); + private static final Cache cache= CacheManager.getDistributed(); public CacheEventWorker() { diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/events/ReplicatedQueueList.java b/src/firefly/java/edu/caltech/ipac/firefly/server/events/ReplicatedQueueList.java index ffe3d36bc1..6c09abb3f4 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/events/ReplicatedQueueList.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/events/ReplicatedQueueList.java @@ -22,7 +22,7 @@ class ReplicatedQueueList { private static final String HOST_NAME= FileUtil.getHostname(); private static final StringKey REP_QUEUE_MAP = new StringKey("ReplicatedEventQueueMap"); - private static Cache getCache() { return CacheManager.getCache(Cache.TYPE_PERM_SMALL); } + private static Cache getCache() { return CacheManager.getLocal(); } synchronized void setQueueListForNode(List list) { Cache cache= getCache(); diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/persistence/GuestHistoryCache.java b/src/firefly/java/edu/caltech/ipac/firefly/server/persistence/GuestHistoryCache.java index 8206b24e37..33098f06a6 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/persistence/GuestHistoryCache.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/persistence/GuestHistoryCache.java @@ -85,7 +85,7 @@ public static void updateSearch(final String userKey,int searchId, boolean isFav } public static List getSearchHistory(String userKey) { - Cache cache = CacheManager.getCache(Cache.TYPE_PERM_SMALL); + Cache cache = CacheManager.getLocal(); List searchHistory = (List) cache.get(new StringKey("GuestHistoryCache", userKey)); return searchHistory == null ? new ArrayList() : searchHistory; } @@ -96,7 +96,7 @@ public static List getSearchHistory() { } static void updateSearchHistory(String userKey, List searchHistory) { - Cache cache = CacheManager.getCache(Cache.TYPE_PERM_SMALL); + Cache cache = CacheManager.getLocal(); cache.put(new StringKey("GuestHistoryCache", userKey), searchHistory, LIFE_TO_LIVE); } diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/persistence/QueryIBE.java b/src/firefly/java/edu/caltech/ipac/firefly/server/persistence/QueryIBE.java index f660c70a5b..d4eaa43378 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/persistence/QueryIBE.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/persistence/QueryIBE.java @@ -11,7 +11,6 @@ import edu.caltech.ipac.astro.ibe.IBE; import edu.caltech.ipac.astro.ibe.IbeDataSource; import edu.caltech.ipac.astro.ibe.IbeQueryParam; -import edu.caltech.ipac.firefly.data.ServerRequest; import edu.caltech.ipac.firefly.data.SortInfo; import edu.caltech.ipac.firefly.data.TableServerRequest; import edu.caltech.ipac.firefly.data.table.MetaConst; @@ -102,7 +101,7 @@ private void addAddtlMeta(TableMeta meta, List columns, TableServerReq IBE ibe = IBEUtils.getIBE(mission, paramMap); IbeDataSource source = ibe.getIbeDataSource(); CacheKey cacheKey = new StringKey("ibemeta", source.getIbeHost(), source.getMission(), source.getDataset(), source.getTableName()); - Cache cache = CacheManager.getCache(Cache.TYPE_PERM_SMALL); + Cache cache = CacheManager.getLocal(); DataGroup coldefs = (DataGroup) cache.get(cacheKey); if (coldefs == null) { File ofile = File.createTempFile(mission+"-dd", ".tbl", QueryUtil.getTempDir(request)); diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/query/BaseFileInfoProcessor.java b/src/firefly/java/edu/caltech/ipac/firefly/server/query/BaseFileInfoProcessor.java index c8e105bb19..37224fb188 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/query/BaseFileInfoProcessor.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/query/BaseFileInfoProcessor.java @@ -10,14 +10,12 @@ import edu.caltech.ipac.firefly.server.ServerContext; import edu.caltech.ipac.firefly.server.util.Logger; import edu.caltech.ipac.table.DataType; -import edu.caltech.ipac.table.TableUtil; import edu.caltech.ipac.util.StringUtils; import edu.caltech.ipac.util.cache.Cache; import edu.caltech.ipac.util.cache.CacheManager; import edu.caltech.ipac.util.cache.StringKey; import java.io.IOException; -import java.io.OutputStream; import java.util.List; @@ -55,7 +53,7 @@ public FileInfo getData(ServerRequest request) throws DataAccessException { } public Cache getCache(ServerRequest request) { - return CacheManager.getCache(Cache.TYPE_PERM_SMALL); + return CacheManager.getLocal(); } public QueryDescResolver getDescResolver() { diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/query/FileGroupsProcessor.java b/src/firefly/java/edu/caltech/ipac/firefly/server/query/FileGroupsProcessor.java index a1c5834123..fa23d3ac6a 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/query/FileGroupsProcessor.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/query/FileGroupsProcessor.java @@ -19,7 +19,6 @@ import java.io.File; import java.io.IOException; -import java.io.OutputStream; import java.net.MalformedURLException; import java.net.URL; import java.util.List; @@ -42,7 +41,7 @@ public List getData(ServerRequest sr) throws DataAccessException { DownloadRequest request= (DownloadRequest)sr; List fileGroups = null; StringKey key = new StringKey(FileGroupsProcessor.class.getName(), getUniqueID(request)); - Cache cache = CacheManager.getCache(Cache.TYPE_TEMP_FILE); + Cache cache = CacheManager.getLocalFile(); fileGroups = (List) cache.get(key); if (fileGroups == null || isStaled(fileGroups)) { diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/query/IpacTablePartProcessor.java b/src/firefly/java/edu/caltech/ipac/firefly/server/query/IpacTablePartProcessor.java index a04d87c5e3..b2db3861f9 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/query/IpacTablePartProcessor.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/query/IpacTablePartProcessor.java @@ -14,7 +14,6 @@ import edu.caltech.ipac.firefly.data.TableServerRequest; import edu.caltech.ipac.firefly.server.Counters; import edu.caltech.ipac.firefly.server.ServerContext; -import edu.caltech.ipac.firefly.server.cache.PrivateCache; import edu.caltech.ipac.firefly.server.network.HttpServiceInput; import edu.caltech.ipac.firefly.server.util.Logger; import edu.caltech.ipac.firefly.server.util.QueryUtil; @@ -133,7 +132,7 @@ public DataGroupPart getData(ServerRequest sr) throws DataAccessException { File dgFile = null; try { TableServerRequest request = (TableServerRequest) sr; - Cache cache = CacheManager.getCache(Cache.TYPE_TEMP_FILE); + Cache cache = CacheManager.getLocalFile(); // get unique key without page info StringKey key = new StringKey(this.getClass().getName(), getDataKey(request)); @@ -313,7 +312,7 @@ public FileInfo writeData(OutputStream out, ServerRequest sr, FormatUtil.Format public File getDataFile(TableServerRequest request) throws IpacTableException, IOException, DataAccessException { LOGGER.warn("<< slow getDataFile called." + this.getClass().getSimpleName()); - Cache cache = CacheManager.getCache(Cache.TYPE_TEMP_FILE); + Cache cache = CacheManager.getLocalFile(); // if decimation or sorting is requested, you cannot background writing the file to speed up response time. boolean noBgWrite = DecimationProcessor.getDecimateInfo(request) != null || request.getSortInfo() != null; @@ -440,7 +439,7 @@ protected void doSort(File inFile, File outFile, SortInfo sortInfo, TableServerR //==================================================================== protected Cache getCache() { - return new PrivateCache(ServerContext.getRequestOwner().getUserKey(), CacheManager.getCache(Cache.TYPE_PERM_FILE)); + return CacheManager.getLocalMap(ServerContext.getRequestOwner().getUserKey()); } /** diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/query/JsonStringProcessor.java b/src/firefly/java/edu/caltech/ipac/firefly/server/query/JsonStringProcessor.java index 6e71c84de1..f52838a392 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/query/JsonStringProcessor.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/query/JsonStringProcessor.java @@ -9,7 +9,6 @@ import edu.caltech.ipac.firefly.server.ServerContext; import edu.caltech.ipac.firefly.server.util.Logger; import edu.caltech.ipac.firefly.server.util.QueryUtil; -import edu.caltech.ipac.firefly.server.util.StopWatch; import edu.caltech.ipac.table.DataType; import edu.caltech.ipac.table.TableMeta; import edu.caltech.ipac.util.FileUtil; @@ -109,7 +108,7 @@ protected void cacheData(ServerRequest request, String results) { try { jsonFile = File.createTempFile("tmp-", ".json", QueryUtil.getTempDir()); FileUtil.writeStringToFile(jsonFile, results); - CacheManager.getCache(Cache.TYPE_TEMP_FILE) + CacheManager.getLocalFile() .put(new StringKey(getUniqueID(request)), jsonFile); } catch (IOException e) { LOGGER.error("Cannot create temp file: " + e.getMessage()); @@ -118,7 +117,7 @@ protected void cacheData(ServerRequest request, String results) { } protected String getCachedData(ServerRequest request) { - Cache cache = CacheManager.getCache(Cache.TYPE_TEMP_FILE); + Cache cache = CacheManager.getLocalFile(); File jsonFile = (File)cache.get(new StringKey(getUniqueID(request))); if (jsonFile != null) { try { diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/query/TableResultSearch.java b/src/firefly/java/edu/caltech/ipac/firefly/server/query/TableResultSearch.java index f30d94f99c..38bf5dbeee 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/query/TableResultSearch.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/query/TableResultSearch.java @@ -159,7 +159,7 @@ public static void addToCache(MultiPartData data) { } public static Cache getParamCache() { - Cache cache= CacheManager.getCache(Cache.TYPE_PERM_SMALL); + Cache cache= CacheManager.getLocal(); return cache; } diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/query/TemplateGenerator.java b/src/firefly/java/edu/caltech/ipac/firefly/server/query/TemplateGenerator.java index 619247a4b7..0deb0a5830 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/query/TemplateGenerator.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/query/TemplateGenerator.java @@ -40,7 +40,7 @@ public static DataGroup generate(String templateName, String querySql, DataSourc } try { CacheKey cacheKey = new StringKey("TemplateGenerator", java.lang.System.currentTimeMillis() ); - Cache cache = CacheManager.getCache(Cache.TYPE_PERM_SMALL); + Cache cache = CacheManager.getLocal(); DataGroup template = (DataGroup) cache.get(cacheKey); if (template == null) { template = loadTemplate(templateName, dataSource); diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/query/UserCatalogQuery.java b/src/firefly/java/edu/caltech/ipac/firefly/server/query/UserCatalogQuery.java index 25991532fc..12c0967676 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/query/UserCatalogQuery.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/query/UserCatalogQuery.java @@ -16,6 +16,7 @@ import edu.caltech.ipac.table.TableMeta; import edu.caltech.ipac.table.TableUtil; import edu.caltech.ipac.util.StringUtils; +import edu.caltech.ipac.util.cache.CacheManager; import edu.caltech.ipac.util.cache.StringKey; import java.io.File; @@ -57,7 +58,7 @@ protected File loadDataFile(TableServerRequest req) throws IOException, DataAcce public void prepareTableMeta(TableMeta meta, List columns, ServerRequest request) { String filePath= request.getParam("filePath"); if (filePath!=null) { - UploadFileInfo uFi=(UploadFileInfo)UserCache.getInstance().get(new StringKey(filePath)); + UploadFileInfo uFi=(UploadFileInfo) CacheManager.getUserCache().get(new StringKey(filePath)); if (uFi!=null) { meta.setAttribute("title", uFi.getFileName()); } diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/servlets/AnyFileDownload.java b/src/firefly/java/edu/caltech/ipac/firefly/server/servlets/AnyFileDownload.java index 3c8a38b81f..e08a07882a 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/servlets/AnyFileDownload.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/servlets/AnyFileDownload.java @@ -6,11 +6,11 @@ import edu.caltech.ipac.firefly.data.FileInfo; import edu.caltech.ipac.firefly.server.ServerContext; import edu.caltech.ipac.firefly.server.SrvParam; -import edu.caltech.ipac.firefly.server.cache.UserCache; import edu.caltech.ipac.firefly.server.util.Logger; import edu.caltech.ipac.util.FileUtil; import edu.caltech.ipac.util.StringUtils; import edu.caltech.ipac.util.cache.Cache; +import edu.caltech.ipac.util.cache.CacheManager; import edu.caltech.ipac.util.download.FailedRequestException; import edu.caltech.ipac.util.download.URLDownload; @@ -223,6 +223,6 @@ private static void logActivity(File f) { "u", FileUtil.getSizeAsString(f.length()), "file", f.getPath()); } - public static Cache getCache() { return UserCache.getInstance(); } + public static Cache getCache() { return CacheManager.getUserCache(); } } diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/servlets/AnyFileUpload.java b/src/firefly/java/edu/caltech/ipac/firefly/server/servlets/AnyFileUpload.java index e3be9d2e21..c1403ee36e 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/servlets/AnyFileUpload.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/servlets/AnyFileUpload.java @@ -10,7 +10,6 @@ import edu.caltech.ipac.firefly.server.Counters; import edu.caltech.ipac.firefly.server.ServerContext; import edu.caltech.ipac.firefly.server.SrvParam; -import edu.caltech.ipac.firefly.server.cache.UserCache; import edu.caltech.ipac.firefly.server.util.StopWatch; import edu.caltech.ipac.firefly.server.util.multipart.UploadFileInfo; import edu.caltech.ipac.firefly.server.visualize.LockingVisNetwork; @@ -23,6 +22,7 @@ import edu.caltech.ipac.firefly.visualize.WebPlotRequest; import edu.caltech.ipac.util.FileUtil; import edu.caltech.ipac.util.StringUtils; +import edu.caltech.ipac.util.cache.CacheManager; import edu.caltech.ipac.util.cache.StringKey; import edu.caltech.ipac.util.download.FailedRequestException; import org.apache.commons.fileupload.FileItemIterator; @@ -140,7 +140,7 @@ public static void doFileUpload(HttpServletRequest req, HttpServletResponse res) // save info in a cache for downstream use String fileCacheKey = sp.getOptional(CACHE_KEY); fileCacheKey = fileCacheKey == null ? uploadFileInfo.getPname() : fileCacheKey; - UserCache.getInstance().put(new StringKey(fileCacheKey), uploadFileInfo); + CacheManager.getUserCache().put(new StringKey(fileCacheKey), uploadFileInfo); // returns the fileCacheKey or full analysis json String returnVal= analyzeFile ? callAnalysis(sp,statusFileInfo,uploadFileInfo,fileCacheKey) : fileCacheKey; diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/servlets/MultiPartHandler.java b/src/firefly/java/edu/caltech/ipac/firefly/server/servlets/MultiPartHandler.java index 594e8a563e..171e3021f7 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/servlets/MultiPartHandler.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/servlets/MultiPartHandler.java @@ -7,6 +7,7 @@ import edu.caltech.ipac.firefly.server.util.Logger; import edu.caltech.ipac.firefly.server.util.multipart.MultiPartData; import edu.caltech.ipac.util.StringUtils; +import edu.caltech.ipac.util.cache.CacheManager; import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; @@ -35,7 +36,7 @@ protected void processRequest(HttpServletRequest req, HttpServletResponse res) t try { MultiPartData data= MultipartDataUtil.handleRequest(req); - UserCache.getInstance().put(data.getCacheKey(), data); + CacheManager.getUserCache().put(data.getCacheKey(), data); LOG.info("Multipart request processed.", "File(s) uploaded: " + StringUtils.toString(data.getFiles()), "Form parameters : " + data.getParams()); diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/servlets/MultipartDataUtil.java b/src/firefly/java/edu/caltech/ipac/firefly/server/servlets/MultipartDataUtil.java index 314e1f4b07..18e97790a6 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/servlets/MultipartDataUtil.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/servlets/MultipartDataUtil.java @@ -9,7 +9,6 @@ */ -import edu.caltech.ipac.firefly.server.ServerContext; import edu.caltech.ipac.firefly.server.util.QueryUtil; import edu.caltech.ipac.firefly.server.util.multipart.MultiPartData; import edu.caltech.ipac.util.cache.Cache; @@ -64,7 +63,7 @@ public static MultiPartData handleRequest(StringKey key, HttpServletRequest req) item.write(uf); data.addFile(fieldName, uf, fileName, contentType); StringKey fileKey= new StringKey(fileName, System.currentTimeMillis()); - CacheManager.getCache(Cache.TYPE_TEMP_FILE).put(fileKey, uf); + CacheManager.getLocalFile().put(fileKey, uf); } } return data; diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/servlets/ServerStatus.java b/src/firefly/java/edu/caltech/ipac/firefly/server/servlets/ServerStatus.java index 92c068d849..8a1d70e5e5 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/servlets/ServerStatus.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/servlets/ServerStatus.java @@ -3,6 +3,7 @@ */ package edu.caltech.ipac.firefly.server.servlets; +import edu.caltech.ipac.firefly.core.RedisService; import edu.caltech.ipac.firefly.core.background.JobManager; import edu.caltech.ipac.firefly.messaging.Messenger; import edu.caltech.ipac.firefly.server.Counters; @@ -97,7 +98,6 @@ protected void processRequest(HttpServletRequest req, HttpServletResponse res) t EhcacheProvider prov = (EhcacheProvider) edu.caltech.ipac.util.cache.CacheManager.getCacheProvider(); displayCacheInfo(writer, prov.getEhcacheManager(), sInfo); - displayCacheInfo(writer, prov.getSharedManager(), sInfo); if (showHeaders) { skip(writer); @@ -307,8 +307,10 @@ private static String makeQueueList() { } private static void showMessagingStatus(PrintWriter w) { - w.println("Messenger: Redis host: " + Messenger.getRedisHostPortDesc()); - w.println("Messaging Pool: " + Messenger.getStats()); + w.println("Redis information: "); + w.println("----------------- "); + Map stats = RedisService.getStats(); + stats.forEach((k,v)-> w.println(" - " + k + ": " + v)); } private static void showPackagingStatus(PrintWriter w, boolean details) { diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/visualize/FitsCacher.java b/src/firefly/java/edu/caltech/ipac/firefly/server/visualize/FitsCacher.java index 0bb116333c..ae5e7c1985 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/visualize/FitsCacher.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/visualize/FitsCacher.java @@ -33,8 +33,8 @@ */ public class FitsCacher { - private static final Cache memCache= CacheManager.getCache(Cache.TYPE_VIS_SHARED_MEM); - private static final Cache fileInfoCache= CacheManager.getCache(Cache.TYPE_PERM_SMALL); + private static final Cache memCache= CacheManager.getVisMemCache(); + private static final Cache fileInfoCache= CacheManager.getLocal(); private static final Map activeRequest = new ConcurrentHashMap<>(61); private static final Logger.LoggerImpl _log = Logger.getLogger(); diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/visualize/PlotServUtils.java b/src/firefly/java/edu/caltech/ipac/firefly/server/visualize/PlotServUtils.java index 8e0a134b4d..d0d5c14125 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/visualize/PlotServUtils.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/visualize/PlotServUtils.java @@ -14,6 +14,7 @@ import edu.caltech.ipac.util.StringUtils; import edu.caltech.ipac.util.cache.Cache; import edu.caltech.ipac.util.cache.CacheKey; +import edu.caltech.ipac.util.cache.CacheManager; import edu.caltech.ipac.util.cache.StringKey; import edu.caltech.ipac.visualize.plot.ActiveFitsReadGroup; import edu.caltech.ipac.visualize.plot.Circle; @@ -44,7 +45,7 @@ public class PlotServUtils { public static final String PROCESSING_COMPLETED_MSG = "Processing Images Completed"; public static void updateProgress(ProgressStat pStat) { - Cache cache= UserCache.getInstance(); + Cache cache= CacheManager.getUserCache(); CacheKey key= new StringKey(pStat.getId()); ProgressStat lastPstat= (ProgressStat) cache.get(key); boolean fireAction= true; @@ -292,14 +293,14 @@ static ProgressMessage getPlotProgressMessage(ProgressStat stat) { private static ProgressMessage getSingleStatusMessage(String key) { - ProgressStat stat = (ProgressStat) UserCache.getInstance().get(new StringKey(key)); + ProgressStat stat = (ProgressStat) CacheManager.getUserCache().get(new StringKey(key)); if (stat != null) return new ProgressMessage(stat.getMessage(), stat.isDone()); return EMPTY_MESSAGE; } private static ProgressMessage getMultiStatMessage(ProgressStat stat) { ProgressMessage retval; - Cache cache = UserCache.getInstance(); + Cache cache = CacheManager.getUserCache(); List keyList = stat.getMemberIDList(); ProgressStat statEntry; diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/visualize/VisServerOps.java b/src/firefly/java/edu/caltech/ipac/firefly/server/visualize/VisServerOps.java index 064e3e2771..c0e23f2b30 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/visualize/VisServerOps.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/visualize/VisServerOps.java @@ -194,7 +194,7 @@ public static byte[] getByteStretchArray(PlotState state, int tileSize, boolean DirectStretchUtils.StretchDataInfo data; try { ActiveFitsReadGroup frGroup= CtxControl.prepare(state); - Cache memCache= CacheManager.getCache(Cache.TYPE_VIS_SHARED_MEM); + Cache memCache= CacheManager.getVisMemCache(); CacheKey stretchDataKey= new StringKey(state.getContextString()+"byte-data"); data= (StretchDataInfo)memCache.get(stretchDataKey); String fromCache= ""; @@ -371,13 +371,13 @@ public static WebPlotResult saveDS9RegionFile(String regionData) { private static File getRegFile(String fileKey) { File regFile = ServerContext.convertToFile(fileKey); if (regFile != null && regFile.canRead()) return regFile; - UploadFileInfo tmp = (UploadFileInfo) (UserCache.getInstance().get(new StringKey(fileKey))); + UploadFileInfo tmp = (UploadFileInfo) (CacheManager.getUserCache().get(new StringKey(fileKey))); return tmp !=null ? tmp.getFile() : new File(""); } private static String getRegTitle(String fileKey) { File f= getRegFile(fileKey); - UploadFileInfo fi = (UploadFileInfo) UserCache.getInstance().get(new StringKey(fileKey)); + UploadFileInfo fi = (UploadFileInfo) CacheManager.getUserCache().get(new StringKey(fileKey)); return (fi!=null) ? fi.getFileName() : fileKey.startsWith("UPLOAD") ? "Region file" : f.getName(); } diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/visualize/imageretrieve/LocalFileRetriever.java b/src/firefly/java/edu/caltech/ipac/firefly/server/visualize/imageretrieve/LocalFileRetriever.java index 7153007b61..d2f555b56f 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/visualize/imageretrieve/LocalFileRetriever.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/visualize/imageretrieve/LocalFileRetriever.java @@ -14,6 +14,7 @@ import edu.caltech.ipac.firefly.visualize.WebPlotRequest; import edu.caltech.ipac.util.StringUtils; import edu.caltech.ipac.util.cache.Cache; +import edu.caltech.ipac.util.cache.CacheManager; import edu.caltech.ipac.util.cache.StringKey; import edu.caltech.ipac.util.download.FailedRequestException; import edu.caltech.ipac.visualize.plot.plotdata.GeomException; @@ -39,7 +40,7 @@ public FileInfo getFile(WebPlotRequest request) throws FailedRequestException, G public FileInfo getFileByName(String fileName) throws FailedRequestException, GeomException, SecurityException { String fStr= StringUtils.crunch(fileName); if (fStr!=null) { - Cache sessionCache= UserCache.getInstance(); + Cache sessionCache= CacheManager.getUserCache(); UploadFileInfo uFI= (UploadFileInfo)(sessionCache.get(new StringKey(fStr))); File f= ServerContext.convertToFile(fStr); if (f==null || !f.canRead()) { diff --git a/src/firefly/java/edu/caltech/ipac/table/io/TableParseHandler.java b/src/firefly/java/edu/caltech/ipac/table/io/TableParseHandler.java index b700b09513..7d04669338 100644 --- a/src/firefly/java/edu/caltech/ipac/table/io/TableParseHandler.java +++ b/src/firefly/java/edu/caltech/ipac/table/io/TableParseHandler.java @@ -21,7 +21,7 @@ import static edu.caltech.ipac.firefly.server.db.DuckDbAdapter.addRow; import static edu.caltech.ipac.firefly.server.db.EmbeddedDbUtil.colIdxWithArrayData; -import static edu.caltech.ipac.firefly.server.db.EmbeddedDbUtil.serialize; +import static edu.caltech.ipac.firefly.core.Util.serialize; import static edu.caltech.ipac.util.StringUtils.tryIt; /** diff --git a/src/firefly/java/edu/caltech/ipac/util/StringUtils.java b/src/firefly/java/edu/caltech/ipac/util/StringUtils.java index eadea0b592..d8b1b9179b 100644 --- a/src/firefly/java/edu/caltech/ipac/util/StringUtils.java +++ b/src/firefly/java/edu/caltech/ipac/util/StringUtils.java @@ -773,7 +773,7 @@ public static void tryIt(TryWithEx func) { * @param defaultValue returns when encountering exception or test fail * @return the value of func if it passes test. otherwise, return null */ - public static R getWith(FuncWithEx func, Predicate test, R defaultValue) { + public static R getIf(FuncWithEx func, Predicate test, R defaultValue) { try { var result = func.get(); if (test.test(result)) return result; @@ -790,9 +790,9 @@ public static R getWith(FuncWithEx func, Predicate test, R defaultValu * @param tries the number of times to try * @return the value of func if it passes test. otherwise, return null */ - public static R getWith(FuncWithEx func, Predicate test, int tries) { + public static R getIf(FuncWithEx func, Predicate test, int tries) { for (int i = 0; i < tries; i++) { - R v = getWith(func, test, null); + R v = getIf(func, test, null); if (v != null) return v; } return null; diff --git a/src/firefly/java/edu/caltech/ipac/util/cache/Cache.java b/src/firefly/java/edu/caltech/ipac/util/cache/Cache.java index be6f5250e6..5c7e1fc013 100644 --- a/src/firefly/java/edu/caltech/ipac/util/cache/Cache.java +++ b/src/firefly/java/edu/caltech/ipac/util/cache/Cache.java @@ -12,24 +12,6 @@ * @version $Id: Cache.java,v 1.4 2009/06/23 18:57:17 loi Exp $ */ public interface Cache { - /** - * This is a list of cache types each implementing cache may support. The type - * is described using 2 words; the first is length of time it may idle, and the - * second is the type of data. - */ - String TYPE_PERM_SMALL = "PERM_SMALL"; - String TYPE_PERM_FILE = "PERM_FILE"; - String TYPE_TEMP_FILE = "TEMP_FILE"; - String TYPE_VIS_SHARED_MEM = "VIS_SHARED_MEM"; - - /** - * This is used to save User's session information. It is backed - * by UserKey. UserKey may have longer lifespan than - * the session data. Currently, UserKey last for 2 weeks, - * while session only last for 30min. - */ - static final String TYPE_HTTP_SESSION = "HTTP_SESSION"; - void put(CacheKey key, Object value); void put(CacheKey key, Object value, int lifespanInSecs); diff --git a/src/firefly/java/edu/caltech/ipac/util/cache/CacheManager.java b/src/firefly/java/edu/caltech/ipac/util/cache/CacheManager.java index 8a21af4640..ba02ca6fbf 100644 --- a/src/firefly/java/edu/caltech/ipac/util/cache/CacheManager.java +++ b/src/firefly/java/edu/caltech/ipac/util/cache/CacheManager.java @@ -4,11 +4,16 @@ package edu.caltech.ipac.util.cache; import edu.caltech.ipac.firefly.server.ServerContext; -import edu.caltech.ipac.firefly.server.cache.KeyBasedCache; -import edu.caltech.ipac.util.AppProperties; +import edu.caltech.ipac.firefly.server.cache.DistribMapCache; +import edu.caltech.ipac.firefly.server.cache.DistributedCache; +import edu.caltech.ipac.firefly.server.cache.LocalMapCache; +import edu.caltech.ipac.firefly.server.cache.UserCache; +import edu.caltech.ipac.firefly.server.util.Logger; import java.util.List; +import static edu.caltech.ipac.firefly.server.cache.EhcacheProvider.*; + /** * Date: Jul 2, 2008 * @@ -17,35 +22,105 @@ */ public class CacheManager { - private static final String DEF_TYPE = Cache.TYPE_PERM_SMALL; private static final String DEF_PROVIDER = "edu.caltech.ipac.firefly.server.cache.EhcacheProvider"; // ie... "edu.caltech.ipac.firefly.server.cache.EhcacheProvider"; - private static final String CACHEMANAGER_DISABLED_PROP = "CacheManager.disabled"; - private static Cache.Provider cacheProvider; - private static boolean isDisabled = false; + private static Cache.Provider cacheProvider; + private static final Logger.LoggerImpl LOG = Logger.getLogger(); - static { - // disable caching is it's a preference - isDisabled = AppProperties.getBooleanProperty(CACHEMANAGER_DISABLED_PROP, false); + /** + * Same as {@code getLocal}. + * @return A Cache that represents the local cache, backed by both memory and disk overflow. + */ + public static Cache getCache() { + return getLocal(); } + /** + * Returns a local cache backed by Ehcache. The cache uses both in-memory storage and overflow to disk. + * @return A Cache that represents the local cache, backed by both memory and disk overflow. + */ + public static Cache getLocal() { + return getCacheProvider().getCache(PERM_SMALL); + } - public static Cache getCache() { - return getCache(DEF_TYPE); + /** + * A variant of {@code getLocal}, specifically designed with convenience features for handling + * File and FileInfo objects. + * @return A Cache tailored for File and FileInfo objects + */ + public static Cache getLocalFile() { + return new FileCache(getCacheProvider().getCache(PERM_SMALL)); } - public static Cache getCache(String type) { - if (!isDisabled) { - try { - return getCacheProvider().getCache(type); - } catch (Exception e){ - System.err.println("Unable to get Cache type:" + type + " returning EmptyCache."); - } - } - return new EmptyCache(); + /** + * A cache specifically designed for image visualization purposes. + * @return A Cache optimized for storing and accessing image data. + */ + public static Cache getVisMemCache() { + return getCacheProvider().getCache(VIS_SHARED_MEM); + } + + /** + * Returns a distributed cache, used for sharing cache data across + * different instances in a distributed environment. + * @return A Cache instance for distributed environments. + */ + public static Cache getDistributed() { + return new DistributedCache(); + } + + /** + * A variant of {@code getDistributed}, specifically designed with convenience features for handling + * File and FileInfo objects. + * @return A Cache tailored for File and FileInfo objects in a distributed environment + */ + public static Cache getDistributedFile() { + return new FileCache(new DistributedCache()); + } + + /** + * Returns a local cache specifically designed for storing data during + * a single session. This cache is for temporary storage + * of data that is specific to a user's session. + * @return A Cache instance for session-specific data storage. + */ + public static Cache getSessionCache() { + return getLocalMap(ServerContext.getRequestOwner().getRequestAgent().getSessId()); + } + + /** + * Returns a distributed cache designed specifically for storing user-related information. + * This cache is intended to live longer than a typical session cache and is backed by a long-lived user key + * stored in a cookie, which allows user data to persist across multiple sessions. + * @return A distributed Cache instance for storing user-related information with persistence across sessions. + */ + public static Cache getUserCache() { + return UserCache.getInstance(); + } + + /** + * Returns a local cache for storing data mapped to a unique key. + * @param mapKey The unique key associated with the cache. + * @return A Cache instance specifically for the data associated with the provided map key. + */ + public static Cache getLocalMap(String mapKey) { + return new LocalMapCache(mapKey); + } + + /** + * Returns a distributed cache designed for storing data mapped to a unique key. + * @param mapKey The unique key used to identify and access the data in the distributed cache. + * @return A Cache instance specifically for the data associated with the provided map key in the distributed environment. + */ + public static Cache getDistributedMap(String mapKey) { + return new DistribMapCache(mapKey); } +//==================================================================== +// +//==================================================================== + public static Cache.Provider getCacheProvider() { if (cacheProvider == null) { cacheProvider = newInstanceOf(DEF_PROVIDER); @@ -55,7 +130,6 @@ public static Cache.Provider getCacheProvider() { } public static boolean setCacheProvider(String cacheProviderClassName) { - if (isDisabled) return true; setCacheProvider(newInstanceOf(cacheProviderClassName)); return cacheProvider != null; } @@ -64,30 +138,17 @@ public static void setCacheProvider(Cache.Provider cacheProvider) { CacheManager.cacheProvider = cacheProvider; } - public static boolean isDisabled() { - return isDisabled; - } - - public static Cache getSessionCache() { - return new KeyBasedCache(ServerContext.getRequestOwner().getRequestAgent().getSessId()); - } - - public static Cache getUserCache() { - return new KeyBasedCache(ServerContext.getRequestOwner().getUserKey()); - } - - //==================================================================== // helper functions //==================================================================== + private static Cache.Provider newInstanceOf(String className) { - if (className != null && className.length() > 0) { + if (className != null && !className.isEmpty()) { try { Class cc = (Class) Cache.Provider.class.forName(className); return cc.newInstance(); } catch (Exception e) { - System.out.println("Can't create Cache.Provider:" + className + "\nThis could be a configuration error."); - e.printStackTrace(); + LOG.error(e, "Can't create Cache.Provider:" + className + "\nThis could be a configuration error."); } } return null; @@ -100,7 +161,7 @@ private static Cache.Provider newInstanceOf(String className) { * An empty implementation of Cache. This is a simple method to * disable caching by providing a permanently empty cache. */ - private static class EmptyCache implements Cache { + public static class EmptyCache implements Cache { public void put(CacheKey key, Object value) {} diff --git a/src/firefly/java/edu/caltech/ipac/util/download/CacheHelper.java b/src/firefly/java/edu/caltech/ipac/util/download/CacheHelper.java index 110deb2d0d..4db04bbd60 100644 --- a/src/firefly/java/edu/caltech/ipac/util/download/CacheHelper.java +++ b/src/firefly/java/edu/caltech/ipac/util/download/CacheHelper.java @@ -18,7 +18,7 @@ public class CacheHelper { private static File _cacheDir= null; - private final static Cache fileCache= CacheManager.getCache(Cache.TYPE_PERM_FILE); + private final static Cache fileCache= CacheManager.getDistributedFile(); public static void setCacheDir(File dir) { _cacheDir= dir; } public static File makeFitsFile(BaseNetParams params) { return makeFile(params.getUniqueString()+ ".fits"); } diff --git a/src/firefly/test/edu/caltech/ipac/firefly/ConfigTest.java b/src/firefly/test/edu/caltech/ipac/firefly/ConfigTest.java index 3dd0651e69..b306aef086 100644 --- a/src/firefly/test/edu/caltech/ipac/firefly/ConfigTest.java +++ b/src/firefly/test/edu/caltech/ipac/firefly/ConfigTest.java @@ -12,12 +12,15 @@ import java.io.FileInputStream; import java.io.IOException; import java.net.URL; +import java.nio.file.Files; +import java.nio.file.Path; import java.nio.file.Paths; +import java.nio.file.StandardCopyOption; import java.util.Properties; import java.util.UUID; /** - * Should load the logger and app properties to apply to any test runnner + * Should load the logger and app properties to apply to any test runner * For logger, using "test" alias name in unit test and change test properties to be used in {@link AppProperties} * Use the class to extend you test case and make use of particular log level * @@ -29,7 +32,7 @@ public class ConfigTest { public static String WS_USER_ID = AppProperties.getProperty("workspace.user","test@ipac.caltech.edu"); /** - * Use the logger in the test case that would extends this class. + * Use the logger in the test case that would extend this class. */ public static final Logger.LoggerImpl LOG = Logger.getLogger("test"); @@ -110,17 +113,39 @@ public static void setupServerContext(RequestAgent requestAgent, String contextN String contextPath = System.getenv("contextPath"); String webappConfigPath = System.getenv("webappConfigPath"); - AppProperties.setProperty("CacheManager.disabled", "true"); - AppProperties.setProperty("work.directory", Paths.get("build").toAbsolutePath().toString()); - contextPath = contextPath == null ? "/firefly" : contextPath; contextName = contextName == null ? "firefly" : contextName; webappConfigPath = webappConfigPath == null ? Paths.get("build/%s/war/WEB-INF/config".formatted(contextName)).toAbsolutePath().toString() : webappConfigPath; + AppProperties.setProperty("work.directory", Paths.get("build").toAbsolutePath().toString()); + Path buildConfg = Paths.get("build/firefly/war/WEB-INF/config"); + + copyWithSub(Paths.get("./config/ehcache.xml"), buildConfg, "app-name", contextName); + copy(Paths.get("config/test/app-test.prop"), buildConfg); + copy(Paths.get("config/ignore_sizeof.txt"), buildConfg); + + webappConfigPath = webappConfigPath != null ? webappConfigPath : buildConfg.toAbsolutePath().toString(); requestAgent = requestAgent == null ? new RequestAgent(null, "localhost", "/test", "localhost:8080/", "127.0. 0.1", UUID.randomUUID().toString(), contextPath): requestAgent; ServerContext.getRequestOwner().setRequestAgent(requestAgent); ServerContext.init(contextPath, contextName, webappConfigPath); } + private static void copy(Path src, Path dstDir) { + try { + Files.copy(src, dstDir.resolve(src.getFileName()), StandardCopyOption.REPLACE_EXISTING); + } catch (IOException ignored) {} + } + + private static void copyWithSub(Path src, Path dstDir, String token, String val) { + try { + String content = new String(Files.readAllBytes(src)); + content = content.replace("@%s@".formatted(token), val); + if (!Files.exists(dstDir)) { + Files.createDirectories(dstDir); + } + Files.write(dstDir.resolve(src.getFileName()), content.getBytes()); + } catch (IOException ignored) {} + } + } diff --git a/src/firefly/test/edu/caltech/ipac/firefly/core/RedisServiceTest.java b/src/firefly/test/edu/caltech/ipac/firefly/core/RedisServiceTest.java new file mode 100644 index 0000000000..b8e340c893 --- /dev/null +++ b/src/firefly/test/edu/caltech/ipac/firefly/core/RedisServiceTest.java @@ -0,0 +1,106 @@ +/* + * License information at https://github.com/Caltech-IPAC/firefly/blob/master/License.txt + */ + +package edu.caltech.ipac.firefly.core; + +import edu.caltech.ipac.firefly.ConfigTest; +import edu.caltech.ipac.firefly.server.util.Logger; +import edu.caltech.ipac.util.AppProperties; +import org.apache.logging.log4j.Level; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import redis.clients.jedis.Jedis; + +import static edu.caltech.ipac.firefly.core.RedisService.MAX_POOL_SIZE; +import static edu.caltech.ipac.firefly.core.RedisService.REDIS_HOST; +import static org.junit.Assert.assertEquals; + +/** + * This test suite requires a running Redis. To start one locally.. + * $ docker run --name test-redis -p 6379:6379 -d redis + * + * Date: 2019-03-15 + * @author loi + * @version $Id: $ + */ +public class RedisServiceTest extends ConfigTest { + + @Before + public void setup() { + RedisService.connect(); + if (RedisService.isOffline()) { + System.out.println("Messenger is offline; skipping test."); + } + if (true) Logger.setLogLevel(Level.TRACE); // for debugging. + } + + @After + public void teardown() { + RedisService.disconnect(); + LOG.trace("tear down"); + } + + @Test + public void testExternalRedis() { + AppProperties.setProperty(REDIS_HOST, "localhost"); // setup for external Redis + testRedis(); + } + + @Test + public void testEmbeddedRedis() { + // without REDIS_HOST set, an internal Redis will start up. + testRedis(); + } + + @Test + public void testExceedMaxConnections() { + AppProperties.setProperty(MAX_POOL_SIZE, "20"); + try { + for (int i=0; i<25; i++) { + Jedis conn = RedisService.getConnection(); + assertEquals("PONG", conn.ping()); + } + Assert.assertTrue(true); // should finish with some wait time. + } catch (Exception e) { + Assert.fail("Expected to fail"); + } + } + + private void testRedis() { + + if (RedisService.isOffline()) return; + + assertEquals(RedisService.Status.ONLINE, RedisService.getStatus()); + + // ping test + try (Jedis conn = RedisService.getConnection()) { + assertEquals("PONG", conn.ping()); + } catch (Exception e) { + Assert.fail("Can't connect: " + e); + } + + // set with expiry + try (Jedis conn = RedisService.getConnection()) { + conn.setex("key1", 1, "val1"); + assertEquals("val1", conn.get("key1")); + Thread.sleep(1_000); // expired after 1s + assertEquals(false, conn.exists("key1")); + } catch (Exception e) { + Assert.fail("Can't connect: " + e); + } + + // lots of connections test + for (int i=0; i<100; i++) { + try (Jedis conn = RedisService.getConnection()) { + assertEquals("PONG", conn.ping()); + } catch (Exception e) { + Assert.fail("Can't connect: " + e); + } + } + } + + +} diff --git a/src/firefly/test/edu/caltech/ipac/firefly/messaging/MessengerTest.java b/src/firefly/test/edu/caltech/ipac/firefly/messaging/MessengerTest.java index bf5c5aa1d5..9927e4ae21 100644 --- a/src/firefly/test/edu/caltech/ipac/firefly/messaging/MessengerTest.java +++ b/src/firefly/test/edu/caltech/ipac/firefly/messaging/MessengerTest.java @@ -6,8 +6,14 @@ import edu.caltech.ipac.TestCategory; import edu.caltech.ipac.firefly.ConfigTest; +import edu.caltech.ipac.firefly.core.RedisService; +import edu.caltech.ipac.firefly.server.util.Logger; import edu.caltech.ipac.firefly.util.Ref; +import org.apache.logging.log4j.Level; +import org.junit.After; +import org.junit.AfterClass; import org.junit.Before; +import org.junit.BeforeClass; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -27,16 +33,23 @@ * @version $Id: $ */ public class MessengerTest extends ConfigTest { - private boolean isOffline; + private static boolean isOffline; @Before - public void checkAlive() { - if (Messenger.isOffline()) { + public void setup() { + RedisService.connect(); + if (RedisService.isOffline()) { System.out.println("Messenger is offline; skipping test."); isOffline = true; } + if (true) Logger.setLogLevel(Level.TRACE); // for debugging. } + @After + public void teardown() { + RedisService.disconnect(); + LOG.trace("tear down"); + } @Test public void testMsgContent() throws InterruptedException { @@ -76,7 +89,7 @@ public void testMsgCount() throws InterruptedException { if (isOffline) return; LOG.debug("testMsgCount"); - Message testMsg = new Message(); + Message testMsg = new Message().setValue("abc", "a"); String topic1 = "test1"; String topic2 = "test2"; @@ -90,7 +103,7 @@ public void testMsgCount() throws InterruptedException { tester.set(new CountDownLatch(4)); Messenger.publish(topic1, testMsg); Messenger.publish(topic2, testMsg); - tester.get().await(1, TimeUnit.SECONDS); // wait up to 1s for msg delivery.. + tester.get().await(5, TimeUnit.SECONDS); // wait up to 1s for msg delivery. assertEquals("latch(4) should drain", 0, tester.get().getCount()); LOG.debug("same as above, but 1 sub removed from topic1... = 3"); @@ -131,26 +144,26 @@ public void testSubscribe() throws InterruptedException { Subscriber sub22 = Messenger.subscribe(topic2, msg -> msg = null); LOG.debug("2 topics, 2 subs per topic.. = 2 connections"); - Thread.sleep(100); - assertEquals("init", 2, Messenger.getConnectionCount()); + Thread.sleep(300); // give time for pool to update its stats + assertEquals("init", 2, RedisService.getConnectionCount()); LOG.debug("remove 1 sub from topic 1.. = 2 connections"); Messenger.unSubscribe(sub11); Thread.sleep(100); - assertEquals("3 subs left", 2, Messenger.getConnectionCount()); + assertEquals("3 subs left", 2, RedisService.getConnectionCount()); LOG.debug("remove both subs from topic 1.. = 1 connections"); Messenger.unSubscribe(sub12); Thread.sleep(100); - assertEquals("only topic2 left", 1, Messenger.getConnectionCount()); + assertEquals("only topic2 left", 1, RedisService.getConnectionCount()); LOG.debug("remove both topic.. = 0 connections"); Messenger.unSubscribe(sub21); Messenger.unSubscribe(sub22); Thread.sleep(100); - assertEquals("no subs", 0, Messenger.getConnectionCount()); + assertEquals("no subs", 0, RedisService.getConnectionCount()); - LOG.debug("Messenger stats: " + Messenger.getStats()); + LOG.debug("Messenger stats: " + RedisService.getStats()); LOG.debug("testSubscribe.. done!"); } @@ -159,7 +172,10 @@ public void testSubscribe() throws InterruptedException { @Category({TestCategory.Perf.class}) @Test public void perfTest() throws InterruptedException { - int numSent = 1000*100; + + if (isOffline) return; + + int numSent = 100_000; long startTime = System.currentTimeMillis(); final CountDownLatch numRevc = new CountDownLatch(numSent); @@ -168,14 +184,14 @@ public void perfTest() throws InterruptedException { if (numRevc.getCount() == 0) { long stopTime = System.currentTimeMillis(); System.out.println("\t elapsed time: " + (stopTime - startTime)/1000.0 + "s"); - System.out.println("\t Messenger stats: " + Messenger.getStats()); + System.out.println("\t Messenger stats: " + RedisService.getStats()); } }); // This is a performance as well as a stress test. // Sending a total of 100k messages using 50 simultaneous threads - // This will exhaust Messenger's pool size of 25 so we can see how it does under stress. - ExecutorService exec = Executors.newFixedThreadPool(50); + // This will exhaust Messenger's pool size of 100, so we can see how it does under stress. + ExecutorService exec = Executors.newFixedThreadPool(200); System.out.printf("Test sending %d number of messages...\n", numSent); for(int i = 0; i < numSent; i++) { @@ -183,6 +199,7 @@ public void perfTest() throws InterruptedException { exec.submit(() -> Messenger.publish("perfTest", new Message().setValue("idx" + finalI))); } numRevc.await(); + Thread.sleep(100); } } diff --git a/src/firefly/test/edu/caltech/ipac/table/DuckDbAdapterTest.java b/src/firefly/test/edu/caltech/ipac/table/DuckDbAdapterTest.java index 8e0c8afce8..3e6703200f 100644 --- a/src/firefly/test/edu/caltech/ipac/table/DuckDbAdapterTest.java +++ b/src/firefly/test/edu/caltech/ipac/table/DuckDbAdapterTest.java @@ -200,7 +200,7 @@ public void testDecimateQuery() throws DataAccessException { hsqlReq.setParam(SEARCH_REQUEST, JsonTableUtil.toJsonTableRequest(hsql).toJSONString()); DataGroup duckTbl = new SearchManager().getDataGroup(duckReq).getData(); - DataGroup hsqlTbl = new SearchManager().getDataGroup(hsqlReq).getData(); + DataGroup hsqlTbl = new SearchManager().getDataGroup(hsqlReq).getData(); // failed. TODO fullTableTest(duckTbl, hsqlTbl); diff --git a/src/firefly/test/edu/caltech/ipac/table/EmbeddedDbUtilTest.java b/src/firefly/test/edu/caltech/ipac/table/EmbeddedDbUtilTest.java index c1d8408503..2e0a6aba5b 100644 --- a/src/firefly/test/edu/caltech/ipac/table/EmbeddedDbUtilTest.java +++ b/src/firefly/test/edu/caltech/ipac/table/EmbeddedDbUtilTest.java @@ -4,6 +4,7 @@ package edu.caltech.ipac.table; import edu.caltech.ipac.TestCategory; +import edu.caltech.ipac.firefly.core.Util; import edu.caltech.ipac.firefly.server.db.DbAdapter; import edu.caltech.ipac.firefly.server.db.HsqlDbAdapter; import edu.caltech.ipac.firefly.server.query.DataAccessException; @@ -17,7 +18,6 @@ import edu.caltech.ipac.firefly.server.db.EmbeddedDbUtil; import edu.caltech.ipac.firefly.server.query.tables.IpacTableFromSource; import edu.caltech.ipac.firefly.util.FileLoader; -import org.apache.commons.codec.digest.DigestUtils; import org.apache.logging.log4j.Level; import org.junit.AfterClass; import org.junit.Assert; @@ -28,7 +28,6 @@ import java.io.File; import java.io.IOException; -import java.util.Arrays; import java.util.List; import static edu.caltech.ipac.firefly.server.db.DbAdapter.getAdapter; @@ -314,24 +313,24 @@ public void testAddColumnErrors() { public void testSerializer() { // array of boolean Object in = new Boolean[]{true, false, true}; - String s = EmbeddedDbUtil.serialize(in); - Object d = EmbeddedDbUtil.deserialize(s); + String s = Util.serialize(in); + Object d = Util.deserialize(s); if (d instanceof Boolean[] v) { Assert.assertArrayEquals(v, (Boolean[])in); } else Assert.fail("Deserialized type Boolean mismatch"); // array of double in = new Double[]{1.0, 2.0, 3.0}; - s = EmbeddedDbUtil.serialize(in); - d = EmbeddedDbUtil.deserialize(s); + s = Util.serialize(in); + d = Util.deserialize(s); if (d instanceof Double[] v) { Assert.assertArrayEquals(v, (Double[])in); } else Assert.fail("Deserialized type Double mismatch"); // array of integer in = new Integer[]{1, 2, 3}; - s = EmbeddedDbUtil.serialize(in); - d = EmbeddedDbUtil.deserialize(s); + s = Util.serialize(in); + d = Util.deserialize(s); if (d instanceof Integer[] v) { Assert.assertArrayEquals(v, (Integer[])in); } else Assert.fail("Deserialized type Integer mismatch"); diff --git a/src/firefly/test/edu/caltech/ipac/util/cache/CachePeerProviderFactoryTest.java b/src/firefly/test/edu/caltech/ipac/util/cache/CachePeerProviderFactoryTest.java index 4d59f46387..d9748286c5 100644 --- a/src/firefly/test/edu/caltech/ipac/util/cache/CachePeerProviderFactoryTest.java +++ b/src/firefly/test/edu/caltech/ipac/util/cache/CachePeerProviderFactoryTest.java @@ -5,12 +5,15 @@ package edu.caltech.ipac.util.cache; import edu.caltech.ipac.firefly.ConfigTest; +import edu.caltech.ipac.firefly.core.RedisService; import edu.caltech.ipac.firefly.messaging.Messenger; +import edu.caltech.ipac.firefly.server.util.Logger; import net.sf.ehcache.CacheManager; import net.sf.ehcache.Ehcache; import net.sf.ehcache.Element; import net.sf.ehcache.distribution.CacheManagerPeerProvider; import net.sf.ehcache.distribution.CachePeer; +import org.apache.logging.log4j.Level; import org.junit.BeforeClass; import org.junit.Test; @@ -80,10 +83,11 @@ public class CachePeerProviderFactoryTest extends ConfigTest { @BeforeClass public static void setUp() throws InterruptedException { - if (Messenger.isOffline()) { + if (RedisService.isOffline()) { System.out.println("Messenger is offline; skipping all tests in CachePeerProviderFactoryTest."); return; } + if (true) Logger.setLogLevel(Level.TRACE); // for debugging. LOG.debug("Initial setup for PubSub, creating 3 peers"); peer1 = createCM("PubSub", "peer1"); @@ -102,7 +106,7 @@ public static void setUp() throws InterruptedException { @Test public void pubSub_InitialSetup() throws InterruptedException, RemoteException { - if (Messenger.isOffline()) return; + if (RedisService.isOffline()) return; assertEquals("Number of caches", 2, peer1.getCacheNames().length); @@ -119,7 +123,7 @@ public void pubSub_InitialSetup() throws InterruptedException, RemoteException { @Test public void pubSub_CacheReplication() throws InterruptedException, RemoteException { - if (Messenger.isOffline()) return; + if (RedisService.isOffline()) return; LOG.debug("put a few entries into peer1... it should be replicated to the rest of the peers"); peer1.getCache("cache_1").put(new Element("key1", "value1")); @@ -142,7 +146,7 @@ public void pubSub_CacheReplication() throws InterruptedException, RemoteExcepti @Test public void pubSub_PeerAddDrop() throws InterruptedException, RemoteException { - if (Messenger.isOffline()) return; + if (RedisService.isOffline()) return; LOG.debug("testing drop-off.... shutdown peer1"); peer1.shutdown(); @@ -174,7 +178,7 @@ public void pubSub_PeerAddDrop() throws InterruptedException, RemoteException { @Test public void multicast_InitialSetup() throws InterruptedException, RemoteException { - if (Messenger.isOffline()) return; + if (RedisService.isOffline()) return; assertEquals("Number of caches", 2, mcPeer1.getCacheNames().length); @@ -191,7 +195,7 @@ public void multicast_InitialSetup() throws InterruptedException, RemoteExceptio @Test public void multicast_CacheReplication() throws InterruptedException, RemoteException { - if (Messenger.isOffline()) return; + if (RedisService.isOffline()) return; LOG.debug("put a few entries into mcPeer1... it should be replicated to the rest of the peers"); mcPeer1.getCache("cache_1").put(new Element("key1", "value1")); @@ -214,7 +218,7 @@ public void multicast_CacheReplication() throws InterruptedException, RemoteExce @Test public void multicast_PeerAddDrop() throws InterruptedException, RemoteException { - if (Messenger.isOffline()) return; + if (RedisService.isOffline()) return; LOG.debug("testing drop-off.... shutdown mcPeer1"); mcPeer1.shutdown(); diff --git a/src/firefly/test/edu/caltech/ipac/util/cache/CacheTest.java b/src/firefly/test/edu/caltech/ipac/util/cache/CacheTest.java new file mode 100644 index 0000000000..09eee5d751 --- /dev/null +++ b/src/firefly/test/edu/caltech/ipac/util/cache/CacheTest.java @@ -0,0 +1,120 @@ +/* + * License information at https://github.com/Caltech-IPAC/firefly/blob/master/License.txt + */ + +package edu.caltech.ipac.util.cache; + +import edu.caltech.ipac.firefly.ConfigTest; +import edu.caltech.ipac.firefly.data.userdata.UserInfo; +import edu.caltech.ipac.firefly.server.util.Logger; +import org.apache.logging.log4j.Level; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.io.File; + +import static edu.caltech.ipac.util.cache.Cache.*; +import static org.junit.Assert.*; + +/** + * This test suite requires a running Redis. To start one locally.. + * $ docker run --name test-redis -p 6379:6379 -d redis + * + * Date: 2019-04-11 + * @author loi + * @version $Id: $ + */ +public class CacheTest extends ConfigTest { + + @BeforeClass + public static void setUp() throws InterruptedException { + setupServerContext(null); + if (true) Logger.setLogLevel(Level.TRACE); // for debugging. + } + + + @Test + public void localCache() { + Cache cache = CacheManager.getLocal(); + testObject(cache); + } + + @Test + public void distributedCache() { + Cache cache = CacheManager.getDistributed(); + testObject(cache); + } + + @Test + public void localFileCache() { + Cache cache = CacheManager.getLocalFile(); + testFile(cache); + } + + @Test + public void distributedFileCache() { + Cache cache = CacheManager.getDistributedFile(); + testFile(cache); + } + + @Test + public void userCache() { + Cache cache = CacheManager.getUserCache(); + testObject(cache); + } + + @Test + public void localMap() { + Cache cache = CacheManager.getLocalMap("test"); + testObject(cache); + } + + @Test + public void distributedMap() { + Cache cache = CacheManager.getDistributedMap("test"); + testObject(cache); + } + + private void testFile(Cache cache) { + cache.put(new StringKey("1"), 1); + assertNull(cache.get(new StringKey("1"))); + + File badf = new File("bad/path/not-found.txt"); + cache.put(new StringKey(badf.getName()), badf); + Object bf = cache.get(new StringKey(badf.getName())); + assertNull(bf); + + File goodf = new File(System.getProperty("java.io.tmpdir")); + cache.put(new StringKey(goodf.getName()), goodf); + Object gf = cache.get(new StringKey(goodf.getName())); + assertTrue(gf instanceof File); + assertEquals(goodf.toString(), gf.toString()); + + } + + + private void testObject(Cache cache) { + cache.put(new StringKey("1"), 1); + assertEquals(1, cache.get(new StringKey("1"))); + + cache.put(new StringKey("string"), "string"); + assertEquals("string", cache.get(new StringKey("string"))); + + cache.put(new StringKey("string"), "string"); + assertEquals("string", cache.get(new StringKey("string"))); + + UserInfo u = new UserInfo("userId", "password"); + u.setEmail("me@acme.com"); + cache.put(new StringKey(u.getLoginName()), u); + Object ru = cache.get(new StringKey(u.getLoginName())); + assertTrue(ru instanceof UserInfo); + assertEquals(u.toString(), ru.toString()); + + File f = new File("bad/path/not-found.txt"); + cache.put(new StringKey(f.getName()), f); + Object rf = cache.get(new StringKey(f.getName())); + assertTrue(rf instanceof File); + assertEquals(f.getAbsolutePath(), ((File) rf).getAbsolutePath()); + } + +} From c597d765689a708fb7dedded41339aa68b445af8 Mon Sep 17 00:00:00 2001 From: loi Date: Fri, 20 Dec 2024 18:23:12 -0800 Subject: [PATCH 2/3] convert events to redis sub/pub --- config/ehcache.xml | 18 +- config/shared_ehcache.xml | 379 ------------------ .../ipac/firefly/core/RedisService.java | 36 +- .../edu/caltech/ipac/firefly/core/Util.java | 130 ++++++ .../ipac/firefly/messaging/Message.java | 102 ++++- .../ipac/firefly/messaging/Messenger.java | 13 +- .../ipac/firefly/server/db/DbInstance.java | 2 +- .../ipac/firefly/server/db/DbMonitor.java | 11 +- .../ipac/firefly/server/db/DuckDbAdapter.java | 4 +- .../firefly/server/db/EmbeddedDbUtil.java | 5 +- .../firefly/server/db/spring/JdbcFactory.java | 9 +- .../server/events/CacheEventWorker.java | 3 + .../server/events/MessageEventWorker.java | 36 ++ .../server/events/ReplicatedQueueList.java | 26 +- .../server/events/ServerEventManager.java | 46 ++- .../server/events/ServerEventQueue.java | 5 + .../server/events/WebsocketConnector.java | 4 +- .../server/query/EmbeddedDbProcessor.java | 7 +- .../firefly/server/query/SearchProcessor.java | 26 -- .../caltech/ipac/firefly/util/event/Name.java | 10 +- .../ipac/table/io/TableParseHandler.java | 10 +- .../edu/caltech/ipac/util/StringUtils.java | 78 ---- .../ipac/firefly/core/RedisServiceTest.java | 14 +- .../caltech/ipac/firefly/core/UtilTest.java | 179 +++++++++ .../ipac/firefly/messaging/MessengerTest.java | 2 +- .../caltech/ipac/table/DuckDbAdapterTest.java | 49 --- .../ipac/table/EmbeddedDbUtilTest.java | 27 -- .../cache/CachePeerProviderFactoryTest.java | 2 +- .../caltech/ipac/util/cache/CacheTest.java | 9 +- 29 files changed, 559 insertions(+), 683 deletions(-) delete mode 100644 config/shared_ehcache.xml create mode 100644 src/firefly/java/edu/caltech/ipac/firefly/server/events/MessageEventWorker.java create mode 100644 src/firefly/test/edu/caltech/ipac/firefly/core/UtilTest.java diff --git a/config/ehcache.xml b/config/ehcache.xml index 9cc67d78ac..4741c2cb39 100644 --- a/config/ehcache.xml +++ b/config/ehcache.xml @@ -118,15 +118,6 @@ use 239.250.0.1 for org-local use 230.0.0.1 for global --> - - - - - - - - - - - - - - - - - - - - - - - - - - - - \ No newline at end of file diff --git a/src/firefly/java/edu/caltech/ipac/firefly/core/RedisService.java b/src/firefly/java/edu/caltech/ipac/firefly/core/RedisService.java index 6383a3d612..160c184351 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/core/RedisService.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/core/RedisService.java @@ -89,6 +89,7 @@ static void startLocal() { .setting("maxmemory %s".formatted(MAX_MEM)) .setting("dir %s".formatted(DB_DIR)) // Directory where redis database files are stored .setting("dbfilename redis.rdb") // RDB file name + .setting("save 600 1") // RDB file name .build(); redisServer.start(); connect(); @@ -138,27 +139,42 @@ public static Map getStats() { } catch (NoSuchAlgorithmException e) {/* ignore */} try (Jedis redis = getConnection()) { + var infos = redis.info().split("\r\n"); + Arrays.stream(infos).filter(s -> s.contains("version")).findFirst() + .ifPresent(s -> stats.put("version", s.split(":")[1].trim())); + stats.put("active conn", jedisPool.getNumActive()); stats.put("idle conn", jedisPool.getNumIdle()); stats.put("max conn", maxPoolSize); stats.put("max-wait", jedisPool.getMaxBorrowWaitTimeMillis()); stats.put("avg-wait", jedisPool.getMeanBorrowWaitTimeMillis()); - stats.put("cache-size", redis.dbSize()); - stats.put("DB_DIR", DB_DIR); - stats.put("MAX_MEM", MAX_MEM); stats.put("password", passwd); - - Arrays.stream(redis.info().split("\r\n")).forEach((s) -> { - if (!s.startsWith("#")) { - String[] kv = s.split(":"); - stats.put(kv[0], kv[1]); - } - }); + stats.put("db-size", redis.dbSize()); + addStat(stats, redis, "maxmemory"); + addStat(stats, redis, "save"); + addStat(stats, redis, "dir"); + addStat(stats, redis, "dbfilename"); + addStat(stats, redis, "appendfilename"); + stats.put("---MEMORY STATS----", ""); + var mem = redis.memoryStats(); + stats.put("Total memory used", mem.get("dataset.bytes")); + stats.put("Total memory allocated", mem.get("allocator.allocated")); + stats.put("Fragmented memory", mem.get("fragmentation")); + stats.put("Fragmentation ratio", mem.get("allocator-fragmentation.ratio")); + stats.put("Number of keys stored", mem.get("keys.count")); + stats.put("Avg per key", mem.get("keys.bytes-per-key")); + stats.put("Pct of memory used", mem.get("dataset.percentage")); + stats.put("Peak memory used", mem.get("peak.allocated")); } catch (Exception ignored) {} } return stats; } + public static void addStat(Map stats, Jedis redis, String key) { + var c = redis.configGet(key); + if (c.size() > 1) stats.put(key, c.get(1)); + } + public static String getRedisHostPortDesc() { return redisHost + ":" + REDIS_PORT + " ("+ getStatus() + ")"; } diff --git a/src/firefly/java/edu/caltech/ipac/firefly/core/Util.java b/src/firefly/java/edu/caltech/ipac/firefly/core/Util.java index f23567824b..ef3c7b9d5e 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/core/Util.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/core/Util.java @@ -11,6 +11,10 @@ import java.io.ObjectInputStream; import java.io.ObjectOutputStream; import java.util.Base64; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.locks.ReentrantLock; +import java.util.function.Consumer; +import java.util.function.Predicate; /** * Date: 11/19/24 @@ -70,6 +74,132 @@ public static Object deserialize(String base64) { } +//==================================================================== +// Functional Helpers +//==================================================================== + /** + * A function that throws exception + * @param the return type of this function + */ + @FunctionalInterface + public interface FuncWithEx { + T get() throws Exception; + } + + /** + * A callable that throws exception + */ + @FunctionalInterface + public interface CallWithEx { + void run() throws Exception; + } + /** + * A function with parameter that throws exception + */ + @FunctionalInterface + public interface FuncParamWithEx { + T apply(P p) throws Exception; + } + + public static class Try { + private final T val; + private final Exception ex; + + Try(T val, Exception ex) { + this.val = val; + this.ex = ex; + } + + /** + * Get the value if no exception was thrown + * @return the value if no exception was thrown, or null + */ + public T get() { + return ex == null ? val : null; + } + + public T getOrElse(T defVal) { + return ex == null ? val : defVal; + } + + public T getOrElse(Consumer onError) { + if (ex == null) return val; + onError.accept(ex); + return null; + } + + public static Try it(FuncParamWithEx func, P param) { + try { + return new Try<>(func.apply(param), null); + } catch (Exception e) { + return new Try<>(null, e); + } + } + + public static Try it(CallWithEx func) { + try { + func.run(); + return new Try<>(null, null); + } catch (Exception e) { + return new Try<>(null, e); + } + } + + public static Try it(FuncWithEx func) { + try { + return new Try<>(func.get(), null); + } catch (Exception e) { + return new Try<>(null, e); + } + } + + /** + * Execute the given function until it passes test, then return the result + * @param func the function to execute + * @param test test the returned value + * @param tries the number of times to try + * @return Try results + */ + public static Try until(FuncWithEx func, Predicate test, int tries) { + for (int i = 0; i < tries; i++) { + var res = it(func).get(); + if (res != null && test.test(res)) return new Try<>(res, null); + } + return new Try<>(null, new IndexOutOfBoundsException("Exceeded max tries")); + } + + } + + + public static class SynchronizedAccess { + private final ConcurrentHashMap activeRequests = new ConcurrentHashMap<>(); + + @FunctionalInterface + public interface LockHandle { + void unlock(); + } + + /** + * Acquires a lock associated with the given ID. If the lock does not already exist, it is created. + * + * @param id the identifier for the lock + * @return a {@code Runnable} that, when executed, releases the lock and removes it from the active requests + */ + public LockHandle lock(String id) { + ReentrantLock lock = activeRequests.computeIfAbsent(id, k -> new ReentrantLock()); + Logger.getLogger().trace("waiting %s: %s\n".formatted(id, lock)); + lock.lock(); + Logger.getLogger().trace("got lock %s: %s\n".formatted(id, lock)); + return () -> { + try { + lock.unlock(); // Ensure lock is released even if an exception occurs + } finally { + if (!lock.isLocked()) activeRequests.remove(id); // Remove the lock from activeRequests if no threads are using it + Logger.getLogger().trace("unlock %s: %s\n".formatted(id, lock)); + } + }; + } + } } diff --git a/src/firefly/java/edu/caltech/ipac/firefly/messaging/Message.java b/src/firefly/java/edu/caltech/ipac/firefly/messaging/Message.java index b0e4e93ef2..3133511fa0 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/messaging/Message.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/messaging/Message.java @@ -6,38 +6,35 @@ import edu.caltech.ipac.firefly.core.background.JobInfo; import edu.caltech.ipac.firefly.core.background.JobManager; +import edu.caltech.ipac.firefly.data.ServerEvent; import edu.caltech.ipac.firefly.data.ServerEvent.Scope; import edu.caltech.ipac.firefly.server.ServerContext; +import edu.caltech.ipac.firefly.util.event.Name; import edu.caltech.ipac.util.AppProperties; +import java.io.Serializable; + +import static edu.caltech.ipac.util.StringUtils.applyIfNotEmpty; +import static edu.caltech.ipac.firefly.data.ServerEvent.*; + /** * The message object sent or received by Messenger. As implemented, Messenger * uses JSON when serialize/deserialize is needed. However, this is an internal * implementation, and it can be changed without breaking Messenger/Message contract. * - * Date: 2019-03-15 - * * @author loi * @version $Id: $ */ public class Message { + private static final String TOPIC_KEY = "msg_topic"; protected JsonHelper helper = new JsonHelper(); - public MsgHeader getHeader() { - Scope scope = Scope.valueOf(helper.getValue(Scope.SELF.name(), "header", "scope")); - String to = helper.getValue("", "header", "to"); - String from = helper.getValue("", "header", "from"); - String subject = helper.getValue("", "header", "subject"); - MsgHeader header = new MsgHeader(scope, to, subject); - header.setFrom(from); - return header; + public void setTopic(String topic) { + setValue(topic, TOPIC_KEY); } - public void setHeader(Scope scope, String to, String subject, String from) { - helper.setValue(scope.name(), "header", "scope"); - if (to != null) helper.setValue(to, "header", "to"); - if (from != null) helper.setValue(from, "header", "from"); - if (subject != null) helper.setValue(subject, "header", "subject"); + public String getTopic() { + return getValue(null, TOPIC_KEY); } public Message setValue(Object value, String... paths) { @@ -63,24 +60,89 @@ public String toJson() { // Predefined messages //==================================================================== + /** + * Represents a server event message that can be sent or received by the Messenger. + * This class encapsulates the details of a server event, including its name, target, data type, data, and source. + * It provides methods to construct an event from a ServerEvent object and to parse a Message object back into a ServerEvent. + * + *

Example usage:

+ *
+     * {@code
+     * ServerEvent serverEvent = new ServerEvent(name, scope, data);
+     * Message.Event eventMessage = new Message.Event(serverEvent);
+     * }
+     * 
+ **/ + public static final class Event extends Message { + public static final String TOPIC = "firefly-events"; + + public Event(ServerEvent se) { + setTopic(TOPIC); + setValue(se.getName().getName(), "name"); + applyIfNotEmpty(se.getTarget().getScope(), (s) -> setValue(s.name(), "target", "scope")); + applyIfNotEmpty(se.getTarget().getChannel(), (s) -> setValue(s, "target", "channel")); + applyIfNotEmpty(se.getTarget().getConnID(), (s) -> setValue(s, "target", "connID")); + applyIfNotEmpty(se.getTarget().getUserKey(), (s) -> setValue(s, "target", "userKey")); + setValue(se.getDataType().name(), "dataType"); + setValue(se.getData(), "data"); + setValue(se.getFrom(), "from"); + } + public static ServerEvent parse(Message msg) { + try { + Name name = Name.parse(msg.getValue(null, "name")); + if (name == null) return null; + + Scope scope = Scope.valueOf(msg.getValue(null, "target", "scope")); + String connID = msg.getValue(null, "target", "connID"); + String channel = msg.getValue(null, "target", "channel"); + String userKey = msg.getValue(null, "target", "userKey"); + DataType dtype = DataType.valueOf(msg.getValue(null, "dataType")); + Serializable data = msg.getValue(null, "data"); + String from = msg.getValue(null, "from"); + return new ServerEvent(name, new EventTarget(scope, connID, channel, userKey), dtype, data, from); + } catch (Exception e) { + return null; + } + } + } + public static final class JobCompleted extends Message { public static final String TOPIC = "JobCompleted"; static final String FROM = AppProperties.getProperty("mail.smtp.from", "donotreply@ipac.caltech.edu"); JobInfo job; public JobCompleted(JobInfo jobInfo) { job = jobInfo; + setTopic(TOPIC); setHeader(Scope.CHANNEL, jobInfo.getOwner(), TOPIC, FROM); - helper.setValue(JobManager.toJsonObject(jobInfo), "jobInfo"); + setValue(JobManager.toJsonObject(jobInfo), "jobInfo"); var user = ServerContext.getRequestOwner().getUserInfo(); if (user != null) { - helper.setValue(user.getName(), "user", "name"); - helper.setValue(user.getEmail(), "user", "email"); - helper.setValue(user.getLoginName(), "user", "loginName"); + setValue(user.getName(), "user", "name"); + setValue(user.getEmail(), "user", "email"); + setValue(user.getLoginName(), "user", "loginName"); } var ssoAdpt = ServerContext.getRequestOwner().getSsoAdapter(); if (ssoAdpt != null) { - helper.setValue(ssoAdpt.getAuthToken(), "user", "authToken"); + setValue(ssoAdpt.getAuthToken(), "user", "authToken"); } } + + public MsgHeader getHeader() { + Scope scope = Scope.valueOf(getValue(Scope.SELF.name(), "header", "scope")); + String to = getValue("", "header", "to"); + String from = getValue("", "header", "from"); + String subject = getValue("", "header", "subject"); + MsgHeader header = new MsgHeader(scope, to, subject); + header.setFrom(from); + return header; + } + + public void setHeader(Scope scope, String to, String subject, String from) { + setValue(scope.name(), "header", "scope"); + if (to != null) setValue(to, "header", "to"); + if (from != null) setValue(from, "header", "from"); + if (subject != null) setValue(subject, "header", "subject"); + } } + } \ No newline at end of file diff --git a/src/firefly/java/edu/caltech/ipac/firefly/messaging/Messenger.java b/src/firefly/java/edu/caltech/ipac/firefly/messaging/Messenger.java index 77c2fc108a..8fd31ecf5b 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/messaging/Messenger.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/messaging/Messenger.java @@ -13,6 +13,8 @@ import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.atomic.AtomicInteger; +import static edu.caltech.ipac.util.StringUtils.applyIfNotEmpty; + /** * An implementation of publish-subscribe messaging pattern based on Jedis client and Redis backend. * This class abstract the use of threads, and it ensures that there is only 1 thread used per topic. @@ -77,16 +79,13 @@ public static void publish(String topic, Message msg) { } /** - * Send the given message. The message's subject is used as the topic. - * @param msg the message to send + * Publishes the given message to its topic. + * @param msg the message to be published */ - public static void publish(Message msg) { - // some firefly's specific logic here... - String topic = msg.getHeader().getSubject(); - publish(topic, msg); + public static void publish(Message msg) { + applyIfNotEmpty(msg.getTopic(), (topic) -> publish(topic, msg)); } - /** * Internal handler class used to manage the one-to-many relationship of Messenger's subscriber and * Jedis's subscriber diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/db/DbInstance.java b/src/firefly/java/edu/caltech/ipac/firefly/server/db/DbInstance.java index f4710aaaa0..a3f43da12d 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/db/DbInstance.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/db/DbInstance.java @@ -109,7 +109,7 @@ public String getDbUrl() { public void setPooled(boolean pooled) { isPooled = pooled;} public boolean testConn(Connection conn) { try (Statement stmt = conn.createStatement()) { - stmt.execute("SELECT 1"); // works with all DMBS + stmt.execute("SELECT 1 FROM (VALUES (0))"); // HSQL required FROM clause return true; } catch (SQLException e) { return false; } }; diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/db/DbMonitor.java b/src/firefly/java/edu/caltech/ipac/firefly/server/db/DbMonitor.java index 3edda7f52a..da4811675c 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/db/DbMonitor.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/db/DbMonitor.java @@ -5,13 +5,18 @@ package edu.caltech.ipac.firefly.server.db; import edu.caltech.ipac.firefly.server.util.Logger; +import edu.caltech.ipac.firefly.util.Ref; import edu.caltech.ipac.util.AppProperties; import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.concurrent.ConcurrentHashMap; -import java.util.stream.Collectors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + +import static edu.caltech.ipac.firefly.core.Util.Try; +import static edu.caltech.ipac.firefly.server.ServerContext.SHORT_TASK_EXEC; /** * Date: 5/3/24 @@ -92,9 +97,11 @@ public static DbAdapter.EmbeddedDbStats getRuntimeStats(boolean doUpdate) { public static void updateDbStats() { LOGGER.trace("DbAdapter -> updateDbStats"); + Ref> t = new Ref<>(); for (DbAdapter.EmbeddedDbInstance db : dbInstances.values()) { - db.updateStats(); + t.set(SHORT_TASK_EXEC.submit(db::updateStats)); } + Try.it(() -> t.get().get(5, TimeUnit.SECONDS)); // run all in parallel, but wait for up to 5 seconds } //==================================================================== diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/db/DuckDbAdapter.java b/src/firefly/java/edu/caltech/ipac/firefly/server/db/DuckDbAdapter.java index cf80c612e1..633785b747 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/db/DuckDbAdapter.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/db/DuckDbAdapter.java @@ -32,6 +32,7 @@ import java.util.Date; import java.util.List; +import static edu.caltech.ipac.firefly.core.Util.Try; import static edu.caltech.ipac.firefly.server.db.DuckDbUDF.*; import static edu.caltech.ipac.firefly.server.db.EmbeddedDbUtil.*; import static edu.caltech.ipac.util.StringUtils.*; @@ -253,7 +254,8 @@ public String interpretError(Throwable e) { if (e instanceof SQLException ex) { JSONObject json = (JSONObject) JSONValue.parse(ex.getMessage().replace("%s:".formatted(ex.getClass().getName()), "")); String msg = json.get("exception_message").toString().split("\n")[0]; - String type = getSafe(() -> json.get("error_subtype").toString(), json.get("exception_type").toString()); + String type = Try.it(() -> json.get("error_subtype").toString()) + .getOrElse(json.get("exception_type").toString()); return type + ":" + msg; } return super.interpretError(e); diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/db/EmbeddedDbUtil.java b/src/firefly/java/edu/caltech/ipac/firefly/server/db/EmbeddedDbUtil.java index ab30369b14..f1965c0488 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/db/EmbeddedDbUtil.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/db/EmbeddedDbUtil.java @@ -35,6 +35,7 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; +import static edu.caltech.ipac.firefly.core.Util.Try; import static edu.caltech.ipac.firefly.server.ServerContext.SHORT_TASK_EXEC; import static edu.caltech.ipac.firefly.data.TableServerRequest.TBL_FILE_PATH; import static edu.caltech.ipac.firefly.data.TableServerRequest.TBL_FILE_TYPE; @@ -413,10 +414,10 @@ public static void dbToDataType(DataType dtype, ResultSet rs) { // into base64 string for storage. //==================================================================== public static Object deserialize(ResultSet rs, String cname) { - return getSafe(() -> Util.deserialize(rs.getString(cname))); + return Try.it(() -> Util.deserialize(rs.getString(cname))).get(); } public static Object deserialize(ResultSet rs, int cidx) { - return getSafe(() -> Util.deserialize(rs.getString(cidx))); + return Try.it(() -> Util.deserialize(rs.getString(cidx))).get(); } //==================================================================== // privates functions diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/db/spring/JdbcFactory.java b/src/firefly/java/edu/caltech/ipac/firefly/server/db/spring/JdbcFactory.java index c2801d296d..98026b2d09 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/db/spring/JdbcFactory.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/db/spring/JdbcFactory.java @@ -20,9 +20,10 @@ import java.sql.SQLException; import java.util.HashMap; import java.util.Map; +import java.util.Objects; import java.util.Properties; -import static edu.caltech.ipac.util.StringUtils.getIf; +import static edu.caltech.ipac.firefly.core.Util.Try; /** * Date: Oct 7, 2008 @@ -40,7 +41,8 @@ public class JdbcFactory { * @return */ public static JdbcTemplate getTemplate(DbInstance dbInstance) { - DataSource datasource = getIf( () -> getDataSource(dbInstance), (ds) -> ds != null,3); // return null if failed after 3 tries + DataSource datasource = Try.until( () -> getDataSource(dbInstance), Objects::nonNull,3) + .getOrElse((e) -> logger.info("Failed to get DataSource after 3 tries")); return datasource == null ? null : new JdbcTemplate(datasource); } @@ -65,7 +67,8 @@ public static TransactionTemplate getTransactionTemplate(DataSource dataSource) * @return */ public static SimpleJdbcTemplate getSimpleTemplate(DbInstance dbInstance) { - DataSource datasource = getIf( () -> getDataSource(dbInstance), (ds) -> ds != null,3); // return null if failed after 3 tries + DataSource datasource = Try.until( () -> getDataSource(dbInstance), Objects::nonNull,3) + .getOrElse((e) -> logger.info("Failed to get DataSource after 3 tries")); return datasource == null ? null : new SimpleJdbcTemplate(datasource); } diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/events/CacheEventWorker.java b/src/firefly/java/edu/caltech/ipac/firefly/server/events/CacheEventWorker.java index 95c1a93f3c..6f050ba1c1 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/events/CacheEventWorker.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/events/CacheEventWorker.java @@ -24,7 +24,10 @@ /** * @author Trey Roby + * + * @deprecated - no longer using replicating cache; use {@link edu.caltech.ipac.firefly.server.events.MessageEventWorker} instead */ +@Deprecated public class CacheEventWorker implements ServerEventManager.EventWorker { private static final Cache cache= CacheManager.getDistributed(); diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/events/MessageEventWorker.java b/src/firefly/java/edu/caltech/ipac/firefly/server/events/MessageEventWorker.java new file mode 100644 index 0000000000..37ff6ddfa2 --- /dev/null +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/events/MessageEventWorker.java @@ -0,0 +1,36 @@ +/* + * License information at https://github.com/Caltech-IPAC/firefly/blob/master/License.txt + */ +package edu.caltech.ipac.firefly.server.events; + +import edu.caltech.ipac.firefly.data.ServerEvent; +import edu.caltech.ipac.firefly.messaging.Message; +import edu.caltech.ipac.firefly.messaging.Messenger; + +import static edu.caltech.ipac.firefly.messaging.Message.Event.TOPIC; + +/** + * Use Messenger to deliver events. + */ +public class MessageEventWorker implements ServerEventManager.EventWorker { + + public MessageEventWorker() { + Messenger.subscribe(TOPIC, msg -> { + ServerEvent sev = Message.Event.parse(msg); + if (sev != null) { + processEvent(sev); + } + }); + } + + public void deliver(ServerEvent sev) { + if (sev != null) { + Messenger.publish(new Message.Event(sev)); + } + } + + public void processEvent(final ServerEvent sev) { + ServerEventManager.processEvent(sev); + } +} + diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/events/ReplicatedQueueList.java b/src/firefly/java/edu/caltech/ipac/firefly/server/events/ReplicatedQueueList.java index 6c09abb3f4..b4b2641d3e 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/events/ReplicatedQueueList.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/events/ReplicatedQueueList.java @@ -20,31 +20,21 @@ */ class ReplicatedQueueList { - private static final String HOST_NAME= FileUtil.getHostname(); - private static final StringKey REP_QUEUE_MAP = new StringKey("ReplicatedEventQueueMap"); - private static Cache getCache() { return CacheManager.getLocal(); } + private static final StringKey HOST_NAME= new StringKey(FileUtil.getHostname()); + private static final String REP_QUEUE_MAP = "ReplicatedEventQueueMap"; + private static Cache getCache() { return CacheManager.getDistributedMap(REP_QUEUE_MAP); } synchronized void setQueueListForNode(List list) { Cache cache= getCache(); - List replicatedList= new ArrayList<>(); - - for(ServerEventQueue q : list) { - replicatedList.add(new ServerEventQueue(q.getConnID(),q.getChannel(),q.getUserKey(),null)); - } - Map allListMap= (Map)cache.get(REP_QUEUE_MAP); - if (allListMap==null) allListMap= new HashMap(); - allListMap.put(HOST_NAME,replicatedList); - cache.put(REP_QUEUE_MAP, allListMap); + cache.put(HOST_NAME, list); } synchronized List getCombinedNodeList() { - Cache cache= getCache(); List retList= new ArrayList<>(); - Map allListMap= (Map)cache.get(REP_QUEUE_MAP); - if (allListMap==null) return Collections.emptyList(); - - for(Object v : allListMap.values()) retList.addAll((List)v); - + Cache cache= getCache(); + for(String k : cache.getKeys()) { + retList.addAll((List)cache.get(new StringKey(k))); + } return retList; } diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/events/ServerEventManager.java b/src/firefly/java/edu/caltech/ipac/firefly/server/events/ServerEventManager.java index 58f42f113f..c0181c03d0 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/events/ServerEventManager.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/events/ServerEventManager.java @@ -23,14 +23,14 @@ */ public class ServerEventManager { - private static final boolean USE_CACHE_EVENT_WORKER = true; - private static final EventWorker eventWorker = USE_CACHE_EVENT_WORKER ? - new CacheEventWorker() : new SimpleEventWorker(); - private static final List evQueueList= new CopyOnWriteArrayList(); + private static final boolean USE_MESSAGE_EVENT_WORKER = true; + private static final EventWorker eventWorker = USE_MESSAGE_EVENT_WORKER ? + new MessageEventWorker() : new LocalEventWorker(); + private static final List localEventQueues = new CopyOnWriteArrayList<>(); + private static final ReplicatedQueueList allEventQueues = new ReplicatedQueueList(); private static final Logger.LoggerImpl LOG = Logger.getLogger(); private static long totalEventCnt; private static long deliveredEventCnt; - private static ReplicatedQueueList repQueueList= new ReplicatedQueueList(); /** @@ -99,22 +99,30 @@ public static void fireEvent(ServerEvent sev) { public static void addEventQueue(ServerEventQueue queue) { Logger.briefInfo("Channel: create new Queue for: "+ queue.getQueueID() ); - evQueueList.add(queue); - repQueueList.setQueueListForNode(evQueueList); + localEventQueues.add(queue); + allEventQueues.setQueueListForNode(localEventQueues); } - static List getEvQueueList() { - return evQueueList; + /** + * Get the list of ServerEventQueue that are local to this node. + * @return list of ServerEventQueue + */ + static List getLocalEventQueues() { + return localEventQueues; } - static List getAllServerEvQueueList() { - return repQueueList.getCombinedNodeList(); + /** + * Get the list of all ServerEventQueue across all nodes(multiple instances of Firefly). + * @return list of ServerEventQueue + */ + static List getAllEventQueue() { + return allEventQueues.getCombinedNodeList(); } static void processEvent(ServerEvent ev) { totalEventCnt++; boolean delivered = false; - for(ServerEventQueue queue : evQueueList) { + for(ServerEventQueue queue : localEventQueues) { try { if (queue.matches(ev)) { try { @@ -137,8 +145,8 @@ static void processEvent(ServerEvent ev) { } public static void removeEventQueue(ServerEventQueue queue) { - evQueueList.remove(queue); - repQueueList.setQueueListForNode(evQueueList); + localEventQueues.remove(queue); + allEventQueues.setQueueListForNode(localEventQueues); } //==================================================================== @@ -147,7 +155,7 @@ public static void removeEventQueue(ServerEventQueue queue) { public static int getActiveQueueCnt() { int cnt = 0; - for(ServerEventQueue queue : evQueueList) { + for(ServerEventQueue queue : localEventQueues) { if (queue.getEventConnector().isOpen()) { cnt++; } @@ -156,7 +164,7 @@ public static int getActiveQueueCnt() { } public static List getQueueDescriptionList(int limit) { - return evQueueList.stream() + return localEventQueues.stream() .map(ServerEventQueue::convertToDescription) .sorted((d1,d2) -> (int)(d2.lastPutTime()-d1.lastPutTime())) .limit(limit) @@ -173,7 +181,7 @@ public static List getQueueDescriptionList(in public static int getActiveQueueChannelCnt(String channel) { int cnt = 0; if (StringUtils.isEmpty(channel)) return 0; - for(ServerEventQueue queue : evQueueList) { + for(ServerEventQueue queue : localEventQueues) { if (channel.equals(queue.getChannel()) && queue.getEventConnector().isOpen()) { cnt++; } else { @@ -198,10 +206,10 @@ public static long getDeliveredEventCnt() { //==================================================================== public interface EventWorker { - public void deliver(ServerEvent sev); + void deliver(ServerEvent sev); } - private static class SimpleEventWorker implements EventWorker { + private static class LocalEventWorker implements EventWorker { public void deliver(ServerEvent sev) { ServerEventManager.processEvent(sev); } diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/events/ServerEventQueue.java b/src/firefly/java/edu/caltech/ipac/firefly/server/events/ServerEventQueue.java index a81da0091e..9a524b6673 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/events/ServerEventQueue.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/events/ServerEventQueue.java @@ -14,6 +14,11 @@ import java.io.Serializable; /** + * This class manages server events and their delivery to clients. + * It holds information about the connection ID, channel, and user key associated with the event queue. + * The class provides methods to convert events to JSON, parse JSON events, and deliver events to the appropriate clients. + * It also includes functionality to match events based on their scope and target information. + * * @author Trey Roby */ public class ServerEventQueue implements Serializable { diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/events/WebsocketConnector.java b/src/firefly/java/edu/caltech/ipac/firefly/server/events/WebsocketConnector.java index f5f2cb01e7..83f55bb501 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/events/WebsocketConnector.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/events/WebsocketConnector.java @@ -142,7 +142,7 @@ public void close() { * @param userKey */ public static void pingClient(String userKey, String eventConnId) { - List conns = ServerEventManager.getEvQueueList(); + List conns = ServerEventManager.getLocalEventQueues(); for (ServerEventQueue seq : conns) { // need to notify clients that are affected by update if (seq.getConnID().equals(eventConnId) || seq.getUserKey().equals(userKey)) { @@ -164,7 +164,7 @@ public static void pingClient(String userKey, String eventConnId) { */ private void updateClientConnections(String type, String channelID, String userKey) { // List conns = ServerEventManager.getEvQueueList(); - List conns = ServerEventManager.getAllServerEvQueueList(); + List conns = ServerEventManager.getAllEventQueue(); for (ServerEventQueue seq : conns) { // need to notify clients that are affected by update if (seq.getChannel().equals(channelID) || seq.getUserKey().equals(userKey)) { diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/query/EmbeddedDbProcessor.java b/src/firefly/java/edu/caltech/ipac/firefly/server/query/EmbeddedDbProcessor.java index 1f3125a213..cbe2db4c9b 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/query/EmbeddedDbProcessor.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/query/EmbeddedDbProcessor.java @@ -3,6 +3,7 @@ */ package edu.caltech.ipac.firefly.server.query; +import edu.caltech.ipac.firefly.core.Util; import edu.caltech.ipac.firefly.server.ServCommand; import edu.caltech.ipac.firefly.server.db.DuckDbReadable; import edu.caltech.ipac.firefly.server.util.Logger; @@ -84,7 +85,7 @@ abstract public class EmbeddedDbProcessor implements SearchProcessor, SearchProcessor.CanGetDataFile, SearchProcessor.CanFetchDataGroup, Job.Worker { private static final Logger.LoggerImpl logger = Logger.getLogger(); - private static final SynchronizedAccess GET_DATA_CHECKER = new SynchronizedAccess(); + private static final Util.SynchronizedAccess GET_DATA_CHECKER = new Util.SynchronizedAccess(); private Job job; public void setJob(Job job) { @@ -124,7 +125,7 @@ public DataGroupPart getData(ServerRequest request) throws DataAccessException { // make sure multiple requests for the same data waits for the first one to create before accessing. String uniqueID = this.getUniqueID(request); - var release = GET_DATA_CHECKER.lock(uniqueID); + var locked = GET_DATA_CHECKER.lock(uniqueID); try { var dbAdapter = getDbAdapter(treq); jobExecIf(v -> v.progress(10, "fetching data...")); @@ -170,7 +171,7 @@ public DataGroupPart getData(ServerRequest request) throws DataAccessException { logger.error(e); throw e; } finally { - release.run(); + locked.unlock(); } } diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/query/SearchProcessor.java b/src/firefly/java/edu/caltech/ipac/firefly/server/query/SearchProcessor.java index d9024a01ac..6db288dc42 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/query/SearchProcessor.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/query/SearchProcessor.java @@ -24,8 +24,6 @@ import java.io.OutputStream; import java.util.List; import java.util.SortedSet; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.locks.ReentrantLock; import static edu.caltech.ipac.firefly.data.TableServerRequest.FF_SESSION_ID; import static edu.caltech.ipac.util.StringUtils.applyIfNotEmpty; @@ -103,28 +101,4 @@ interface CanGetDataFile { File getDataFile(TableServerRequest request) throws IpacTableException, IOException, DataAccessException; } - class SynchronizedAccess { - private final ConcurrentHashMap activeRequests = new ConcurrentHashMap<>(); - - /** - * Acquires a lock associated with the given ID. If the lock does not already exist, it is created. - * - * @param id the identifier for the lock - * @return a {@code Runnable} that, when executed, releases the lock and removes it from the active requests - */ - public Runnable lock(String id) { - ReentrantLock lock = activeRequests.computeIfAbsent(id, k -> new ReentrantLock()); - Logger.getLogger().trace("waiting %s: %s\n".formatted(id, lock)); - lock.lock(); - Logger.getLogger().trace("got lock %s: %s\n".formatted(id, lock)); - return () -> { - try { - lock.unlock(); // Ensure lock is released even if an exception occurs - } finally { - if (!lock.isLocked()) activeRequests.remove(id); // Remove the lock from activeRequests if no threads are using it - Logger.getLogger().trace("unlock %s: %s\n".formatted(id, lock)); - } - }; - } - } } diff --git a/src/firefly/java/edu/caltech/ipac/firefly/util/event/Name.java b/src/firefly/java/edu/caltech/ipac/firefly/util/event/Name.java index 1e3c162bf7..5d2bb97d82 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/util/event/Name.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/util/event/Name.java @@ -21,11 +21,10 @@ public class Name implements Serializable { public static final Name EVT_CONN_EST = new Name("EVT_CONN_EST", "Event connection established. Along with this event, you can expect connID and channel in the event's data. ie. {connID: val, channel: val}"); - public static final Name REPORT_USER_ACTION = new Name("REPORT_USER_ACTION", "report a user response"); - public static final Name ACTION = new Name("FLUX_ACTION", "an action message."); + public static final Name REPORT_USER_ACTION = new Name("REPORT_USER_ACTION", "report a user response"); + public static final Name ACTION = new Name("FLUX_ACTION", "an action message."); public static final Name PING = new Name("PING", "keepalive ping"); - private final String _name; private final String _desc; @@ -69,6 +68,11 @@ else if (other!=null && other instanceof Name) { * @return */ public static Name parse(String name) { + if (name==null) return null; + if (EVT_CONN_EST.getName().equals(name)) return EVT_CONN_EST; + if (REPORT_USER_ACTION.getName().equals(name)) return REPORT_USER_ACTION; + if (ACTION.getName().equals(name)) return ACTION; + if (PING.getName().equals(name)) return PING; return new Name(name, "unknown"); } } diff --git a/src/firefly/java/edu/caltech/ipac/table/io/TableParseHandler.java b/src/firefly/java/edu/caltech/ipac/table/io/TableParseHandler.java index 7d04669338..b21d85c9b0 100644 --- a/src/firefly/java/edu/caltech/ipac/table/io/TableParseHandler.java +++ b/src/firefly/java/edu/caltech/ipac/table/io/TableParseHandler.java @@ -22,8 +22,7 @@ import static edu.caltech.ipac.firefly.server.db.DuckDbAdapter.addRow; import static edu.caltech.ipac.firefly.server.db.EmbeddedDbUtil.colIdxWithArrayData; import static edu.caltech.ipac.firefly.core.Util.serialize; -import static edu.caltech.ipac.util.StringUtils.tryIt; - +import static edu.caltech.ipac.firefly.core.Util.Try; /** * Date: 10/23/24 * @@ -139,10 +138,9 @@ public void data(Object[] row) throws IOException { } public void end() { - tryIt(() -> { if (conn != null) conn.commit(); }); - tryIt(() -> { if (appender != null) appender.close(); }); - appender = null; - tryIt(() -> { if (conn != null) conn.close(); }); + if (conn != null) Try.it(() -> conn.commit()); + if (appender != null) Try.it(() -> appender.close()); + if (conn != null) Try.it(() -> conn.close()); } } } diff --git a/src/firefly/java/edu/caltech/ipac/util/StringUtils.java b/src/firefly/java/edu/caltech/ipac/util/StringUtils.java index d8b1b9179b..7ef108e948 100644 --- a/src/firefly/java/edu/caltech/ipac/util/StringUtils.java +++ b/src/firefly/java/edu/caltech/ipac/util/StringUtils.java @@ -18,7 +18,6 @@ import java.util.List; import java.util.Map; import java.util.function.Consumer; -import java.util.function.Predicate; import java.util.regex.Matcher; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -721,83 +720,6 @@ public static void applyIfNotEmpty(T v, Consumer f){ if (!isEmpty(v)) f.accept(v); } - /** - * A supplier that throws exception - * @param the return type of this function - */ - public interface FuncWithEx { - R get() throws Exception; - } - - /** - * @param func a function to execute - * @return the value from executing this func. Null if func produces exception - */ - public static R getSafe(FuncWithEx func) { - return getSafe(func, null); - } - - /** - * @param func a function to execute - * @param defaultVal return defaultVal if func produces an exception or the value is null - * @return the value from executing this func - */ - public static R getSafe(FuncWithEx func, R defaultVal) { - try { - R v = func.get(); - return v == null ? defaultVal : v; - } catch (Exception e) { - return defaultVal; - } - } - - /** - * A supplier that throws exception - */ - public interface TryWithEx { - void run() throws Exception; - } - - public static void tryIt(TryWithEx func) { - try { - func.run(); - } catch (Exception e) { - logger.warn("tryIf failed with:" + e.getMessage()); - } - } - - /** - * Execute the given function and return the value if it passes test - * @param func the function to execute - * @param test test the returned value - * @param defaultValue returns when encountering exception or test fail - * @return the value of func if it passes test. otherwise, return null - */ - public static R getIf(FuncWithEx func, Predicate test, R defaultValue) { - try { - var result = func.get(); - if (test.test(result)) return result; - } catch (Exception e) { - logger.info("returning (%s) because %s failed: %s".formatted(defaultValue, func.toString(), e.getMessage())); - } - return defaultValue; - } - - /** - * Execute the given function and return the value if it passes test - * @param func the function to execute - * @param test test the returned value - * @param tries the number of times to try - * @return the value of func if it passes test. otherwise, return null - */ - public static R getIf(FuncWithEx func, Predicate test, int tries) { - for (int i = 0; i < tries; i++) { - R v = getIf(func, test, null); - if (v != null) return v; - } - return null; - } - /** * Return a URL if the input can be successfully parsed as a URL; otherwise, return null. * @param urlString URL string diff --git a/src/firefly/test/edu/caltech/ipac/firefly/core/RedisServiceTest.java b/src/firefly/test/edu/caltech/ipac/firefly/core/RedisServiceTest.java index b8e340c893..0a5d1d8a93 100644 --- a/src/firefly/test/edu/caltech/ipac/firefly/core/RedisServiceTest.java +++ b/src/firefly/test/edu/caltech/ipac/firefly/core/RedisServiceTest.java @@ -8,9 +8,9 @@ import edu.caltech.ipac.firefly.server.util.Logger; import edu.caltech.ipac.util.AppProperties; import org.apache.logging.log4j.Level; -import org.junit.After; +import org.junit.AfterClass; import org.junit.Assert; -import org.junit.Before; +import org.junit.BeforeClass; import org.junit.Test; import redis.clients.jedis.Jedis; @@ -28,17 +28,17 @@ */ public class RedisServiceTest extends ConfigTest { - @Before - public void setup() { + @BeforeClass + public static void setup() { RedisService.connect(); if (RedisService.isOffline()) { System.out.println("Messenger is offline; skipping test."); } - if (true) Logger.setLogLevel(Level.TRACE); // for debugging. + if (false) Logger.setLogLevel(Level.TRACE); // for debugging. } - @After - public void teardown() { + @AfterClass + public static void teardown() { RedisService.disconnect(); LOG.trace("tear down"); } diff --git a/src/firefly/test/edu/caltech/ipac/firefly/core/UtilTest.java b/src/firefly/test/edu/caltech/ipac/firefly/core/UtilTest.java new file mode 100644 index 0000000000..5162c60a65 --- /dev/null +++ b/src/firefly/test/edu/caltech/ipac/firefly/core/UtilTest.java @@ -0,0 +1,179 @@ +/* + * License information at https://github.com/Caltech-IPAC/firefly/blob/master/License.txt + */ + +package edu.caltech.ipac.firefly.core; + +/** + * Date: 12/20/24 + * + * @author loi + * @version : $ + */ +import edu.caltech.ipac.firefly.ConfigTest; +import edu.caltech.ipac.firefly.server.util.Logger; +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; + +import static org.junit.Assert.*; +import static edu.caltech.ipac.firefly.core.Util.*; + +public class UtilTest extends ConfigTest { + + @Test + public void arrayOfBoolean() { + Object in = new Boolean[]{true, false, true}; + String s = serialize(in); + Object d = deserialize(s); + if (d instanceof Boolean[] v) { + Assert.assertArrayEquals(v, (Boolean[]) in); + } else Assert.fail("Deserialized type Boolean mismatch"); + } + + @Test + public void arrayOfDouble() { + Object in = new Double[]{1.0, 2.0, 3.0}; + String s = serialize(in); + Object d = deserialize(s); + if (d instanceof Double[] v) { + Assert.assertArrayEquals(v, (Double[]) in); + } else Assert.fail("Deserialized type Double mismatch"); + } + + @Test + public void arrayOfInt() { + Object in = new Integer[]{1, 2, 3}; + String s = serialize(in); + Object d = deserialize(s); + if (d instanceof Integer[] v) { + Assert.assertArrayEquals(v, (Integer[])in); + } else Assert.fail("Deserialized type Integer mismatch"); + } + + @Test + public void serializeValidObject() { + String original = "testString"; + String serialized = serialize(original); + assertNotNull(serialized); + } + + @Test + public void serializeNullObject() { + String serialized = serialize(null); + assertNull(serialized); + } + + @Test + public void deserializeValidString() { + String original = "testString"; + String serialized = serialize(original); + Object deserialized = deserialize(serialized); + assertEquals(original, deserialized); + } + + @Test + public void deserializeInvalidString() { + Object deserialized = deserialize("invalidBase64"); + assertNull(deserialized); + } + + @Test + public void deserializeNullString() { + Object deserialized = deserialize(null); + assertNull(deserialized); + } + + @Test + public void tryItFuncWithExSuccess() { + assertEquals("success", Try.it(() -> "success").getOrElse("default")); + } + + @Test + public void tryItFuncWithExFailure() { + Try result = Try.it(() -> { throw new Exception("failure"); }); + assertEquals("default", result.getOrElse("default")); + } + + @Test + public void tryItCallWithExSuccess() { + Try result = Try.it(() -> {}); + assertNull(result.get()); + } + + @Test + public void tryItCallWithExFailure() { + Try result = Try.it(() -> { throw new Exception("failure"); }); + assertNull(result.getOrElse((e) -> Logger.getLogger().trace("error"))); + } + + @Test + public void tryItFuncParamWithExSuccess() { + Try result = Try.it((param) -> param, "success"); + assertEquals("success", result.getOrElse("default")); + } + + @Test + public void tryItFuncParamWithExFailure() { + Try result = Try.it((param) -> { throw new Exception("failure"); }, "param"); + assertEquals("default", result.getOrElse("default")); + } + + @Test + public void tryUntilSuccess() { + AtomicInteger count = new AtomicInteger(); + Try result = Try.until(count::getAndIncrement, c -> c == 3, 5); + assertEquals(3, result.get().intValue()); + } + + @Test + public void testSynchronizedAccess() throws InterruptedException { + ArrayList even = new ArrayList<>(); + ArrayList odd = new ArrayList<>(); + + var locker = new Util.SynchronizedAccess(); + Function setResults = (q) -> { + long start = System.currentTimeMillis(); + var locked = locker.lock(q); + try { + Thread.sleep(1_000); + } catch (InterruptedException e) {} + finally { + locked.unlock(); + } + return System.currentTimeMillis() - start; + }; + + int ntimes = 10; + var p = Executors.newFixedThreadPool(ntimes); // when all threads start at the same time, all be blocked. + for(int i = 0; i < ntimes; i++) { + long a = i % 2; + p.submit(() -> { + if (a == 0 ) { + even.add(Math.round(setResults.apply("even")/1000.0)); + } else { + odd.add(Math.round(setResults.apply("odd")/1000.0)); + } + }); + } + p.shutdown(); + if (!p.awaitTermination(10, TimeUnit.SECONDS)) { + System.out.println("Not all tasks completed in time."); + } +// even.forEach(System.out::println); +// odd.forEach(System.out::println); + + assertEquals(ntimes/2, (long) Collections.max(even)); + assertEquals(1L, (long)Collections.min(even)); + assertEquals(ntimes/2, (long)Collections.max(odd)); + assertEquals(1L, (long)Collections.min(odd)); + } + + +} \ No newline at end of file diff --git a/src/firefly/test/edu/caltech/ipac/firefly/messaging/MessengerTest.java b/src/firefly/test/edu/caltech/ipac/firefly/messaging/MessengerTest.java index 9927e4ae21..8b5e068b82 100644 --- a/src/firefly/test/edu/caltech/ipac/firefly/messaging/MessengerTest.java +++ b/src/firefly/test/edu/caltech/ipac/firefly/messaging/MessengerTest.java @@ -42,7 +42,7 @@ public void setup() { System.out.println("Messenger is offline; skipping test."); isOffline = true; } - if (true) Logger.setLogLevel(Level.TRACE); // for debugging. + if (false) Logger.setLogLevel(Level.TRACE); // for debugging. } @After diff --git a/src/firefly/test/edu/caltech/ipac/table/DuckDbAdapterTest.java b/src/firefly/test/edu/caltech/ipac/table/DuckDbAdapterTest.java index 3e6703200f..037c6766ca 100644 --- a/src/firefly/test/edu/caltech/ipac/table/DuckDbAdapterTest.java +++ b/src/firefly/test/edu/caltech/ipac/table/DuckDbAdapterTest.java @@ -16,7 +16,6 @@ import edu.caltech.ipac.firefly.server.query.DecimationProcessor; import edu.caltech.ipac.firefly.server.query.EmbeddedDbProcessor; import edu.caltech.ipac.firefly.server.query.SearchManager; -import edu.caltech.ipac.firefly.server.query.SearchProcessor; import edu.caltech.ipac.firefly.server.query.tables.IpacTableFromSource; import edu.caltech.ipac.firefly.server.util.Logger; import edu.caltech.ipac.firefly.util.FileLoader; @@ -29,13 +28,8 @@ import org.junit.Test; import java.io.File; -import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.List; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; -import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -253,49 +247,6 @@ public void testCleanup() throws DataAccessException { assertFalse("DuckDb file is deleted", dbAdapter.getDbFile().exists()); } - @Test - public void testSynchronizedAccess() throws InterruptedException { - ArrayList even = new ArrayList<>(); - ArrayList odd = new ArrayList<>(); - - var locker = new SearchProcessor.SynchronizedAccess(); - Function setResults = (q) -> { - long start = System.currentTimeMillis(); - var release = locker.lock(q); - try { - Thread.sleep(1_000); - } catch (InterruptedException e) {} - finally { - release.run(); - } - return System.currentTimeMillis() - start; - }; - - int ntimes = 10; - var p = Executors.newFixedThreadPool(ntimes); // when all threads start at the same time, all be blocked. - for(int i = 0; i < ntimes; i++) { - long a = i % 2; - p.submit(() -> { - if (a == 0 ) { - even.add(Math.round(setResults.apply("even")/1000.0)); - } else { - odd.add(Math.round(setResults.apply("odd")/1000.0)); - } - }); - } - p.shutdown(); - if (!p.awaitTermination(10, TimeUnit.SECONDS)) { - System.out.println("Not all tasks completed in time."); - } -// even.forEach(System.out::println); -// odd.forEach(System.out::println); - - assertEquals(ntimes/2, (long)Collections.max(even)); - assertEquals(1L, (long)Collections.min(even)); - assertEquals(ntimes/2, (long)Collections.max(odd)); - assertEquals(1L, (long)Collections.min(odd)); - } - @Test public void testLikeSubstitution() { // replace uppercase LIKE diff --git a/src/firefly/test/edu/caltech/ipac/table/EmbeddedDbUtilTest.java b/src/firefly/test/edu/caltech/ipac/table/EmbeddedDbUtilTest.java index 2e0a6aba5b..b1d800a8b3 100644 --- a/src/firefly/test/edu/caltech/ipac/table/EmbeddedDbUtilTest.java +++ b/src/firefly/test/edu/caltech/ipac/table/EmbeddedDbUtilTest.java @@ -308,31 +308,4 @@ public void testAddColumnErrors() { Assert.assertEquals("TABLE out-of-sync; Reload table to resume", e.getMessage()); } } - - @Test - public void testSerializer() { - // array of boolean - Object in = new Boolean[]{true, false, true}; - String s = Util.serialize(in); - Object d = Util.deserialize(s); - if (d instanceof Boolean[] v) { - Assert.assertArrayEquals(v, (Boolean[])in); - } else Assert.fail("Deserialized type Boolean mismatch"); - - // array of double - in = new Double[]{1.0, 2.0, 3.0}; - s = Util.serialize(in); - d = Util.deserialize(s); - if (d instanceof Double[] v) { - Assert.assertArrayEquals(v, (Double[])in); - } else Assert.fail("Deserialized type Double mismatch"); - - // array of integer - in = new Integer[]{1, 2, 3}; - s = Util.serialize(in); - d = Util.deserialize(s); - if (d instanceof Integer[] v) { - Assert.assertArrayEquals(v, (Integer[])in); - } else Assert.fail("Deserialized type Integer mismatch"); - } } \ No newline at end of file diff --git a/src/firefly/test/edu/caltech/ipac/util/cache/CachePeerProviderFactoryTest.java b/src/firefly/test/edu/caltech/ipac/util/cache/CachePeerProviderFactoryTest.java index d9748286c5..c24ab4ac26 100644 --- a/src/firefly/test/edu/caltech/ipac/util/cache/CachePeerProviderFactoryTest.java +++ b/src/firefly/test/edu/caltech/ipac/util/cache/CachePeerProviderFactoryTest.java @@ -87,7 +87,7 @@ public static void setUp() throws InterruptedException { System.out.println("Messenger is offline; skipping all tests in CachePeerProviderFactoryTest."); return; } - if (true) Logger.setLogLevel(Level.TRACE); // for debugging. + if (false) Logger.setLogLevel(Level.TRACE); // for debugging. LOG.debug("Initial setup for PubSub, creating 3 peers"); peer1 = createCM("PubSub", "peer1"); diff --git a/src/firefly/test/edu/caltech/ipac/util/cache/CacheTest.java b/src/firefly/test/edu/caltech/ipac/util/cache/CacheTest.java index 09eee5d751..324cdcaae9 100644 --- a/src/firefly/test/edu/caltech/ipac/util/cache/CacheTest.java +++ b/src/firefly/test/edu/caltech/ipac/util/cache/CacheTest.java @@ -5,9 +5,11 @@ package edu.caltech.ipac.util.cache; import edu.caltech.ipac.firefly.ConfigTest; +import edu.caltech.ipac.firefly.core.RedisService; import edu.caltech.ipac.firefly.data.userdata.UserInfo; import edu.caltech.ipac.firefly.server.util.Logger; import org.apache.logging.log4j.Level; +import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; @@ -28,10 +30,15 @@ public class CacheTest extends ConfigTest { @BeforeClass public static void setUp() throws InterruptedException { + RedisService.connect(); setupServerContext(null); - if (true) Logger.setLogLevel(Level.TRACE); // for debugging. + if (false) Logger.setLogLevel(Level.TRACE); // for debugging. } + @AfterClass + public static void tearDown() { + RedisService.disconnect(); + } @Test public void localCache() { From 266e671ecd383711975353c7e99f82d0ebdc4f79 Mon Sep 17 00:00:00 2001 From: loi Date: Fri, 3 Jan 2025 15:55:58 -0800 Subject: [PATCH 3/3] - preinstall DuckDB magic extension in the docker image - minor JdbcFactory refactoring --- docker/Dockerfile | 6 ++ .../firefly/server/cache/EhcacheImpl.java | 12 ---- .../ipac/firefly/server/db/DbInstance.java | 2 +- .../ipac/firefly/server/db/DuckDbAdapter.java | 63 ++++++++++------- .../firefly/server/db/spring/JdbcFactory.java | 69 +++++++++---------- .../server/query/EmbeddedDbProcessor.java | 1 + .../edu/caltech/ipac/firefly/ConfigTest.java | 4 +- 7 files changed, 80 insertions(+), 77 deletions(-) diff --git a/docker/Dockerfile b/docker/Dockerfile index 14471ecc75..4abd020118 100644 --- a/docker/Dockerfile +++ b/docker/Dockerfile @@ -163,6 +163,12 @@ COPY firefly/docker/local.xml conf/Catalina/localhost #copy all wars, typically there should only be one COPY --from=builder /opt/work/${build_dir}/build/dist/*.war ${CATALINA_HOME}/webapps-ref/ +# preinstall DuckDB extensions; remember to update version when DuckDB is updated +ARG DUCKDB_TARGET=v1.1.3/linux_amd64_gcc4 +RUN mkdir -p temp/$DUCKDB_TARGET \ + && wget -P temp/$DUCKDB_TARGET http://community-extensions.duckdb.org/$DUCKDB_TARGET/magic.duckdb_extension.gz \ + && gunzip -f temp/$DUCKDB_TARGET/magic.duckdb_extension.gz + # Add permission to files and directories needed for runtime # increase max header size to avoid failing on large auth token WORKDIR ${CATALINA_HOME} diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/cache/EhcacheImpl.java b/src/firefly/java/edu/caltech/ipac/firefly/server/cache/EhcacheImpl.java index 139656c1d0..9c0bc99e30 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/cache/EhcacheImpl.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/cache/EhcacheImpl.java @@ -29,25 +29,15 @@ public EhcacheImpl(Ehcache cache) { } public void put(CacheKey key, Object value) { -// logger.briefDebug("cache pre-put:" + key + " = " + StringUtils.toString(value)); String keystr = key.getUniqueString(); if (value == null) { cache.remove(keystr); } else { cache.put(new Element(keystr, value)); } -// logger.briefDebug("cache aft-put:" + key + " = " + StringUtils.toString(value)); } public void put(CacheKey key, Object value, int lifespanInSecs) { -// logger.briefDebug("cache pre-put:" + key + " = " + StringUtils.toString(value) + -// " lifespanInSecs:" + lifespanInSecs); - - if (!cache.getCacheConfiguration().isEternal()) { - throw new UnsupportedOperationException("Currently, we do not support cached object" + - " with idle time expiry and lifespan expiry at the same time."); - } - String keystr = key.getUniqueString(); if (value == null) { cache.remove(keystr); @@ -56,8 +46,6 @@ public void put(CacheKey key, Object value, int lifespanInSecs) { el.setTimeToLive(lifespanInSecs); cache.put(el); } -// logger.briefDebug("cache aft-put:" + key + " = " + StringUtils.toString(value) + -// " lifespanInSecs:" + lifespanInSecs); } public Object get(CacheKey key) { diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/db/DbInstance.java b/src/firefly/java/edu/caltech/ipac/firefly/server/db/DbInstance.java index a3f43da12d..c222699495 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/db/DbInstance.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/db/DbInstance.java @@ -109,7 +109,7 @@ public String getDbUrl() { public void setPooled(boolean pooled) { isPooled = pooled;} public boolean testConn(Connection conn) { try (Statement stmt = conn.createStatement()) { - stmt.execute("SELECT 1 FROM (VALUES (0))"); // HSQL required FROM clause + stmt.execute("SELECT 1 FROM (VALUES (0)) AS dummy"); // HSQL required FROM clause; postgres required alias return true; } catch (SQLException e) { return false; } }; diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/db/DuckDbAdapter.java b/src/firefly/java/edu/caltech/ipac/firefly/server/db/DuckDbAdapter.java index 633785b747..dca66d4cc2 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/db/DuckDbAdapter.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/db/DuckDbAdapter.java @@ -9,6 +9,7 @@ import edu.caltech.ipac.firefly.server.db.spring.JdbcFactory; import edu.caltech.ipac.firefly.server.query.DataAccessException; import edu.caltech.ipac.firefly.server.util.Logger; +import edu.caltech.ipac.firefly.util.Ref; import edu.caltech.ipac.table.DataGroup; import edu.caltech.ipac.table.DataType; import edu.caltech.ipac.util.AppProperties; @@ -29,8 +30,11 @@ import java.time.LocalDate; import java.time.LocalDateTime; import java.time.ZoneOffset; +import java.util.Arrays; import java.util.Date; +import java.util.HashMap; import java.util.List; +import java.util.Map; import static edu.caltech.ipac.firefly.core.Util.Try; import static edu.caltech.ipac.firefly.server.db.DuckDbUDF.*; @@ -44,6 +48,7 @@ public class DuckDbAdapter extends BaseDbAdapter { public static final String NAME = "duckdb"; public static final String DRIVER = "org.duckdb.DuckDBDriver"; + public static final String EXT_DIR = AppProperties.getProperty("duckdb.ext.dir", System.getProperty("java.io.tmpdir")); public static String maxMemory = AppProperties.getProperty("duckdb.max.memory"); // in GB; 2G, 5.5G, etc private static int threadCnt=1; // min 125mb per thread. recommend 5gb per thread; we will config 1gb per thread but not more than 4. @@ -87,7 +92,7 @@ public boolean testConn(Connection conn) { } catch (SQLException e) { return false; } } }; - db.consumeProps("memory_limit=%s,threads=%d".formatted(maxMemory, threadCnt)); + db.consumeProps("memory_limit=%s,threads=%d,extension_directory=%s".formatted(maxMemory, threadCnt, EXT_DIR)); return db; } @@ -279,34 +284,40 @@ public static String replaceLike(String input) { public record MimeDesc(String mime, String desc) {} @Nonnull public static MimeDesc getMimeType(File inFile) { - String tmpDir = System.getProperty("java.io.tmpdir"); - try (Connection conn = new DuckDbAdapter().getJdbcTmpl().getDataSource().getConnection(); - Statement stmt = conn.createStatement(); - ResultSet extrs = stmt.executeQuery("SET extension_directory = '%s'; SELECT installed, loaded FROM duckdb_extensions() WHERE extension_name = 'magic'".formatted(tmpDir))) { - - boolean installed = false; - boolean loaded = false; - if (extrs.next()) { - installed = extrs.getBoolean("installed"); - loaded = extrs.getBoolean("loaded"); + try(JdbcFactory.SharedDS ds = JdbcFactory.getSharedDS(new DuckDbAdapter().createDbInstance())) { + loadExtension(ds, "magic"); + Map rs = ds.getJdbc().queryForMap("SELECT file, magic_mime(file) AS mime, magic_type(file) AS desc FROM glob('%s')".formatted(inFile.getAbsolutePath())); + if (!rs.isEmpty()) { + return new MimeDesc(String.valueOf(rs.get("mime")), String.valueOf(rs.get("desc"))); } - // Install and load the extension if not already done - if (!installed) { - stmt.executeUpdate("SET extension_directory = '%s'; INSTALL magic FROM community".formatted(tmpDir)); - } - if (!loaded) { - stmt.executeUpdate("SET extension_directory = '%s'; LOAD magic".formatted(tmpDir)); - } - try (ResultSet rs = stmt.executeQuery("SELECT file, magic_mime(file) AS mime, magic_type(file) AS desc FROM glob('%s')".formatted(inFile.getAbsolutePath()))) { - if (rs.next()) { - return new MimeDesc(rs.getString("mime"), rs.getString("desc")); - } - } - - } catch (SQLException ex) { + } catch (Exception ex) { Logger.getLogger().error(ex, "Failed to detect mime type"); } return new MimeDesc("application/x-unknown", "unknown"); } -} + /** + * Loads the specified extension in DuckDB. + * + * @param ds the shared data source + * @param name the name of the extension to load + */ + static void loadExtension(JdbcFactory.SharedDS ds, String name) { + SimpleJdbcTemplate jdbc = ds.getJdbc(); + Ref installed = new Ref<>(false); + Ref loaded = new Ref<>(false); + jdbc.query("SELECT installed, loaded FROM duckdb_extensions() WHERE extension_name = '%s'".formatted(name), (rs, idx) -> { + installed.set(rs.getBoolean("installed")); + loaded.set(rs.getBoolean("loaded")); + return null; + }); + // Install and load the extension if not already done + if (!installed.get()) { + jdbc.update("INSTALL %s FROM community".formatted(name)); + } + if (!loaded.get()) { + jdbc.update("LOAD %s".formatted(name)); + } + } + +} \ No newline at end of file diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/db/spring/JdbcFactory.java b/src/firefly/java/edu/caltech/ipac/firefly/server/db/spring/JdbcFactory.java index 98026b2d09..c4d12dc7eb 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/db/spring/JdbcFactory.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/db/spring/JdbcFactory.java @@ -73,43 +73,16 @@ public static SimpleJdbcTemplate getSimpleTemplate(DbInstance dbInstance) { } /** - * return a JdbcTemplate with a single underlying connection. - * this allows a user to perform multiple tasks on one connection. - * this implementation is not thread-safe. - * @param dbInstance - * @return - */ - public static JdbcTemplate getStatefulTemplate(DbInstance dbInstance) { - DataSource ds = getSingleConnectionDS(dbInstance); - return ds == null ? null : new JdbcTemplate(ds); - } - - /** - * return a simple JdbcTemplate with a single underlying connection. - * this allow a user to perform multiple tasks on one connection. - * this implementation is not thread-safe. - * @param dbInstance - * @return + * Returns a DataSource wrapper that ensures the underlying connection is not closed automatically. + * The connection will remain open until explicitly closed by the caller, + * allowing multiple tasks to be executed on the same connection. + * Note: This implementation is not thread-safe. + * + * @param dbInstance the database to connect to + * @return a {@link SharedDS} instance for the specified database instance */ - public static SimpleJdbcTemplate getStatefulSimpleTemplate(DbInstance dbInstance) { - DataSource ds = getSingleConnectionDS(dbInstance); - return ds == null ? null : new SimpleJdbcTemplate(ds); - } - - /** - * return a DataSource with a single underlying connection. - * this allow a user to perform multiple tasks on one connection. - * this implementation is not thread-safe. - * @param dbInstance - * @return - */ - public static DataSource getSingleConnectionDS(DbInstance dbInstance) { - try { - return new SingleConnectionDataSource(getDataSource(dbInstance).getConnection(), false); - } catch (SQLException e) { - logger.error(e); - } - return null; + public static SharedDS getSharedDS(DbInstance dbInstance) { + return new SharedDS(dbInstance); } public static DataSource getDataSource(DbInstance dbInstance) { @@ -179,4 +152,28 @@ private Properties addProps(Properties props) { } } + + public static class SharedDS implements AutoCloseable { + DataSource ds; + public SharedDS(DbInstance di) { + try { + ds = new SingleConnectionDataSource(getDataSource(di).getConnection(), false); + } catch (Exception e) { + logger.error(e); + throw new IllegalArgumentException("Failed to get DataSource"); + } + } + public void close() throws Exception { + Try.it(() -> ds.getConnection().close()); + } + public DataSource get() { + return ds; + } + public SimpleJdbcTemplate getJdbc() { + return new SimpleJdbcTemplate(ds); + } + public JdbcTemplate getTmpl() { + return new JdbcTemplate(ds); + } + } } diff --git a/src/firefly/java/edu/caltech/ipac/firefly/server/query/EmbeddedDbProcessor.java b/src/firefly/java/edu/caltech/ipac/firefly/server/query/EmbeddedDbProcessor.java index cbe2db4c9b..a9f0cf0049 100644 --- a/src/firefly/java/edu/caltech/ipac/firefly/server/query/EmbeddedDbProcessor.java +++ b/src/firefly/java/edu/caltech/ipac/firefly/server/query/EmbeddedDbProcessor.java @@ -193,6 +193,7 @@ protected void createDbFromRequest(TableServerRequest treq, DbAdapter dbAdapter) } } } catch (Exception e) { + logger.error(e); dbAdapter.close(true); throw dbAdapter.handleSqlExp("", e); } diff --git a/src/firefly/test/edu/caltech/ipac/firefly/ConfigTest.java b/src/firefly/test/edu/caltech/ipac/firefly/ConfigTest.java index b306aef086..5e1839801b 100644 --- a/src/firefly/test/edu/caltech/ipac/firefly/ConfigTest.java +++ b/src/firefly/test/edu/caltech/ipac/firefly/ConfigTest.java @@ -118,13 +118,13 @@ public static void setupServerContext(RequestAgent requestAgent, String contextN webappConfigPath = webappConfigPath == null ? Paths.get("build/%s/war/WEB-INF/config".formatted(contextName)).toAbsolutePath().toString() : webappConfigPath; AppProperties.setProperty("work.directory", Paths.get("build").toAbsolutePath().toString()); - Path buildConfg = Paths.get("build/firefly/war/WEB-INF/config"); + Path buildConfg = Paths.get(webappConfigPath); + System.setProperty("java.io.tmpdir", "build/%s/tmp".formatted(contextName)); copyWithSub(Paths.get("./config/ehcache.xml"), buildConfg, "app-name", contextName); copy(Paths.get("config/test/app-test.prop"), buildConfg); copy(Paths.get("config/ignore_sizeof.txt"), buildConfg); - webappConfigPath = webappConfigPath != null ? webappConfigPath : buildConfg.toAbsolutePath().toString(); requestAgent = requestAgent == null ? new RequestAgent(null, "localhost", "/test", "localhost:8080/", "127.0. 0.1", UUID.randomUUID().toString(), contextPath): requestAgent; ServerContext.getRequestOwner().setRequestAgent(requestAgent);