diff --git a/conf/smart-default.xml b/conf/smart-default.xml index 27c480859d6..dbfdfdf57f0 100644 --- a/conf/smart-default.xml +++ b/conf/smart-default.xml @@ -367,15 +367,6 @@ - - smart.metastore.mysql.legacy.enabled - false - - Whether to enable support for versions of MySQL lesser than 5.7. - Warning: it requires admin privileges for ssm user to be able to configure the db. - - - smart.ignore.path.templates @@ -393,45 +384,7 @@ - smart.access.count.day.tables.num - 30 - - The max number of access count per day tables in the Metastore. - - - - - smart.access.count.hour.tables.num - 48 - - The max number of access count per hour tables in the Metastore. - It should be at least 24 to cover the full day to be successfully - aggregated into a per day access count table. - - - - - smart.access.count.minute.tables.num - 120 - - The max number of access count per minute tables in the Metastore. - It should be at least 60 to cover the full hour to be successfully - aggregated into a per hour access count table. - - - - - smart.access.count.second.tables.num - 30 - - The max number of access count per second tables in the Metastore. - It should be at least 60000/'smart.access.count.aggregation.interval.ms' - to cover the full minute to be successfully aggregated into a per minute access count table. - - - - - smart.access.event.fetch.interval.ms + smart.file.access.event.fetch.interval.ms 1000 The interval in milliseconds between access event fetches. @@ -479,7 +432,7 @@ - smart.access.count.aggregation.interval.ms + smart.file.access.count.aggregation.interval.ms 5000 The interval in milliseconds that is covered by single second-granularity access count table. @@ -578,17 +531,22 @@ - smart.access.count.aggregator.failover - SUBMIT_NEW_FAILED_EVENTS_LATER + smart.file.access.count.aggregator.failover.retry.count + 60 + + Maximum number of attempts to save file access events + + + + + smart.file.access.count.aggregator.failover + SUBMIT_FAILED_EVENTS_WITH_RETRY Failover strategy for file access events aggregator. Possible values: - DROP_EVENTS - drop file access events that caused exception. FAIL - throw exception, no failover. - SUBMIT_FAILED_EVENTS_LATER - save all file access events that caused exception - for later submission. Should be used carefully, because in case of repeated - exceptions can potentially cause OOM error. - SUBMIT_NEW_FAILED_EVENTS_LATER - save new file access events that caused exception - for later submission and drop old failing events. + SUBMIT_FAILED_EVENTS_WITH_RETRY - save all file access events that caused exception + for later submission with max attempts less or equals than smart.access.count.aggregator.failover.retry.count + diff --git a/docs/admin-user-guide.md b/docs/admin-user-guide.md index 538511651b1..1d0785fdfb0 100755 --- a/docs/admin-user-guide.md +++ b/docs/admin-user-guide.md @@ -78,23 +78,21 @@ Table-4 Condition Ingredient Table-5 Object Properties -| Object | Property | Description | -|----------|----------------------------------|---------------------------------------------------------------------------------------------| -| | age | The time span from last modification moment to now | -| | atime | The last access time | -| | blocksize | The block size of the file | -| | inCache | The file is in cache storage | -| | isDir | The file is a directory | -| | length | Length of the file. Currently, only pure digital is supported, which indicates bytes number.| -| file | path | The file path in HDFS | -| | mtime | The last modification time of the file | -| | unsynced | The file is not synced | -| | storagePolicy | Storage policy of file | -| | accessCount(Time Interval) | The access counts during the last time interval | -| | accessCountTop(interval,N ) | The topmost N for access counts during the last time interval | -| | accessCountBottom(interval,N) | The bottommost N for access counts during the last time interval | -| | accessCountTopOnStoragePolicy(interval,N,$StoragePolicy") | The topmost N for access counts with regard to a storage policy.The supported HDFS storage policies are COLD,WARM,HOT,ONE_SSD,ALL_SSD,LAZY_PERSIST | -| | accessCountBottomOnStoragePolicy(interval,N,$StoragePolicy") | The bottommost N for access counts with regard to a storage policy during the last time interval | +| Object | Property | Description | +|----------|--------------------------------------------------------------|----------------------------------------------------------------------------------------------------------------------------------------------------| +| | age | The time span from last modification moment to now | +| | atime | The last access time | +| | blocksize | The block size of the file | +| | inCache | The file is in cache storage | +| | isDir | The file is a directory | +| | length | Length of the file. Currently, only pure digital is supported, which indicates bytes number. | +| file | path | The file path in HDFS | +| | mtime | The last modification time of the file | +| | unsynced | The file is not synced | +| | storagePolicy | Storage policy of file | +| | accessCount(Time Interval) | The access counts during the last time interval | +| | accessCountTop(interval,N ) | The topmost N for access counts during the last time interval | +| | accessCountBottom(interval,N) | The bottommost N for access counts during the last time interval | Table-6 Commands diff --git a/smart-common/src/main/java/org/smartdata/conf/SmartConfKeys.java b/smart-common/src/main/java/org/smartdata/conf/SmartConfKeys.java index 366ce83b938..6363abc7b5d 100644 --- a/smart-common/src/main/java/org/smartdata/conf/SmartConfKeys.java +++ b/smart-common/src/main/java/org/smartdata/conf/SmartConfKeys.java @@ -79,41 +79,22 @@ public class SmartConfKeys { // Password which get from hadoop credentialProvider used for metastore connect public static final String SMART_METASTORE_PASSWORD = "smart.metastore.password"; - public static final String SMART_METASTORE_MIGRATION_CHANGELOG_PATH_KEY = - "smart.metastore.migration.liquibase.changelog.path"; - public static final String SMART_METASTORE_MIGRATION_CHANGELOG_PATH_DEFAULT = - "db/changelog/changelog-root.xml"; - public static final String SMART_METASTORE_LEGACY_MYSQL_SUPPORT_KEY = - "smart.metastore.mysql.legacy.enabled"; - public static final boolean SMART_METASTORE_LEGACY_MYSQL_SUPPORT_DEFAULT = false; - - public static final String SMART_ACCESS_COUNT_AGGREGATION_INTERVAL_MS = - "smart.access.count.aggregation.interval.ms"; - public static final int SMART_ACCESS_COUNT_AGGREGATION_INTERVAL_MS_DEFAULT = 5000; + public static final String SMART_METASTORE_MIGRATION_CHANGELOG_PATH_KEY = + "smart.metastore.migration.liquibase.changelog.path"; + public static final String SMART_METASTORE_MIGRATION_CHANGELOG_PATH_DEFAULT = + "db/changelog/changelog-root.xml"; public static final String SMART_ACCESS_COUNT_AGGREGATOR_FAILOVER_KEY = - "smart.access.count.aggregator.failover"; + "smart.file.access.count.aggregator.failover"; + public static final String SMART_ACCESS_COUNT_AGGREGATOR_FAILOVER_MAX_RETRIES_KEY = + "smart.file.access.count.aggregator.failover.retry.count"; + public static final int SMART_ACCESS_COUNT_AGGREGATOR_FAILOVER_MAX_RETRIES_DEFAULT = 60; - public static final String SMART_NUM_DAY_TABLES_TO_KEEP_KEY = - "smart.access.count.day.tables.num"; - public static final int SMART_NUM_DAY_TABLES_TO_KEEP_DEFAULT = 30; - - public static final String SMART_NUM_HOUR_TABLES_TO_KEEP_KEY = - "smart.access.count.hour.tables.num"; - public static final int SMART_NUM_HOUR_TABLES_TO_KEEP_DEFAULT = 48; - public static final int SMART_NUM_HOUR_TABLES_TO_KEEP_MIN = 24; - - public static final String SMART_NUM_MINUTE_TABLES_TO_KEEP_KEY = - "smart.access.count.minute.tables.num"; - public static final int SMART_NUM_MINUTE_TABLES_TO_KEEP_DEFAULT = 120; - public static final int SMART_NUM_MINUTE_TABLES_TO_KEEP_MIN = 60; - - public static final String SMART_NUM_SECOND_TABLES_TO_KEEP_KEY = - "smart.access.count.second.tables.num"; - public static final int SMART_NUM_SECOND_TABLES_TO_KEEP_DEFAULT = 30; + public static final String SMART_FILE_ACCESS_PARTITIONS_RETENTION_POLICY_KEY = + "smart.file.access.partition.retention.policy"; public static final String SMART_ACCESS_EVENT_FETCH_INTERVAL_MS_KEY = - "smart.access.event.fetch.interval.ms"; + "smart.file.access.event.fetch.interval.ms"; public static final long SMART_ACCESS_EVENT_FETCH_INTERVAL_MS_DEFAULT = 1000L; public static final String SMART_CACHED_FILE_FETCH_INTERVAL_MS_KEY = @@ -124,7 +105,12 @@ public class SmartConfKeys { "smart.namespace.fetch.interval.ms"; public static final long SMART_NAMESPACE_FETCH_INTERVAL_MS_DEFAULT = 1L; - // StatesManager + // File access partitions + public static final String SMART_FILE_ACCESS_PARTITIONS_RETENTION_COUNT_KEY = + "smart.file.access.partition.retention.count"; + public static final int SMART_FILE_ACCESS_PARTITIONS_RETENTION_COUNT_DEFAULT = 24; + + // StatesManager // RuleManager public static final String SMART_RULE_EXECUTORS_KEY = "smart.rule.executors"; diff --git a/smart-common/src/main/java/org/smartdata/model/request/FileAccessInfoSearchRequest.java b/smart-common/src/main/java/org/smartdata/model/request/FileAccessInfoSearchRequest.java index 74fb02cc231..171cfed1676 100644 --- a/smart-common/src/main/java/org/smartdata/model/request/FileAccessInfoSearchRequest.java +++ b/smart-common/src/main/java/org/smartdata/model/request/FileAccessInfoSearchRequest.java @@ -24,7 +24,6 @@ import org.smartdata.model.TimeInterval; import java.util.List; -import java.util.Set; @Data @Builder @@ -34,7 +33,6 @@ public class FileAccessInfoSearchRequest { private final List ids; private final String pathLike; private final TimeInterval lastAccessedTime; - private final Set accessCountTables; public static FileAccessInfoSearchRequest noFilters() { return builder().build(); diff --git a/smart-engine/pom.xml b/smart-engine/pom.xml index e6d298916e7..f1a1b9f8eb3 100644 --- a/smart-engine/pom.xml +++ b/smart-engine/pom.xml @@ -182,6 +182,16 @@ log4j-slf4j-impl test + + org.testcontainers + jdbc + test + + + org.testcontainers + postgresql + test + diff --git a/smart-engine/src/main/java/org/smartdata/server/engine/StatesManager.java b/smart-engine/src/main/java/org/smartdata/server/engine/StatesManager.java index dbb0e9e765b..2b8a8fd6dc9 100644 --- a/smart-engine/src/main/java/org/smartdata/server/engine/StatesManager.java +++ b/smart-engine/src/main/java/org/smartdata/server/engine/StatesManager.java @@ -25,9 +25,13 @@ import org.smartdata.conf.ReconfigurableRegistry; import org.smartdata.conf.ReconfigureException; import org.smartdata.conf.SmartConfKeys; -import org.smartdata.metastore.MetaStoreException; -import org.smartdata.metastore.dao.accesscount.AccessCountTableManager; -import org.smartdata.metastore.model.AccessCountTable; +import org.smartdata.metastore.accesscount.DbAccessEventAggregator; +import org.smartdata.metastore.accesscount.FileAccessManager; +import org.smartdata.metastore.accesscount.failover.AccessCountFailoverFactory; +import org.smartdata.metastore.partition.FileAccessPartitionManagerImpl; +import org.smartdata.metastore.partition.FileAccessPartitionService; +import org.smartdata.metastore.partition.cleanup.FileAccessPartitionRetentionPolicyExecutorFactory; +import org.smartdata.metastore.transaction.TransactionRunner; import org.smartdata.metrics.FileAccessEvent; import org.smartdata.metrics.FileAccessEventSource; import org.smartdata.metrics.impl.MetricsFactory; @@ -40,6 +44,8 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; +import static org.springframework.transaction.annotation.Isolation.SERIALIZABLE; + /** * Polls metrics and events from NameNode. */ @@ -47,12 +53,14 @@ public class StatesManager extends AbstractService implements Reconfigurable { private final ServerContext serverContext; private ScheduledExecutorService executorService; - private AccessCountTableManager accessCountTableManager; + @Getter + private FileAccessManager fileAccessManager; private AccessEventFetcher accessEventFetcher; private FileAccessEventSource fileAccessEventSource; @Getter private CachedFilesManager cachedFilesManager; private AbstractService statesUpdaterService; + private FileAccessPartitionService fileAccessPartitionService; private PathChecker pathChecker; private volatile boolean working = false; @@ -69,18 +77,39 @@ public StatesManager(ServerContext context) { @Override public void init() throws IOException { LOG.info("Initializing ..."); - this.executorService = Executors.newScheduledThreadPool(4); - this.accessCountTableManager = new AccessCountTableManager( - serverContext.getMetaStore(), executorService, serverContext.getConf()); + this.executorService = Executors.newScheduledThreadPool(5); + TransactionRunner transactionRunner = + new TransactionRunner(serverContext.getMetaStore().transactionManager()); + transactionRunner.setIsolationLevel(SERIALIZABLE); + this.fileAccessManager = new FileAccessManager( + transactionRunner, + serverContext.getMetaStore().accessCountEventDao(), + serverContext.getMetaStore().cacheFileDao()); this.fileAccessEventSource = MetricsFactory.createAccessEventSource(serverContext.getConf()); + AccessCountFailoverFactory accessCountFailoverFactory = + new AccessCountFailoverFactory(serverContext.getConf()); + DbAccessEventAggregator accessEventAggregator = new DbAccessEventAggregator( + serverContext.getMetaStore().fileInfoDao(), + fileAccessManager, + accessCountFailoverFactory.create()); this.accessEventFetcher = new AccessEventFetcher( serverContext.getConf(), - accessCountTableManager.getAccessEventAggregator(), + accessEventAggregator, executorService, fileAccessEventSource.getCollector()); this.pathChecker = new PathChecker(serverContext.getConf()); this.cachedFilesManager = new CachedFilesManager(serverContext.getMetaStore().cacheFileDao()); + FileAccessPartitionRetentionPolicyExecutorFactory + fileAccessPartitionRetentionPolicyExecutorFactory = + new FileAccessPartitionRetentionPolicyExecutorFactory( + serverContext.getMetaStore()); + this.fileAccessPartitionService = new FileAccessPartitionService( + executorService, + new FileAccessPartitionManagerImpl(serverContext.getMetaStore()), + fileAccessPartitionRetentionPolicyExecutorFactory.createPolicyExecutor( + serverContext.getConf()) + ); initStatesUpdaterService(); if (statesUpdaterService == null) { @@ -105,6 +134,7 @@ public boolean inSafeMode() { @Override public void start() throws IOException { LOG.info("Starting ..."); + fileAccessPartitionService.start(); accessEventFetcher.start(); if (statesUpdaterService != null) { statesUpdaterService.start(); @@ -118,6 +148,9 @@ public void stop() throws IOException { working = false; LOG.info("Stopping ..."); + if (fileAccessPartitionService != null) { + fileAccessPartitionService.stop(); + } if (accessEventFetcher != null) { accessEventFetcher.stop(); } @@ -134,10 +167,6 @@ public void stop() throws IOException { LOG.info("Stopped."); } - public List getTablesForLast(long timeInMills) throws MetaStoreException { - return accessCountTableManager.getTablesForLast(timeInMills); - } - public void reportFileAccessEvent(FileAccessEvent event) { String path = event.getPath(); path = path + (path.endsWith("/") ? "" : "/"); @@ -203,8 +232,4 @@ private synchronized void initStatesUpdaterService() { LOG.info("", t); } } - - public AccessCountTableManager getAccessCountTableManager() { - return accessCountTableManager; - } } diff --git a/smart-engine/src/main/java/org/smartdata/server/engine/data/AccessEventFetcher.java b/smart-engine/src/main/java/org/smartdata/server/engine/data/AccessEventFetcher.java index 28305fe6763..e50a54fa293 100644 --- a/smart-engine/src/main/java/org/smartdata/server/engine/data/AccessEventFetcher.java +++ b/smart-engine/src/main/java/org/smartdata/server/engine/data/AccessEventFetcher.java @@ -20,7 +20,7 @@ import org.apache.hadoop.conf.Configuration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.smartdata.metastore.dao.accesscount.AccessEventAggregator; +import org.smartdata.metastore.accesscount.AccessEventAggregator; import org.smartdata.metrics.FileAccessEvent; import org.smartdata.metrics.FileAccessEventCollector; @@ -55,10 +55,8 @@ public AccessEventFetcher( } public void start() { - long current = System.currentTimeMillis(); - long toWait = fetchInterval - (current % fetchInterval); this.scheduledFuture = scheduledExecutorService.scheduleAtFixedRate( - fetchTask, toWait, fetchInterval, TimeUnit.MILLISECONDS); + fetchTask, 0, fetchInterval, TimeUnit.MILLISECONDS); } public void stop() { @@ -71,8 +69,8 @@ private static class FetchTask implements Runnable { private final AccessEventAggregator accessEventAggregator; private final FileAccessEventCollector collector; - public FetchTask( - AccessEventAggregator accessEventAggregator, FileAccessEventCollector collector) { + public FetchTask(AccessEventAggregator accessEventAggregator, + FileAccessEventCollector collector) { this.accessEventAggregator = accessEventAggregator; this.collector = collector; } diff --git a/smart-engine/src/main/java/org/smartdata/server/engine/rule/RuleExecutor.java b/smart-engine/src/main/java/org/smartdata/server/engine/rule/RuleExecutor.java index ba2dc5251ba..1f98fde0db1 100644 --- a/smart-engine/src/main/java/org/smartdata/server/engine/rule/RuleExecutor.java +++ b/smart-engine/src/main/java/org/smartdata/server/engine/rule/RuleExecutor.java @@ -24,7 +24,6 @@ import org.smartdata.exception.QueueFullException; import org.smartdata.metastore.MetaStore; import org.smartdata.metastore.MetaStoreException; -import org.smartdata.metastore.model.AccessCountTable; import org.smartdata.model.CmdletDescriptor; import org.smartdata.model.RuleInfo; import org.smartdata.model.RuleState; @@ -43,7 +42,9 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; -/** Execute rule queries and return result. */ +/** + * Execute rule queries and return result. + */ public class RuleExecutor implements Runnable { private static final Logger LOG = LoggerFactory.getLogger(RuleExecutor.class.getName()); @@ -187,138 +188,38 @@ private void genVirtualAccessCountTableValue(List parameters, boolean to executionCtx.setProperty(var, count == null ? 0L : count); } - public String genVirtualAccessCountTableTopValueOnStoragePolicy(List parameters) { - genVirtualAccessCountTableValueOnStoragePolicy(parameters, true); - return null; - } - - public String genVirtualAccessCountTableBottomValueOnStoragePolicy(List parameters) { - genVirtualAccessCountTableValueOnStoragePolicy(parameters, false); - return null; - } - - private void genVirtualAccessCountTableValueOnStoragePolicy(List parameters, - boolean top) { - List paraList = (List) parameters.get(0); - String table = (String) parameters.get(1); - String var = (String) parameters.get(2); - Long num = (Long) paraList.get(1); - String storage = ((String) paraList.get(2)).toUpperCase(); - String sqlsub; - if (storage.equals("CACHE")) { - sqlsub = String.format("SELECT %s.fid, %s.count FROM %s LEFT JOIN cached_file ON " - + "(%s.fid = cached_file.fid)", table, table, table, table); - } else { - Integer id = null; - try { - id = metastore.getStoragePolicyID(storage); - } catch (Exception e) { - // Ignore - } - if (id == null) { - id = -1; // safe return - } - sqlsub = String.format("SELECT %s.fid, %s.count FROM %s LEFT JOIN file ON " - + "(%s.fid = file.fid) WHERE file.sid = %d", - table, table, table, table, id); - } - - String sql0 = String.format( - "SELECT %s(count) FROM ( SELECT * FROM (%s) AS %s ORDER BY count %sLIMIT %d ) AS %s;", - top ? "min" : "max", - sqlsub, - table + "_AL1_TMP", - top ? "DESC " : "", - num, - table + "_AL2_TMP"); - Long count = null; - try { - count = metastore.queryForLong(sql0); - } catch (MetaStoreException e) { - LOG.error(String.format("Get %s access count on storage [%s] from table '%s' error [%s].", - top ? "top" : "bottom", storage, table, sql0), e); - } - executionCtx.setProperty(var, count == null ? 0L : count); - } - public String genVirtualAccessCountTable(List parameters) { List paraList = (List) parameters.get(0); String newTable = (String) parameters.get(1); - Long interval = (Long) paraList.get(0); + long interval = paraList.isEmpty() ? 0L : (long) paraList.get(0); String countFilter = ""; - List tableNames = getAccessCountTablesDuringLast(interval); - return generateSQL(tableNames, newTable, countFilter, metastore); + long currentTimeMillis = System.currentTimeMillis(); + return generateSQL(newTable, countFilter, metastore, currentTimeMillis - interval, + currentTimeMillis); } @VisibleForTesting static String generateSQL( - List tableNames, String newTable, String countFilter, MetaStore adapter) { + String newTable, + String countFilter, + MetaStore adapter, + long startTime, + long endTime) { String sqlFinal, sqlCreate; - if (tableNames.size() <= 1) { - String tableName = tableNames.isEmpty() ? "blank_access_count_info" : tableNames.get(0); - sqlCreate = "CREATE TABLE " + newTable + "(fid INTEGER NOT NULL, count INTEGER NOT NULL);"; - try { - adapter.execute(sqlCreate); - } catch (MetaStoreException e) { - LOG.error("Cannot create table " + newTable, e); - } - sqlFinal = "INSERT INTO " + newTable + " SELECT * FROM " + tableName + ";"; - } else { - String sqlPrefix = "SELECT fid, SUM(count) AS count FROM (\n"; - String sqlUnion = "SELECT fid, count FROM " + tableNames.get(0) + " \n"; - for (int i = 1; i < tableNames.size(); i++) { - sqlUnion += "UNION ALL\n" + "SELECT fid, count FROM " + tableNames.get(i) + " \n"; - } - String sqlSufix = ") as tmp GROUP BY fid "; - String sqlCountFilter = - (countFilter == null || countFilter.isEmpty()) - ? "" - : "HAVING SUM(count) " + countFilter; - String sqlRe = sqlPrefix + sqlUnion + sqlSufix + sqlCountFilter; - sqlCreate = "CREATE TABLE " + newTable + "(fid INTEGER NOT NULL, count INTEGER NOT NULL);"; - try { - adapter.execute(sqlCreate); - } catch (MetaStoreException e) { - LOG.error("Cannot create table " + newTable, e); - } - sqlFinal = "INSERT INTO " + newTable + " SELECT * FROM (" + sqlRe + ") temp;"; - } - return sqlFinal; - } - - private List getAccessCountTablesDuringLast(long lastInterval) { - List tableNames = new ArrayList<>(); - if (ruleManager == null || ruleManager.getStatesManager() == null) { - return tableNames; - } - - List accTables = null; + sqlCreate = "CREATE TABLE " + newTable + "(fid INTEGER NOT NULL, count INTEGER NOT NULL);"; try { - accTables = ruleManager.getStatesManager().getTablesForLast(lastInterval); + adapter.execute(sqlCreate); } catch (MetaStoreException e) { - LOG.error("Rule " + executionCtx.getRuleId() + " get access info tables exception", e); - } - - if (LOG.isDebugEnabled()) { - LOG.debug("Rule " + executionCtx.getRuleId() + " got " + accTables.size() + " tables:"); - int idx = 1; - for (AccessCountTable t : accTables) { - LOG.debug( - idx + ". " + (t.isEphemeral() ? " [TABLE] " : " ") + t.getTableName() + " "); - } + LOG.error("Cannot create table " + newTable, e); } - - if (accTables == null || accTables.isEmpty()) { - return tableNames; - } - - for (AccessCountTable t : accTables) { - tableNames.add(t.getTableName()); - if (t.isEphemeral()) { - dynamicCleanups.push("DROP TABLE IF EXISTS " + t.getTableName() + ";"); - } - } - return tableNames; + String sqlCountFilter = + (countFilter == null || countFilter.isEmpty()) + ? "" + : " HAVING count(*) " + countFilter; + sqlFinal = "INSERT INTO " + newTable + " SELECT fid, count(*) AS count FROM file_access\n" + + "WHERE access_time >= " + startTime + " AND access_time <= " + endTime + + " GROUP BY fid" + sqlCountFilter + " ;"; + return sqlFinal; } @Override diff --git a/smart-engine/src/test/java/org/smartdata/server/engine/rule/TestRuleExecutor.java b/smart-engine/src/test/java/org/smartdata/server/engine/rule/TestRuleExecutor.java index af02c86987d..f88e7211aa6 100644 --- a/smart-engine/src/test/java/org/smartdata/server/engine/rule/TestRuleExecutor.java +++ b/smart-engine/src/test/java/org/smartdata/server/engine/rule/TestRuleExecutor.java @@ -17,16 +17,14 @@ */ package org.smartdata.server.engine.rule; -import org.junit.Assert; import org.junit.Before; import org.junit.Test; -import org.smartdata.metastore.SqliteTestDaoBase; +import org.smartdata.metastore.TestDaoBase; import org.smartdata.metastore.dao.MetaStoreHelper; -import java.util.ArrayList; -import java.util.List; +import static org.junit.Assert.assertTrue; -public class TestRuleExecutor extends SqliteTestDaoBase { +public class TestRuleExecutor extends TestDaoBase { private MetaStoreHelper metaStoreHelper; @Before @@ -38,31 +36,28 @@ public void initActionDao() { public void generateSQL() { String countFilter = ""; String newTable = "test"; - List tableNames = new ArrayList<>(); - tableNames.add("blank_access_count_info"); String sql; - /*sql = "CREATE TABLE actual as SELECT fid, SUM(count)" + - " as count FROM (SELECT * FROM blank_access_count_info " + - "UNION ALL SELECT * FROM blank_access_count_info " + - "UNION ALL SELECT * FROM blank_access_count_info) as tmp GROUP BY fid"; - metaStoreHelper.execute(sql); - metaStoreHelper.dropTable("actual");*/ - // Test single element - sql = RuleExecutor.generateSQL(tableNames, newTable, countFilter, metaStore); + long interval = 60000; + long currentTimeMillis = System.currentTimeMillis(); + sql = RuleExecutor.generateSQL(newTable, countFilter, metaStore, currentTimeMillis - interval, + currentTimeMillis); try { metaStoreHelper.execute(sql); + assertTrue(sql.contains("GROUP BY fid ;")); metaStoreHelper.dropTable(newTable); } catch (Exception e) { - Assert.assertTrue(false); + assertTrue(false); } - // Test multiple elements - tableNames.add("blank_access_count_info"); - sql = RuleExecutor.generateSQL(tableNames, newTable, countFilter, metaStore); + // Test with count filter + countFilter = "> 10"; + sql = RuleExecutor.generateSQL(newTable, countFilter, metaStore, currentTimeMillis - interval, + currentTimeMillis); try { metaStoreHelper.execute(sql); + assertTrue(sql.contains("GROUP BY fid HAVING count(*) > 10 ;")); metaStoreHelper.dropTable(newTable); } catch (Exception e) { - Assert.assertTrue(false); + assertTrue(false); } } } diff --git a/smart-hadoop-support/smart-hadoop/src/test/java/org/smartdata/hdfs/metric/fetcher/TestCachedListFetcher.java b/smart-hadoop-support/smart-hadoop/src/test/java/org/smartdata/hdfs/metric/fetcher/TestCachedListFetcher.java index 4d918d620e4..c25c86a6c3f 100644 --- a/smart-hadoop-support/smart-hadoop/src/test/java/org/smartdata/hdfs/metric/fetcher/TestCachedListFetcher.java +++ b/smart-hadoop-support/smart-hadoop/src/test/java/org/smartdata/hdfs/metric/fetcher/TestCachedListFetcher.java @@ -36,7 +36,7 @@ import org.smartdata.hdfs.action.CacheFileAction; import org.smartdata.hdfs.action.UncacheFileAction; import org.smartdata.hdfs.scheduler.CacheScheduler; -import org.smartdata.metastore.SqliteTestDaoBase; +import org.smartdata.metastore.TestDaoBase; import org.smartdata.model.CachedFileStatus; import org.smartdata.model.FileInfo; @@ -46,7 +46,7 @@ import java.util.Map; -public class TestCachedListFetcher extends SqliteTestDaoBase { +public class TestCachedListFetcher extends TestDaoBase { private long fid; diff --git a/smart-hadoop-support/smart-hadoop/src/test/java/org/smartdata/hdfs/metric/fetcher/TestInotifyFetcher.java b/smart-hadoop-support/smart-hadoop/src/test/java/org/smartdata/hdfs/metric/fetcher/TestInotifyFetcher.java index a39a0354954..b5cec71f3c4 100644 --- a/smart-hadoop-support/smart-hadoop/src/test/java/org/smartdata/hdfs/metric/fetcher/TestInotifyFetcher.java +++ b/smart-hadoop-support/smart-hadoop/src/test/java/org/smartdata/hdfs/metric/fetcher/TestInotifyFetcher.java @@ -38,7 +38,7 @@ import org.smartdata.hdfs.CompatibilityHelperLoader; import org.smartdata.hdfs.MiniClusterFactory; import org.smartdata.metastore.MetaStore; -import org.smartdata.metastore.SqliteTestDaoBase; +import org.smartdata.metastore.TestDaoBase; import java.io.IOException; import java.io.OutputStream; @@ -49,7 +49,7 @@ import java.util.concurrent.Callable; import java.util.concurrent.Executors; -public class TestInotifyFetcher extends SqliteTestDaoBase { +public class TestInotifyFetcher extends TestDaoBase { private static final int BLOCK_SIZE = 1024; private static class EventApplierForTest extends InotifyEventApplier { diff --git a/smart-integration/src/test/java/org/smartdata/integration/TestCmdletRestApi.java b/smart-integration/src/test/java/org/smartdata/integration/TestCmdletRestApi.java index 44c10b1b1be..6d60055facc 100644 --- a/smart-integration/src/test/java/org/smartdata/integration/TestCmdletRestApi.java +++ b/smart-integration/src/test/java/org/smartdata/integration/TestCmdletRestApi.java @@ -101,7 +101,7 @@ public void testDeleteCmdlet() { .execute(Response::andReturn), response -> response.getStatusCode() == HttpStatus.NOT_FOUND_404, Duration.ofMillis(100), - Duration.ofSeconds(1) + Duration.ofSeconds(5) ); } diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/DBType.java b/smart-metastore/src/main/java/org/smartdata/metastore/DBType.java index da80c1c19cf..096c8f56562 100644 --- a/smart-metastore/src/main/java/org/smartdata/metastore/DBType.java +++ b/smart-metastore/src/main/java/org/smartdata/metastore/DBType.java @@ -21,7 +21,5 @@ * Type of database. */ public enum DBType { - SQLITE, - MYSQL, POSTGRES } diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/MetaStore.java b/smart-metastore/src/main/java/org/smartdata/metastore/MetaStore.java index d414bd663e4..0bcd23b76a4 100644 --- a/smart-metastore/src/main/java/org/smartdata/metastore/MetaStore.java +++ b/smart-metastore/src/main/java/org/smartdata/metastore/MetaStore.java @@ -33,6 +33,8 @@ import org.smartdata.metastore.dao.CompressionFileDao; import org.smartdata.metastore.dao.DaoProvider; import org.smartdata.metastore.dao.ErasureCodingPolicyDao; +import org.smartdata.metastore.dao.FileAccessDao; +import org.smartdata.metastore.dao.FileAccessPartitionDao; import org.smartdata.metastore.dao.FileDiffDao; import org.smartdata.metastore.dao.FileInfoDao; import org.smartdata.metastore.dao.FileStateDao; @@ -45,11 +47,8 @@ import org.smartdata.metastore.dao.SystemInfoDao; import org.smartdata.metastore.dao.UserActivityDao; import org.smartdata.metastore.dao.WhitelistDao; -import org.smartdata.metastore.dao.accesscount.AccessCountEventDao; -import org.smartdata.metastore.dao.accesscount.AccessCountTableDao; import org.smartdata.metastore.db.DbSchemaManager; import org.smartdata.metastore.db.metadata.DbMetadataProvider; -import org.smartdata.metastore.model.AccessCountTable; import org.smartdata.metastore.model.AggregatedAccessCounts; import org.smartdata.metastore.transaction.TransactionRunner; import org.smartdata.metastore.utils.MetaStoreUtils; @@ -112,8 +111,8 @@ public class MetaStore implements CopyMetaService, private final CacheFileDao cacheFileDao; private final StorageDao storageDao; private final FileDiffDao fileDiffDao; - private final AccessCountTableDao accessCountTableDao; - private final AccessCountEventDao accessCountEventDao; + private final FileAccessDao fileAccessDao; + private final FileAccessPartitionDao fileAccessPartitionDao; private final MetaStoreHelper metaStoreHelper; private final ClusterConfigDao clusterConfigDao; private final GlobalConfigDao globalConfigDao; @@ -144,8 +143,6 @@ public MetaStore(DBPool pool, fileInfoDao = daoProvider.fileInfoDao(); cacheFileDao = daoProvider.cacheFileDao(); storageDao = daoProvider.storageDao(); - accessCountTableDao = daoProvider.accessCountDao(); - accessCountEventDao = daoProvider.accessCountEventDao(); fileDiffDao = daoProvider.fileDiffDao(); metaStoreHelper = new MetaStoreHelper(pool.getDataSource()); clusterConfigDao = daoProvider.clusterConfigDao(); @@ -160,6 +157,8 @@ public MetaStore(DBPool pool, ecDao = daoProvider.ecDao(); whitelistDao = daoProvider.whitelistDao(); userActivityDao = daoProvider.userActivityDao(); + fileAccessPartitionDao = daoProvider.fileAccessPartitionDao(); + fileAccessDao = daoProvider.fileAccessDao(); } public DbMetadataProvider dbMetadataProvider() { @@ -182,12 +181,8 @@ public RuleDao ruleDao() { return ruleDao; } - public AccessCountTableDao accessCountTableDao() { - return accessCountTableDao; - } - - public AccessCountEventDao accessCountEventDao() { - return accessCountEventDao; + public FileAccessDao accessCountEventDao() { + return fileAccessDao; } public CacheFileDao cacheFileDao() { @@ -198,6 +193,10 @@ public FileInfoDao fileInfoDao() { return fileInfoDao; } + public FileAccessPartitionDao fileAccessPartitionDao() { + return fileAccessPartitionDao; + } + public PlatformTransactionManager transactionManager() { return defaultTransactionRunner.getTransactionManager(); @@ -351,9 +350,9 @@ public Map getFileIDs(Collection paths) } /** - * @param srcFileId the fid of old file. + * @param srcFileId the fid of old file. * @param destFileId the fid of new file that will take over the access - * count of old file. + * count of old file. */ public void updateAccessCountTableFileIds(long srcFileId, long destFileId) throws MetaStoreException { @@ -362,11 +361,7 @@ public void updateAccessCountTableFileIds(long srcFileId, long destFileId) } try { - defaultTransactionRunner.inTransaction(() -> { - List accessCountTables = - accessCountTableDao.getAllSortedTables(); - accessCountEventDao.updateFileIds(accessCountTables, srcFileId, destFileId); - }); + fileAccessDao.updateFileIds(srcFileId, destFileId); } catch (Exception e) { throw new MetaStoreException(e); } diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/AccessEventAggregator.java b/smart-metastore/src/main/java/org/smartdata/metastore/accesscount/AccessEventAggregator.java similarity index 95% rename from smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/AccessEventAggregator.java rename to smart-metastore/src/main/java/org/smartdata/metastore/accesscount/AccessEventAggregator.java index 8212f7a691d..1d8445fedb8 100644 --- a/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/AccessEventAggregator.java +++ b/smart-metastore/src/main/java/org/smartdata/metastore/accesscount/AccessEventAggregator.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.smartdata.metastore.dao.accesscount; +package org.smartdata.metastore.accesscount; import org.smartdata.metrics.FileAccessEvent; diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/accesscount/DbAccessEventAggregator.java b/smart-metastore/src/main/java/org/smartdata/metastore/accesscount/DbAccessEventAggregator.java new file mode 100644 index 00000000000..15903da4a9d --- /dev/null +++ b/smart-metastore/src/main/java/org/smartdata/metastore/accesscount/DbAccessEventAggregator.java @@ -0,0 +1,84 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.smartdata.metastore.accesscount; + +import lombok.extern.slf4j.Slf4j; +import org.smartdata.metastore.accesscount.failover.AccessCountContext; +import org.smartdata.metastore.accesscount.failover.Failover; +import org.smartdata.metastore.dao.FileInfoDao; +import org.smartdata.metastore.model.AggregatedAccessCounts; +import org.smartdata.metrics.FileAccessEvent; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; + +@Slf4j +public class DbAccessEventAggregator implements AccessEventAggregator { + + private final FileInfoDao fileInfoDao; + private final FileAccessManager dbTableManager; + private final Failover accessCountFailover; + + public DbAccessEventAggregator(FileInfoDao fileInfoDao, + FileAccessManager dbTableManager, + Failover failover) { + this.fileInfoDao = fileInfoDao; + this.dbTableManager = dbTableManager; + this.accessCountFailover = failover; + } + + @Override + public void aggregate(List events) { + List fileAccessCounts = getAggregatedAccessCounts(events); + AccessCountContext accessCountContext = + new AccessCountContext(fileAccessCounts); + accessCountFailover.execute(ctx -> dbTableManager.save(accessCountContext.getAccessCounts()), + accessCountContext); + } + + private List getAggregatedAccessCounts(List events) { + List paths = + events.stream() + .map(FileAccessEvent::getPath) + .collect(Collectors.toList()); + final Map pathFids = getFileIdMap(paths); + return events.stream() + .map(e -> { + Long fileId = pathFids.get(e.getPath()); + if (fileId != null) { + return AggregatedAccessCounts.fromEvent(e).withFileId(fileId); + } else { + return null; + } + }) + .filter(Objects::nonNull) + .collect(Collectors.toList()); + } + + private Map getFileIdMap(List paths) { + try { + return fileInfoDao.getPathFids(paths); + } catch (Exception e) { + log.error("Error fetching file ids for paths {}", paths, e); + return Collections.emptyMap(); + } + } +} diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/accesscount/FileAccessManager.java b/smart-metastore/src/main/java/org/smartdata/metastore/accesscount/FileAccessManager.java new file mode 100644 index 00000000000..e22daf49b7d --- /dev/null +++ b/smart-metastore/src/main/java/org/smartdata/metastore/accesscount/FileAccessManager.java @@ -0,0 +1,117 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.smartdata.metastore.accesscount; + +import lombok.extern.slf4j.Slf4j; +import org.smartdata.metastore.MetaStoreException; +import org.smartdata.metastore.dao.CacheFileDao; +import org.smartdata.metastore.dao.FileAccessDao; +import org.smartdata.metastore.dao.Searchable; +import org.smartdata.metastore.model.AggregatedAccessCounts; +import org.smartdata.metastore.model.SearchResult; +import org.smartdata.metastore.queries.PageRequest; +import org.smartdata.metastore.queries.sort.FileAccessInfoSortField; +import org.smartdata.metastore.transaction.TransactionRunner; +import org.smartdata.model.FileAccessInfo; +import org.smartdata.model.request.FileAccessInfoSearchRequest; + +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.function.Function; +import java.util.stream.Collectors; + +@Slf4j +public class FileAccessManager implements + Searchable { + + private final TransactionRunner transactionRunner; + private final FileAccessDao fileAccessDao; + private final CacheFileDao cacheFileDao; + + public FileAccessManager( + TransactionRunner transactionRunner, + FileAccessDao fileAccessDao, + CacheFileDao cacheFileDao) { + this.fileAccessDao = fileAccessDao; + this.cacheFileDao = cacheFileDao; + this.transactionRunner = transactionRunner; + } + + @Override + public SearchResult search(FileAccessInfoSearchRequest searchRequest, + PageRequest pageRequest) { + return fileAccessDao.search(searchRequest, pageRequest); + } + + @Override + public List search(FileAccessInfoSearchRequest searchRequest) { + return fileAccessDao.search(searchRequest); + } + + public void save(Collection accessCounts) { + if (accessCounts.isEmpty()) { + return; + } + try { + transactionRunner.inTransaction(() -> { + insertFileAccesses(accessCounts); + updateCachedFilesInMetastore(getAggregatedAccessCounts(accessCounts)); + }); + } catch (MetaStoreException e) { + log.error("Failed to save access counts", e); + throw new RuntimeException(e); + } + } + + private void insertFileAccesses( + Collection accessCounts) throws MetaStoreException { + try { + fileAccessDao.insert(accessCounts); + log.debug("Inserted values {} to file access table", accessCounts); + } catch (Exception e) { + log.error("Error inserting file accesses {}", accessCounts, e); + throw new MetaStoreException(e); + } + } + + private void updateCachedFilesInMetastore(Collection accessCounts) + throws MetaStoreException { + try { + cacheFileDao.update(accessCounts); + } catch (Exception e) { + log.error("Error updating cached files {}", accessCounts, e); + throw new MetaStoreException(e); + } + } + + private Collection getAggregatedAccessCounts( + Collection accessCounts) { + Map aggregatedAccessCounts = + accessCounts.stream() + .collect(Collectors.toMap( + AggregatedAccessCounts::getFileId, + Function.identity(), + AggregatedAccessCounts::merge + )); + return aggregatedAccessCounts.values().stream() + .filter(Objects::nonNull) + .collect(Collectors.toList()); + } +} diff --git a/smart-metastore/src/test/java/org/smartdata/metastore/dao/accesscount/NoOpAccessCountTableHandler.java b/smart-metastore/src/main/java/org/smartdata/metastore/accesscount/failover/AccessCountContext.java similarity index 61% rename from smart-metastore/src/test/java/org/smartdata/metastore/dao/accesscount/NoOpAccessCountTableHandler.java rename to smart-metastore/src/main/java/org/smartdata/metastore/accesscount/failover/AccessCountContext.java index ce440f3532c..4a7520ee66d 100644 --- a/smart-metastore/src/test/java/org/smartdata/metastore/dao/accesscount/NoOpAccessCountTableHandler.java +++ b/smart-metastore/src/main/java/org/smartdata/metastore/accesscount/failover/AccessCountContext.java @@ -15,22 +15,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.smartdata.metastore.dao.accesscount; +package org.smartdata.metastore.accesscount.failover; -import org.smartdata.metastore.MetaStoreException; -import org.smartdata.metastore.model.AccessCountTable; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.RequiredArgsConstructor; +import org.smartdata.metastore.model.AggregatedAccessCounts; -import java.util.List; +import java.util.Collection; -public class NoOpAccessCountTableHandler implements AccessCountTableHandler { - @Override - public void dropTable(AccessCountTable accessCountTable) throws MetaStoreException { - - } - - @Override - public void aggregate(AccessCountTable destinationTable, List tablesToAggregate) - throws MetaStoreException { - - } +@Getter +@EqualsAndHashCode +@RequiredArgsConstructor +public class AccessCountContext { + private final Collection accessCounts; } diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/accesscount/failover/AccessCountFailoverFactory.java b/smart-metastore/src/main/java/org/smartdata/metastore/accesscount/failover/AccessCountFailoverFactory.java new file mode 100644 index 00000000000..e7197e7f32d --- /dev/null +++ b/smart-metastore/src/main/java/org/smartdata/metastore/accesscount/failover/AccessCountFailoverFactory.java @@ -0,0 +1,46 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.smartdata.metastore.accesscount.failover; + +import lombok.RequiredArgsConstructor; +import org.smartdata.conf.SmartConf; +import org.smartdata.metastore.accesscount.failover.impl.RetryAccessCountFailover; + +import static org.smartdata.conf.SmartConfKeys.SMART_ACCESS_COUNT_AGGREGATOR_FAILOVER_KEY; +import static org.smartdata.conf.SmartConfKeys.SMART_ACCESS_COUNT_AGGREGATOR_FAILOVER_MAX_RETRIES_DEFAULT; +import static org.smartdata.conf.SmartConfKeys.SMART_ACCESS_COUNT_AGGREGATOR_FAILOVER_MAX_RETRIES_KEY; + +@RequiredArgsConstructor +public class AccessCountFailoverFactory { + private final SmartConf conf; + + public Failover create() { + Failover.Strategy failoverStrategy = + conf.getEnum(SMART_ACCESS_COUNT_AGGREGATOR_FAILOVER_KEY, Failover.Strategy.FAIL); + switch (failoverStrategy) { + case FAIL: + return new Failover() { + }; + case SAVE_FAILED_WITH_RETRY: + default: + int maxRetries = conf.getInt(SMART_ACCESS_COUNT_AGGREGATOR_FAILOVER_MAX_RETRIES_KEY, + SMART_ACCESS_COUNT_AGGREGATOR_FAILOVER_MAX_RETRIES_DEFAULT); + return new RetryAccessCountFailover(maxRetries); + } + } +} diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/AccessCountTableDao.java b/smart-metastore/src/main/java/org/smartdata/metastore/accesscount/failover/Failover.java similarity index 67% rename from smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/AccessCountTableDao.java rename to smart-metastore/src/main/java/org/smartdata/metastore/accesscount/failover/Failover.java index 27188807ca6..4dd400b0a9e 100644 --- a/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/AccessCountTableDao.java +++ b/smart-metastore/src/main/java/org/smartdata/metastore/accesscount/failover/Failover.java @@ -15,20 +15,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.smartdata.metastore.dao.accesscount; +package org.smartdata.metastore.accesscount.failover; -import org.smartdata.metastore.model.AccessCountTable; +public interface Failover { -import java.util.List; + enum Strategy { + FAIL, + SAVE_FAILED_WITH_RETRY, + } -public interface AccessCountTableDao { - String TABLE_NAME = "access_count_table"; - - void insert(AccessCountTable accessCountTable); - - void delete(AccessCountTable table); - - List getAllSortedTables(); - - boolean tableExists(String name); + default void execute(Statement statement, T context) { + try { + statement.execute(context); + } catch (Exception e) { + throw new RuntimeException(e); + } + } } diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/accesscount/failover/Statement.java b/smart-metastore/src/main/java/org/smartdata/metastore/accesscount/failover/Statement.java new file mode 100644 index 00000000000..4bb842a801f --- /dev/null +++ b/smart-metastore/src/main/java/org/smartdata/metastore/accesscount/failover/Statement.java @@ -0,0 +1,22 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.smartdata.metastore.accesscount.failover; + +public interface Statement { + void execute(T context); +} diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/accesscount/failover/impl/RetryAccessCountFailover.java b/smart-metastore/src/main/java/org/smartdata/metastore/accesscount/failover/impl/RetryAccessCountFailover.java new file mode 100644 index 00000000000..a941930b7ab --- /dev/null +++ b/smart-metastore/src/main/java/org/smartdata/metastore/accesscount/failover/impl/RetryAccessCountFailover.java @@ -0,0 +1,60 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.smartdata.metastore.accesscount.failover.impl; + +import lombok.RequiredArgsConstructor; +import org.smartdata.metastore.accesscount.failover.AccessCountContext; +import org.smartdata.metastore.accesscount.failover.Failover; +import org.smartdata.metastore.accesscount.failover.Statement; +import org.smartdata.metastore.model.AggregatedAccessCounts; + +import java.util.ArrayList; +import java.util.List; + +@RequiredArgsConstructor +public class RetryAccessCountFailover implements Failover { + private final List failedAccessCounts = new ArrayList<>(); + private final int maxRetries; + private int retryCount = 0; + + @Override + public void execute(Statement statement, AccessCountContext context) { + try { + if (!failedAccessCounts.isEmpty()) { + List accessCounts = new ArrayList<>(context.getAccessCounts()); + accessCounts.addAll(failedAccessCounts); + context = new AccessCountContext(accessCounts); + } + statement.execute(context); + resetRetries(); + } catch (Exception e) { + retryCount++; + if (retryCount < maxRetries + 1) { + failedAccessCounts.addAll(context.getAccessCounts()); + } else { + resetRetries(); + throw new RuntimeException(e); + } + } + } + + private void resetRetries() { + retryCount = 0; + failedAccessCounts.clear(); + } +} diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/dao/CacheFileDao.java b/smart-metastore/src/main/java/org/smartdata/metastore/dao/CacheFileDao.java index 8569da60788..258fdda5537 100644 --- a/smart-metastore/src/main/java/org/smartdata/metastore/dao/CacheFileDao.java +++ b/smart-metastore/src/main/java/org/smartdata/metastore/dao/CacheFileDao.java @@ -41,7 +41,7 @@ void insert(long fid, String path, long fromTime, void insert(List cachedFileStatusList); - int update(Long fid, Long lastAccessTime, Integer numAccessed); + int update(Long fid, Long lastAccessTime, long numAccessed); void update(Collection events); diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/dao/DaoProvider.java b/smart-metastore/src/main/java/org/smartdata/metastore/dao/DaoProvider.java index 62debe833d5..d9e2ea04a1d 100644 --- a/smart-metastore/src/main/java/org/smartdata/metastore/dao/DaoProvider.java +++ b/smart-metastore/src/main/java/org/smartdata/metastore/dao/DaoProvider.java @@ -17,9 +17,6 @@ */ package org.smartdata.metastore.dao; -import org.smartdata.metastore.dao.accesscount.AccessCountEventDao; -import org.smartdata.metastore.dao.accesscount.AccessCountTableDao; - public interface DaoProvider { RuleDao ruleDao(); @@ -35,9 +32,7 @@ public interface DaoProvider { FileDiffDao fileDiffDao(); - AccessCountTableDao accessCountDao(); - - AccessCountEventDao accessCountEventDao(); + FileAccessDao fileAccessDao(); ClusterConfigDao clusterConfigDao(); @@ -64,4 +59,6 @@ public interface DaoProvider { StoragePolicyDao storagePolicyDao(); UserActivityDao userActivityDao(); + + FileAccessPartitionDao fileAccessPartitionDao(); } diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/dao/DaoProviderFactory.java b/smart-metastore/src/main/java/org/smartdata/metastore/dao/DaoProviderFactory.java index e8f8a085c73..63156300b2b 100644 --- a/smart-metastore/src/main/java/org/smartdata/metastore/dao/DaoProviderFactory.java +++ b/smart-metastore/src/main/java/org/smartdata/metastore/dao/DaoProviderFactory.java @@ -19,9 +19,7 @@ import org.smartdata.metastore.DBPool; import org.smartdata.metastore.DBType; -import org.smartdata.metastore.dao.impl.DefaultDaoProvider; import org.smartdata.metastore.dao.postgres.PostgresDaoProvider; -import org.smartdata.metastore.dao.sqlite.SqliteDaoProvider; import org.springframework.transaction.PlatformTransactionManager; public class DaoProviderFactory { @@ -29,11 +27,8 @@ public DaoProvider createDaoProvider( DBPool dbPool, PlatformTransactionManager transactionManager, DBType dbType) { switch (dbType) { case POSTGRES: - return new PostgresDaoProvider(dbPool, transactionManager); - case SQLITE: - return new SqliteDaoProvider(dbPool, transactionManager); default: - return new DefaultDaoProvider(dbPool, transactionManager); + return new PostgresDaoProvider(dbPool, transactionManager); } } } diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/AccessCountEventDao.java b/smart-metastore/src/main/java/org/smartdata/metastore/dao/FileAccessDao.java similarity index 54% rename from smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/AccessCountEventDao.java rename to smart-metastore/src/main/java/org/smartdata/metastore/dao/FileAccessDao.java index 98e20c55e65..0a3ae756701 100644 --- a/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/AccessCountEventDao.java +++ b/smart-metastore/src/main/java/org/smartdata/metastore/dao/FileAccessDao.java @@ -15,47 +15,26 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.smartdata.metastore.dao.accesscount; +package org.smartdata.metastore.dao; import org.smartdata.metastore.MetaStoreException; -import org.smartdata.metastore.dao.Searchable; -import org.smartdata.metastore.model.AccessCountTable; import org.smartdata.metastore.model.AggregatedAccessCounts; import org.smartdata.metastore.queries.sort.FileAccessInfoSortField; import org.smartdata.model.FileAccessInfo; import org.smartdata.model.request.FileAccessInfoSearchRequest; import java.util.Collection; -import java.util.List; -import java.util.Set; -import java.util.stream.Collectors; -public interface AccessCountEventDao extends +public interface FileAccessDao extends Searchable { + String TABLE_NAME = "file_access"; String FILE_ID_FIELD = "fid"; String ACCESS_COUNT_FIELD = "count"; - String LAST_ACCESSED_TIME_FIELD = "last_accessed_time"; + String ACCESS_TIME_FIELD = "access_time"; - void insert( - AccessCountTable table, - Collection aggregatedAccessCounts) throws MetaStoreException; + void insert(Collection aggregatedAccessCounts) throws MetaStoreException; - void validate(AccessCountTable table) throws MetaStoreException; + void updateFileIds(long fidSrc, long fidDest) throws MetaStoreException; - void updateFileIds(List accessCountTables, - long fidSrc, long fidDest) throws MetaStoreException; - - static String unionTablesQuery(List tables) { - return tables.stream() - .map(AccessCountTable::getTableName) - .collect(Collectors.joining( - " UNION ALL SELECT * FROM ", "SELECT * FROM ", "")); - } - - static String unionTablesQuery(Set tables) { - return tables.stream() - .collect(Collectors.joining( - " UNION ALL SELECT * FROM ", "SELECT * FROM ", "")); - } } diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/AccessCountTableHandler.java b/smart-metastore/src/main/java/org/smartdata/metastore/dao/FileAccessPartitionDao.java similarity index 70% rename from smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/AccessCountTableHandler.java rename to smart-metastore/src/main/java/org/smartdata/metastore/dao/FileAccessPartitionDao.java index 7465cb745e8..0c392f5629f 100644 --- a/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/AccessCountTableHandler.java +++ b/smart-metastore/src/main/java/org/smartdata/metastore/dao/FileAccessPartitionDao.java @@ -15,16 +15,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.smartdata.metastore.dao.accesscount; +package org.smartdata.metastore.dao; import org.smartdata.metastore.MetaStoreException; -import org.smartdata.metastore.model.AccessCountTable; +import org.smartdata.metastore.model.FileAccessPartition; +import java.time.LocalDateTime; import java.util.List; -public interface AccessCountTableHandler { - void dropTable(AccessCountTable accessCountTable) throws MetaStoreException; +public interface FileAccessPartitionDao { - void aggregate(AccessCountTable destinationTable, - List tablesToAggregate) throws MetaStoreException; + void create(LocalDateTime date) throws MetaStoreException; + + List getAll(); + + void remove(FileAccessPartition partition); } diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/AccessCountEventAggregatorFailover.java b/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/AccessCountEventAggregatorFailover.java deleted file mode 100644 index aa56403e0b2..00000000000 --- a/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/AccessCountEventAggregatorFailover.java +++ /dev/null @@ -1,77 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.smartdata.metastore.dao.accesscount; - -import org.smartdata.metastore.model.AggregatedAccessCounts; - -import java.util.Map; - -public interface AccessCountEventAggregatorFailover { - enum Strategy { - FAIL, - DROP_EVENTS, - SUBMIT_FAILED_EVENTS_LATER, - SUBMIT_NEW_FAILED_EVENTS_LATER - } - - /** - * Returns the map of {@link AggregatedAccessCounts} to be saved - * for subsequent submission attempts. - */ - Map handleError( - Map accessCounts, - Map previousUnmergedAccessCounts, - Exception error - ); - - static AccessCountEventAggregatorFailover fail() { - return (accessCounts, previousUnmergedAccessCounts, error) -> { - throw new RuntimeException(error); - }; - } - - static AccessCountEventAggregatorFailover dropEvents() { - return (accessCounts, previousUnmergedAccessCounts, error) -> - previousUnmergedAccessCounts; - } - - static AccessCountEventAggregatorFailover submitFailedEventsLater() { - return (accessCounts, previousUnmergedAccessCounts, error) -> - accessCounts; - } - - static AccessCountEventAggregatorFailover submitNewFailedEventsLater() { - return (accessCounts, previousUnmergedAccessCounts, error) -> { - accessCounts.keySet().removeAll(previousUnmergedAccessCounts.keySet()); - return accessCounts; - }; - } - - static AccessCountEventAggregatorFailover of(Strategy strategy) { - switch (strategy) { - case FAIL: - return fail(); - case SUBMIT_FAILED_EVENTS_LATER: - return submitFailedEventsLater(); - case SUBMIT_NEW_FAILED_EVENTS_LATER: - return submitNewFailedEventsLater(); - default: - return dropEvents(); - } - } -} diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/AccessCountTableDeque.java b/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/AccessCountTableDeque.java deleted file mode 100644 index 68ee435872a..00000000000 --- a/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/AccessCountTableDeque.java +++ /dev/null @@ -1,85 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.smartdata.metastore.dao.accesscount; - -import org.smartdata.metastore.model.AccessCountTable; - -import java.util.ArrayList; -import java.util.List; -import java.util.Optional; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentLinkedDeque; - -import static com.google.common.base.Preconditions.checkNotNull; - -/** - * Use deque to accelerate remove operation. - */ -public class AccessCountTableDeque extends ConcurrentLinkedDeque { - private final TableAddOpListener listener; - private final AccessCountTableEvictor tableEvictor; - - public AccessCountTableDeque(AccessCountTableEvictor tableEvictor) { - this(tableEvictor, TableAddOpListener.noOp()); - } - - public AccessCountTableDeque(AccessCountTableEvictor tableEvictor, TableAddOpListener listener) { - super(); - this.listener = checkNotNull( - listener, "listener should not be null"); - this.tableEvictor = checkNotNull( - tableEvictor, "tableEvictor should not be null"); - } - - public CompletableFuture addAndNotifyListener(AccessCountTable table) { - boolean containsOverlappingTable = Optional.ofNullable(peekLast()) - .map(AccessCountTable::getEndTime) - .filter(endTime -> table.getEndTime() <= endTime) - .isPresent(); - - if (containsOverlappingTable) { - throw new IllegalArgumentException("Overlapping access count table: " + table); - } - - add(table); - return notifyListener(table); - } - - public CompletableFuture notifyListener(AccessCountTable table) { - return listener.tableAdded(this, table) - .thenAccept(this::evictTablesIfHigherGrainedCreated); - } - - private void evictTablesIfHigherGrainedCreated(AccessCountTable higherGrainedTable) { - if (higherGrainedTable == null) { - return; - } - - tableEvictor.evictTables(this, higherGrainedTable.getEndTime()); - } - - public List getTables(Long start, Long end) { - List results = new ArrayList<>(); - for (AccessCountTable table : this) { - if (table.getStartTime() >= start && table.getEndTime() <= end) { - results.add(table); - } - } - return results; - } -} diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/AccessCountTableManager.java b/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/AccessCountTableManager.java deleted file mode 100644 index 0ee3272b4ea..00000000000 --- a/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/AccessCountTableManager.java +++ /dev/null @@ -1,371 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.smartdata.metastore.dao.accesscount; - -import lombok.Getter; -import org.apache.hadoop.conf.Configuration; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.smartdata.conf.SmartConfKeys; -import org.smartdata.metastore.MetaStore; -import org.smartdata.metastore.MetaStoreException; -import org.smartdata.metastore.dao.Searchable; -import org.smartdata.metastore.model.AccessCountTable; -import org.smartdata.metastore.model.AggregatedAccessCounts; -import org.smartdata.metastore.model.SearchResult; -import org.smartdata.metastore.queries.PageRequest; -import org.smartdata.metastore.queries.sort.FileAccessInfoSortField; -import org.smartdata.metastore.transaction.TransactionRunner; -import org.smartdata.metastore.utils.TimeGranularity; -import org.smartdata.model.FileAccessInfo; -import org.smartdata.model.TimeInterval; -import org.smartdata.model.request.FileAccessInfoSearchRequest; -import org.smartdata.utils.DateTimeUtils; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.Deque; -import java.util.List; -import java.util.Optional; -import java.util.Set; -import java.util.concurrent.ExecutorService; -import java.util.stream.Collectors; - -import static org.smartdata.conf.SmartConfKeys.SMART_ACCESS_COUNT_AGGREGATION_INTERVAL_MS; -import static org.smartdata.conf.SmartConfKeys.SMART_ACCESS_COUNT_AGGREGATION_INTERVAL_MS_DEFAULT; -import static org.smartdata.conf.SmartConfKeys.SMART_ACCESS_COUNT_AGGREGATOR_FAILOVER_KEY; -import static org.smartdata.metastore.utils.TimeGranularity.decreaseGranularity; -import static org.smartdata.metastore.utils.TimeGranularity.increaseGranularity; -import static org.springframework.transaction.annotation.Isolation.SERIALIZABLE; - -@Getter -public class AccessCountTableManager implements - Searchable { - private final InMemoryAccessEventAggregator accessEventAggregator; - private final InMemoryAccessCountTableManager inMemoryTableManager; - private final DbAccessCountTableManager dbTableManager; - private final TransactionRunner transactionRunner; - private final int defaultHotFilesLimit; - - public static final Logger LOG = - LoggerFactory.getLogger(AccessCountTableManager.class); - - public AccessCountTableManager(MetaStore metaStore, - ExecutorService service, - Configuration configuration) { - int aggregationIntervalMs = configuration.getInt( - SMART_ACCESS_COUNT_AGGREGATION_INTERVAL_MS, - SMART_ACCESS_COUNT_AGGREGATION_INTERVAL_MS_DEFAULT); - AccessCountEventAggregatorFailover.Strategy eventAggregatorFailoverStrategy = - configuration.getEnum(SMART_ACCESS_COUNT_AGGREGATOR_FAILOVER_KEY, - AccessCountEventAggregatorFailover.Strategy.SUBMIT_NEW_FAILED_EVENTS_LATER); - - this.accessEventAggregator = new InMemoryAccessEventAggregator( - metaStore.fileInfoDao(), - this::onAggregationWindowFinish, - AccessCountEventAggregatorFailover.of(eventAggregatorFailoverStrategy), - aggregationIntervalMs); - - this.transactionRunner = new TransactionRunner(metaStore.transactionManager()); - transactionRunner.setIsolationLevel(SERIALIZABLE); - - this.dbTableManager = new DbAccessCountTableManager( - metaStore.getDataSource(), - transactionRunner, - metaStore.accessCountTableDao(), - metaStore.accessCountEventDao(), - metaStore.cacheFileDao()); - this.inMemoryTableManager = new InMemoryAccessCountTableManager( - dbTableManager, service, configuration); - this.defaultHotFilesLimit = configuration.getInt( - SmartConfKeys.SMART_TOP_HOT_FILES_NUM_KEY, - SmartConfKeys.SMART_TOP_HOT_FILES_NUM_DEFAULT); - - recoverTables(); - } - - public List getTablesForLast(long intervalMillis) throws MetaStoreException { - Deque secondDeque = - inMemoryTableManager.getTablesOfGranularity(TimeGranularity.SECOND); - if (secondDeque.isEmpty()) { - return Collections.emptyList(); - } - - long endTime = secondDeque.getLast().getEndTime(); - return getAccessCountTables(endTime - intervalMillis, endTime); - } - - public void createTable(AccessCountTable table) throws MetaStoreException { - try { - transactionRunner.inTransaction(() -> { - dbTableManager.createTable(table); - inMemoryTableManager.addTable(table); - }); - } catch (MetaStoreException exception) { - LOG.error("Error creating access count table {}", table, exception); - throw exception; - } - } - - private void onAggregationWindowFinish( - long windowStart, - long windowEnd, - Collection aggregatedAccessCounts) { - if (aggregatedAccessCounts.isEmpty()) { - return; - } - - AccessCountTable table = new AccessCountTable(windowStart, windowEnd); - try { - transactionRunner.inTransaction(() -> { - createTable(table); - dbTableManager.handleAggregatedEvents(table, aggregatedAccessCounts); - }); - } catch (MetaStoreException exception) { - LOG.error("Error creating access count table {}", table, exception); - throw new RuntimeException(exception); - } - } - - private void recoverTables() { - try { - transactionRunner.inTransaction(() -> { - List tables = dbTableManager.getTables(); - inMemoryTableManager.recoverTables(tables); - }); - } catch (MetaStoreException exception) { - LOG.error("Error recovering existing access count tables", exception); - } - } - - /** - * Getting access count tables by interval: - * 1. found base granularity by searching interval, if tables are not exist, increase granularity - * 2. get access count tables by base granularity with startTime less or equals searching endTime - * 3. Filter tables by starTime less or equals searching endTime. - * 4. Filter tables by endTime > searching startTime and if parentTable != null, - * also filter by endTime <= parentTable.endTime. It is needed because we are getting - * all tables of some granularity. - * 5. For all found tables from the list we should add corresponding tables to the result - * according to their intervals and search granularity. For non-second granularity we get child - * tables recursively. If we have case when one of the searching border (startTime or endTime) - * bigger (less) than table startTime (endTime), we have to create partial table, which is based - * on current table (in some cases we create partial table base on parentTable, - * when prentTable != null and there are no tables of small granularity between - * parentTable.startTime and table.startTime or between searching startTime and - * parentTable.endTime). Add corresponding tables to the result and update searching startTime and - * 6. Decrease searching granularity and update it, check that searching - * startTime = endTime, if no, repeat from p.2 - * - * @param startTime start time of the search interval - * @param endTime end time of the search interval - * @return List of access count tables - * @throws MetaStoreException error - */ - private List getAccessCountTables(long startTime, - long endTime) throws MetaStoreException { - final TimeGranularity startGranularity = TimeGranularity.of(endTime - startTime); - TimeGranularity searchIntervalGranularity = startGranularity; - Collection tables = inMemoryTableManager.getTablesOfGranularity( - searchIntervalGranularity).stream() - .filter(t -> t.getStartTime() <= endTime) - .collect(Collectors.toList()); - boolean foundTables = !tables.isEmpty(); - while (!foundTables) { - searchIntervalGranularity = increaseGranularity(searchIntervalGranularity); - if (searchIntervalGranularity == null) { - searchIntervalGranularity = startGranularity; - foundTables = true; - } else { - tables = inMemoryTableManager.getTablesOfGranularity( - searchIntervalGranularity); - Long accessCountTableStartTime = tables.stream() - .findFirst() - .map(AccessCountTable::getStartTime) - .orElse(Long.MAX_VALUE); - foundTables = accessCountTableStartTime <= startTime; - } - } - return getAccessCountTables(startTime, endTime, null, searchIntervalGranularity); - } - - private List getAccessCountTables(long startTime, - long endTime, - AccessCountTable parentTable, - TimeGranularity baseGranularity) - throws MetaStoreException { - final TimeGranularity startGranularity = TimeGranularity.of(endTime - startTime); - final List result = new ArrayList<>(); - TimeGranularity searchIntervalGranularity; - if (baseGranularity != null) { - searchIntervalGranularity = baseGranularity; - } else { - searchIntervalGranularity = startGranularity; - } - do { - long finalStartTime = startTime; - Collection tables = inMemoryTableManager.getTablesOfGranularity( - searchIntervalGranularity).stream() - .filter(t -> t.getStartTime() <= endTime) - .filter(t -> { - if (parentTable != null) { - //include child tables which has endTime after startTime - // and includes into parent table - return t.getEndTime() > finalStartTime && t.getEndTime() <= parentTable.getEndTime(); - } - return t.getEndTime() > finalStartTime; - }) - .collect(Collectors.toList()); - int n = 0; - for (AccessCountTable table : tables) { - n++; - //skip tables if it starts after specified startTime - if (table.getStartTime() > endTime) { - continue; - } - if (table.getStartTime() <= startTime) { - if (table.getStartTime() == startTime) { - if (table.getEndTime() <= endTime) { - result.add(table); - startTime = table.getEndTime(); - } else { - //table.endTime > endTime - result.addAll(getChildOrCreatePartialTables(searchIntervalGranularity, - table, startTime, endTime)); - startTime = endTime; - } - } else { - if (table.getEndTime() <= endTime) { - result.addAll(getChildOrCreatePartialTables(searchIntervalGranularity, - table, startTime, table.getEndTime())); - startTime = table.getEndTime(); - } else { - result.addAll(getChildOrCreatePartialTables(searchIntervalGranularity, - table, startTime, endTime)); - startTime = endTime; - } - } - continue; - } - //table.startTime > startTime - if (table.getEndTime() <= endTime) { - //if table is child and the first, we have to create partial table for this interval - if (parentTable != null && n <= 1) { - //create partial table based on parent table from startTime to table.startTime - addPartialTable(parentTable, startTime, table.getStartTime()) - .ifPresent(result::add); - } - result.add(table); - startTime = table.getEndTime(); - } else { - addPartialTable(table, table.getStartTime(), endTime) - .ifPresent(result::add); - startTime = endTime; - } - } - searchIntervalGranularity = decreaseGranularity(searchIntervalGranularity); - if (searchIntervalGranularity == null && startTime != endTime) { - if (parentTable != null) { - //create partial table based on parent table from startTime to endTime - addPartialTable(parentTable, startTime, endTime) - .ifPresent(result::add); - } - startTime = endTime; - } - } while (startTime != endTime); - return result; - } - - private List getChildOrCreatePartialTables( - TimeGranularity searchIntervalGranularity, - AccessCountTable table, - long startTime, - long endTime) - throws MetaStoreException { - if (searchIntervalGranularity != TimeGranularity.SECOND) { - return getAccessCountTables(startTime, endTime, table, null); - } else { - return addPartialTable(table, startTime, endTime) - .map(Collections::singletonList) - .orElse(Collections.emptyList()); - } - } - - private Optional addPartialTable(AccessCountTable sourceTable, - long startTime, - long endTime) throws MetaStoreException { - // The access count table interval intersects with the search interval - if (!inMemoryTableManager.tableExists(startTime, endTime)) { - // Create ephemeral table with file access events whose last access time - // is greater than or equal to the startTime. We also assume that file accesses - // occurred evenly over time, so we can simply divide the number of file accesses - // during the table interval by the ratio of the search interval and the table interval - AccessCountTable partialTable = - new AccessCountTable(startTime, endTime, true); - dbTableManager.createPartialTable(partialTable, sourceTable); - return Optional.of(partialTable); - } else { - return Optional.empty(); - } - } - - @Override - public SearchResult search(FileAccessInfoSearchRequest searchRequest, - PageRequest pageRequest) { - try { - return transactionRunner.inTransaction(() -> { - Set tables = getAccessCountTables(searchRequest.getLastAccessedTime()); - if (tables.isEmpty()) { - return SearchResult.of(Collections.emptyList(), 0); - } - return dbTableManager.getFileAccessInfoList(searchRequest.withAccessCountTables(tables), - pageRequest); - }); - } catch (MetaStoreException e) { - LOG.error("Failed to get file access count information with pagination", e); - throw new RuntimeException(e); - } - } - - @Override - public List search(FileAccessInfoSearchRequest searchRequest) { - try { - return transactionRunner.inTransaction(() -> { - Set tables = getAccessCountTables(searchRequest.getLastAccessedTime()); - if (tables.isEmpty()) { - return Collections.emptyList(); - } - return dbTableManager.getFileAccessInfoList(searchRequest.withAccessCountTables(tables)); - }); - } catch (MetaStoreException e) { - LOG.error("Failed to get file access count information", e); - throw new RuntimeException(e); - } - } - - private Set getAccessCountTables(TimeInterval timeInterval) throws MetaStoreException { - long startTime = Optional.ofNullable(DateTimeUtils.intervalStartToEpoch(timeInterval)) - .orElse(0L); - long endTime = Optional.ofNullable(DateTimeUtils.intervalEndToEpoch(timeInterval)) - .orElse(System.currentTimeMillis()); - return getAccessCountTables(startTime, endTime).stream() - .map(AccessCountTable::getTableName) - .collect(Collectors.toSet()); - } -} diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/AggregatingTableAddOpListener.java b/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/AggregatingTableAddOpListener.java deleted file mode 100644 index 7d41f867618..00000000000 --- a/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/AggregatingTableAddOpListener.java +++ /dev/null @@ -1,100 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.smartdata.metastore.dao.accesscount; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.smartdata.metastore.model.AccessCountTable; - -import java.util.List; -import java.util.Set; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ExecutorService; - -public class AggregatingTableAddOpListener implements TableAddOpListener { - static final Logger LOG = LoggerFactory.getLogger(AggregatingTableAddOpListener.class); - - private final AccessCountTableDeque coarseGrainedTableDeque; - private final AccessCountTableHandler tableAggregator; - private final ExecutorService executorService; - private final Set tablesUnderConstruction; - private final long millisPerGranularity; - - AggregatingTableAddOpListener( - AccessCountTableDeque deque, - AccessCountTableHandler tableAggregator, - ExecutorService executorService, - long millisPerGranularity) { - this.coarseGrainedTableDeque = deque; - this.tableAggregator = tableAggregator; - this.executorService = executorService; - this.millisPerGranularity = millisPerGranularity; - tablesUnderConstruction = ConcurrentHashMap.newKeySet(); - } - - @Override - public CompletableFuture tableAdded( - AccessCountTableDeque fineGrainedTableDeque, AccessCountTable table) { - final AccessCountTable lastCoarseGrainedTable = lastCoarseGrainedTableFor(table.getEndTime()); - - // Todo: optimize contains - if (coarseGrainedTableDeque.contains(lastCoarseGrainedTable)) { - return CompletableFuture.completedFuture(lastCoarseGrainedTable); - } - - final List tablesToAggregate = - fineGrainedTableDeque.getTables( - lastCoarseGrainedTable.getStartTime(), lastCoarseGrainedTable.getEndTime()); - - if (!tablesToAggregate.isEmpty() - && !tablesUnderConstruction.contains(lastCoarseGrainedTable)) { - tablesUnderConstruction.add(lastCoarseGrainedTable); - return aggregateTableAsync(lastCoarseGrainedTable, tablesToAggregate); - } - - return TableAddOpListener.EMPTY_RESULT; - } - - private CompletableFuture aggregateTableAsync( - AccessCountTable lastCoarseGrainedTable, List tablesToAggregate) { - return CompletableFuture.supplyAsync( - () -> aggregateTable(lastCoarseGrainedTable, tablesToAggregate), executorService); - } - - private AccessCountTable aggregateTable( - AccessCountTable lastCoarseGrainedTable, List tablesToAggregate) { - try { - tableAggregator.aggregate(lastCoarseGrainedTable, tablesToAggregate); - coarseGrainedTableDeque.addAndNotifyListener(lastCoarseGrainedTable); - } catch (Exception e) { - LOG.error( - "Add AccessCount Table {} error", - lastCoarseGrainedTable.getTableName(), e); - } - tablesUnderConstruction.remove(lastCoarseGrainedTable); - return lastCoarseGrainedTable; - } - - protected AccessCountTable lastCoarseGrainedTableFor(long endTime) { - long lastEnd = endTime - (endTime % millisPerGranularity); - long lastStart = lastEnd - millisPerGranularity; - return new AccessCountTable(lastStart, lastEnd); - } - // Todo: WeekTableListener, MonthTableListener, YearTableListener -} diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/CountTableEvictor.java b/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/CountTableEvictor.java deleted file mode 100644 index 77eb2c6f18e..00000000000 --- a/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/CountTableEvictor.java +++ /dev/null @@ -1,49 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.smartdata.metastore.dao.accesscount; - -import org.smartdata.metastore.model.AccessCountTable; - -import java.util.Iterator; - -public class CountTableEvictor extends AccessCountTableEvictor { - private final int maxCount; - - public CountTableEvictor(AccessCountTableHandler tableDeleter, int count) { - super(tableDeleter); - this.maxCount = count; - } - - @Override - public void evictTables(AccessCountTableDeque tables, long lastAggregatedIntervalEndTimestamp) { - int elementsToRemove = tables.size() - maxCount; - - for (Iterator iterator = tables.iterator(); - iterator.hasNext() && elementsToRemove-- > 0; ) { - AccessCountTable table = iterator.next(); - - if (table.getEndTime() > lastAggregatedIntervalEndTimestamp) { - // table belongs to not yet aggregated higher granularity interval - return; - } - - iterator.remove(); - dropTable(table); - } - } -} diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/DbAccessCountTableManager.java b/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/DbAccessCountTableManager.java deleted file mode 100644 index 904f2949f7c..00000000000 --- a/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/DbAccessCountTableManager.java +++ /dev/null @@ -1,209 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.smartdata.metastore.dao.accesscount; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.smartdata.metastore.MetaStore; -import org.smartdata.metastore.MetaStoreException; -import org.smartdata.metastore.dao.AbstractDao; -import org.smartdata.metastore.dao.CacheFileDao; -import org.smartdata.metastore.model.AccessCountTable; -import org.smartdata.metastore.model.AggregatedAccessCounts; -import org.smartdata.metastore.model.SearchResult; -import org.smartdata.metastore.queries.PageRequest; -import org.smartdata.metastore.queries.sort.FileAccessInfoSortField; -import org.smartdata.metastore.transaction.TransactionRunner; -import org.smartdata.model.FileAccessInfo; -import org.smartdata.model.request.FileAccessInfoSearchRequest; - -import javax.sql.DataSource; - -import java.util.Collection; -import java.util.List; -import java.util.stream.Collectors; - -public class DbAccessCountTableManager - extends AbstractDao implements AccessCountTableHandler { - static final Logger LOG = LoggerFactory.getLogger(DbAccessCountTableManager.class); - - private final TransactionRunner transactionRunner; - private final AccessCountTableDao accessCountTableDao; - private final AccessCountEventDao accessCountEventDao; - private final CacheFileDao cacheFileDao; - - public DbAccessCountTableManager(MetaStore metastore) { - this( - metastore.getDataSource(), - new TransactionRunner(metastore.transactionManager()), - metastore.accessCountTableDao(), - metastore.accessCountEventDao(), - metastore.cacheFileDao() - ); - } - - public DbAccessCountTableManager( - DataSource dataSource, - TransactionRunner transactionRunner, - AccessCountTableDao accessCountTableDao, - AccessCountEventDao accessCountEventDao, - CacheFileDao cacheFileDao) { - super(dataSource, AccessCountTableDao.TABLE_NAME); - this.accessCountTableDao = accessCountTableDao; - this.accessCountEventDao = accessCountEventDao; - this.cacheFileDao = cacheFileDao; - this.transactionRunner = transactionRunner; - } - - @Override - public void aggregate( - AccessCountTable destinationTable, - List tablesToAggregate) throws MetaStoreException { - if (tablesToAggregate.isEmpty()) { - return; - } - - transactionRunner.inTransaction(() -> { - createTable(destinationTable); - aggregateTablesInternal(destinationTable, tablesToAggregate); - }); - } - - public void createTable(AccessCountTable table) throws MetaStoreException { - transactionRunner.inTransaction(() -> { - createAccessCountTable(table); - accessCountTableDao.insert(table); - }); - } - - public void createPartialTable(AccessCountTable dest, AccessCountTable source) - throws MetaStoreException { - transactionRunner.inTransaction(() -> { - createAccessCountTable(dest); - fillPartialTable(dest, source); - }); - } - - public SearchResult getFileAccessInfoList( - FileAccessInfoSearchRequest searchRequest, - PageRequest pageRequest) { - return accessCountEventDao.search(searchRequest, pageRequest); - } - - public List getFileAccessInfoList(FileAccessInfoSearchRequest searchRequest) { - return accessCountEventDao.search(searchRequest); - } - - @Override - public void dropTable(AccessCountTable accessCountTable) throws MetaStoreException { - transactionRunner.inTransaction(() -> { - dropDbTable(accessCountTable); - accessCountTableDao.delete(accessCountTable); - }); - } - - public void handleAggregatedEvents( - AccessCountTable table, - Collection accessCounts) throws MetaStoreException { - if (accessCounts.isEmpty()) { - return; - } - - transactionRunner.inTransaction(() -> { - insertAccessCountsToMetastore(table, accessCounts); - updateCachedFilesInMetastore(accessCounts); - }); - } - - public List getTables() { - return accessCountTableDao.getAllSortedTables() - .stream() - .filter(this::isValid) - .collect(Collectors.toList()); - } - - private boolean isValid(AccessCountTable table) { - try { - accessCountEventDao.validate(table); - return true; - } catch (MetaStoreException exception) { - LOG.error("Can't recover table {}, dropping it", table, exception); - dropDbTable(table); - return false; - } - } - - private void fillPartialTable(AccessCountTable dest, AccessCountTable source) { - String statement = - "INSERT INTO " + dest.getTableName() - + " SELECT src.fid, ROUND(src.count * " + dest.intervalRatio(source) - + "), src.last_accessed_time" - + " FROM " + source.getTableName() + " as src" - + " WHERE src.last_accessed_time >= " + source.getStartTime(); - - jdbcTemplate.execute(statement); - } - - private void aggregateTablesInternal(AccessCountTable destination, - List sources) { - String query = "INSERT INTO " + destination.getTableName() - + " SELECT aggregated_events.fid, " - + "aggregated_events.count, " - + "aggregated_events.last_accessed_time " - + "FROM (" - + "SELECT fid, SUM(count) AS count, MAX(last_accessed_time) as last_accessed_time " - + "FROM (" + AccessCountEventDao.unionTablesQuery(sources) + ") as union_events " - + "GROUP BY fid) AS aggregated_events " - + "JOIN file ON file.fid = aggregated_events.fid;"; - - LOG.debug("Executing access count tables aggregation: {}", query); - jdbcTemplate.execute(query); - } - - private void createAccessCountTable(AccessCountTable table) { - String createStatement = "CREATE TABLE IF NOT EXISTS " + table.getTableName() + "(" - + "fid BIGINT NOT NULL, " - + "count INTEGER NOT NULL, " - + "last_accessed_time BIGINT NOT NULL)"; - - jdbcTemplate.execute(createStatement); - } - - private void dropDbTable(AccessCountTable accessCountTable) { - String sql = "DROP TABLE IF EXISTS " + accessCountTable.getTableName(); - jdbcTemplate.execute(sql); - } - - private void insertAccessCountsToMetastore( - AccessCountTable table, Collection accessCounts) { - try { - accessCountEventDao.insert(table, accessCounts); - LOG.debug("Inserted values {} to access count table {}", accessCounts, table); - } catch (Exception e) { - LOG.error("Error inserting access counts {} to table {}", accessCounts, table, e); - } - } - - private void updateCachedFilesInMetastore(Collection accessCounts) { - try { - cacheFileDao.update(accessCounts); - } catch (Exception e) { - LOG.error("Error updating cached files {}", accessCounts, e); - } - } -} diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/InMemoryAccessCountTableManager.java b/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/InMemoryAccessCountTableManager.java deleted file mode 100644 index 123703e8137..00000000000 --- a/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/InMemoryAccessCountTableManager.java +++ /dev/null @@ -1,195 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.smartdata.metastore.dao.accesscount; - -import org.apache.hadoop.conf.Configuration; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.smartdata.metastore.model.AccessCountTable; -import org.smartdata.metastore.utils.TimeGranularity; - -import java.util.ArrayDeque; -import java.util.Deque; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.concurrent.ExecutorService; - -import static org.smartdata.conf.SmartConfKeys.SMART_ACCESS_COUNT_AGGREGATION_INTERVAL_MS; -import static org.smartdata.conf.SmartConfKeys.SMART_ACCESS_COUNT_AGGREGATION_INTERVAL_MS_DEFAULT; -import static org.smartdata.conf.SmartConfKeys.SMART_NUM_DAY_TABLES_TO_KEEP_DEFAULT; -import static org.smartdata.conf.SmartConfKeys.SMART_NUM_DAY_TABLES_TO_KEEP_KEY; -import static org.smartdata.conf.SmartConfKeys.SMART_NUM_HOUR_TABLES_TO_KEEP_DEFAULT; -import static org.smartdata.conf.SmartConfKeys.SMART_NUM_HOUR_TABLES_TO_KEEP_KEY; -import static org.smartdata.conf.SmartConfKeys.SMART_NUM_HOUR_TABLES_TO_KEEP_MIN; -import static org.smartdata.conf.SmartConfKeys.SMART_NUM_MINUTE_TABLES_TO_KEEP_DEFAULT; -import static org.smartdata.conf.SmartConfKeys.SMART_NUM_MINUTE_TABLES_TO_KEEP_KEY; -import static org.smartdata.conf.SmartConfKeys.SMART_NUM_MINUTE_TABLES_TO_KEEP_MIN; -import static org.smartdata.conf.SmartConfKeys.SMART_NUM_SECOND_TABLES_TO_KEEP_DEFAULT; -import static org.smartdata.conf.SmartConfKeys.SMART_NUM_SECOND_TABLES_TO_KEEP_KEY; -import static org.smartdata.metastore.utils.Constants.ONE_MINUTE_IN_MILLIS; - -public class InMemoryAccessCountTableManager { - private final Map tableDeques; - private final Configuration configuration; - private AccessCountTableDeque secondTableDeque; - - public static final Logger LOG = - LoggerFactory.getLogger(InMemoryAccessCountTableManager.class); - - public InMemoryAccessCountTableManager( - AccessCountTableHandler tableHandler, - ExecutorService executorService, - Configuration configuration) { - this.configuration = configuration; - this.tableDeques = new HashMap<>(); - - initTables(tableHandler, executorService); - } - - public void recoverTables(List tables) { - if (tables.isEmpty()) { - LOG.info("No existing access count tables to recover."); - return; - } - - LOG.info("Loading existing access count tables: {}", tables); - for (AccessCountTable table : tables) { - if (tableDeques.containsKey(table.getGranularity())) { - tableDeques.get(table.getGranularity()).add(table); - } - } - - // aggregate old tables if needed - AccessCountTable lastOldTable = tables.get(tables.size() - 1); - Optional.ofNullable(tableDeques.get(lastOldTable.getGranularity())) - .ifPresent(deque -> deque.notifyListener(lastOldTable)); - - LOG.info("Existing access count tables were successfully loaded"); - } - - public void addTable(AccessCountTable accessCountTable) { - if (LOG.isDebugEnabled()) { - LOG.debug(accessCountTable.toString()); - } - secondTableDeque.addAndNotifyListener(accessCountTable); - } - - public Deque getTablesOfGranularity(TimeGranularity timeGranularity) { - return Optional.>ofNullable( - tableDeques.get(timeGranularity)) - .orElseGet(ArrayDeque::new); - } - - public boolean tableExists(long start, long end) { - TimeGranularity granularity = TimeGranularity.of(end - start); - AccessCountTable fakeTable = new AccessCountTable(start, end); - return tableDeques.containsKey(granularity) - && tableDeques.get(granularity).contains(fakeTable); - } - - private void initTables( - AccessCountTableHandler tableHandler, - ExecutorService executorService) { - - TableAddOpListener dayTableListener = - createPerDayTableListener(tableHandler, executorService); - - TableAddOpListener hourTableListener = - createPerHourTableListener(tableHandler, executorService, dayTableListener); - - TableAddOpListener minuteTableListener = - createPerMinuteTableListener(tableHandler, executorService, hourTableListener); - - createSecondTableDeque(tableHandler, minuteTableListener); - } - - private TableAddOpListener createPerDayTableListener( - AccessCountTableHandler tableHandler, ExecutorService executorService) { - int perDayAccessTablesCount = configuration.getInt( - SMART_NUM_DAY_TABLES_TO_KEEP_KEY, - SMART_NUM_DAY_TABLES_TO_KEEP_DEFAULT); - AccessCountTableDeque dayTableDeque = new AccessCountTableDeque( - new CountTableEvictor(tableHandler, perDayAccessTablesCount)); - - tableDeques.put(TimeGranularity.DAY, dayTableDeque); - return TableAddOpListener.perDay(dayTableDeque, tableHandler, executorService); - } - - private TableAddOpListener createPerHourTableListener( - AccessCountTableHandler tableHandler, - ExecutorService executorService, - TableAddOpListener dayTableListener) { - int perHourAccessTablesCount = getAccessTablesCount( - SMART_NUM_HOUR_TABLES_TO_KEEP_KEY, - SMART_NUM_HOUR_TABLES_TO_KEEP_DEFAULT, - SMART_NUM_HOUR_TABLES_TO_KEEP_MIN); - AccessCountTableDeque hourTableDeque = new AccessCountTableDeque( - new CountTableEvictor(tableHandler, perHourAccessTablesCount), dayTableListener); - - tableDeques.put(TimeGranularity.HOUR, hourTableDeque); - return TableAddOpListener.perHour(hourTableDeque, tableHandler, executorService); - } - - private TableAddOpListener createPerMinuteTableListener( - AccessCountTableHandler tableHandler, - ExecutorService executorService, - TableAddOpListener hourTableListener) { - int perMinuteAccessTablesCount = getAccessTablesCount( - SMART_NUM_MINUTE_TABLES_TO_KEEP_KEY, - SMART_NUM_MINUTE_TABLES_TO_KEEP_DEFAULT, - SMART_NUM_MINUTE_TABLES_TO_KEEP_MIN); - AccessCountTableDeque minuteTableDeque = new AccessCountTableDeque( - new CountTableEvictor(tableHandler, perMinuteAccessTablesCount), hourTableListener); - - tableDeques.put(TimeGranularity.MINUTE, minuteTableDeque); - return TableAddOpListener.perMinute(minuteTableDeque, tableHandler, executorService); - } - - private void createSecondTableDeque( - AccessCountTableHandler tableHandler, - TableAddOpListener minuteTableListener) { - int aggregationIntervalMs = configuration.getInt( - SMART_ACCESS_COUNT_AGGREGATION_INTERVAL_MS, - SMART_ACCESS_COUNT_AGGREGATION_INTERVAL_MS_DEFAULT); - int minimalSecondAccessTablesCount = - (int) (ONE_MINUTE_IN_MILLIS / aggregationIntervalMs); - - int perSecondAccessTablesCount = getAccessTablesCount( - SMART_NUM_SECOND_TABLES_TO_KEEP_KEY, - SMART_NUM_SECOND_TABLES_TO_KEEP_DEFAULT, - minimalSecondAccessTablesCount); - this.secondTableDeque = new AccessCountTableDeque( - new CountTableEvictor(tableHandler, perSecondAccessTablesCount), minuteTableListener); - - tableDeques.put(TimeGranularity.SECOND, secondTableDeque); - } - - private int getAccessTablesCount(String configKey, int defaultValue, int minimalCount) { - int tableCount = configuration.getInt(configKey, defaultValue); - - if (tableCount < minimalCount) { - String errorMessage = String.format( - "Wrong value for option %s. It should be at least %d", configKey, minimalCount); - LOG.error(errorMessage); - throw new IllegalArgumentException(errorMessage); - } - return tableCount; - } -} diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/InMemoryAccessEventAggregator.java b/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/InMemoryAccessEventAggregator.java deleted file mode 100644 index b1b82cf46eb..00000000000 --- a/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/InMemoryAccessEventAggregator.java +++ /dev/null @@ -1,166 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.smartdata.metastore.dao.accesscount; - -import lombok.Data; -import org.apache.commons.lang.StringUtils; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.smartdata.metastore.dao.FileInfoDao; -import org.smartdata.metastore.model.AggregatedAccessCounts; -import org.smartdata.metrics.FileAccessEvent; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.stream.Collectors; - -public class InMemoryAccessEventAggregator implements AccessEventAggregator { - public static final Logger LOG = LoggerFactory.getLogger(InMemoryAccessEventAggregator.class); - - private final FileInfoDao fileInfoDao; - private final WindowClosedCallback windowClosedCallback; - private final AccessCountEventAggregatorFailover failover; - private final List eventBuffer; - private final long aggregationGranularity; - private Window currentWindow; - private Map unmergedAccessCounts = new HashMap<>(); - - public InMemoryAccessEventAggregator( - FileInfoDao fileInfoDao, - WindowClosedCallback windowClosedCallback, - AccessCountEventAggregatorFailover failover, - long aggregationGranularity) { - this.fileInfoDao = fileInfoDao; - this.windowClosedCallback = windowClosedCallback; - this.aggregationGranularity = aggregationGranularity; - this.failover = failover; - this.eventBuffer = new ArrayList<>(); - } - - @Override - public void aggregate(List events) { - if (currentWindow == null && !events.isEmpty()) { - currentWindow = assignWindow(events.get(0).getTimestamp()); - } - for (FileAccessEvent event : events) { - if (!currentWindow.contains(event.getTimestamp())) { - // New Window occurs - closeCurrentWindow(); - currentWindow = assignWindow(event.getTimestamp()); - eventBuffer.clear(); - } - // Exclude watermark event - if (!StringUtils.isBlank(event.getPath())) { - eventBuffer.add(event); - } - } - } - - private void closeCurrentWindow() { - Map aggregatedAccessCounts = aggregateWindowEvents(); - - try { - windowClosedCallback.onWindowClosed( - currentWindow.start, currentWindow.end, aggregatedAccessCounts.values()); - } catch (Exception exception) { - unmergedAccessCounts = failover.handleError( - aggregatedAccessCounts, unmergedAccessCounts, exception); - } - } - - private Map aggregateWindowEvents() { - if (eventBuffer.isEmpty() && unmergedAccessCounts.isEmpty()) { - return Collections.emptyMap(); - } - - Map allAccessesCounts = aggregateAccessCounts(eventBuffer); - Set newAccessPaths = new HashSet<>(allAccessesCounts.keySet()); - mergeMapsInPlace(allAccessesCounts, unmergedAccessCounts); - - Map pathToIds = getFileIds(allAccessesCounts); - - unmergedAccessCounts = new HashMap<>(); - Map accessCountsWithFileId = new HashMap<>(); - - allAccessesCounts.forEach((path, accessesCounts) -> { - Long fileId = pathToIds.get(path); - - if (fileId != null) { - accessCountsWithFileId.put(path, accessesCounts.withFileId(fileId)); - return; - } - - if (newAccessPaths.contains(path)) { - // save only files from event buffer, i.e. drop unsuccessful files - // from previous unmergedAccessCounts - unmergedAccessCounts.put(path, accessesCounts); - } - }); - return accessCountsWithFileId; - } - - private Map getFileIds(Map allAccessesCounts) { - try { - return fileInfoDao.getPathFids(allAccessesCounts.keySet()); - } catch (Exception exception) { - LOG.error("Error fetching file ids for paths {}", allAccessesCounts.keySet(), exception); - return Collections.emptyMap(); - } - } - - private void mergeMapsInPlace(Map resultMap, - Map mapToMerge) { - mapToMerge.forEach((key, value) -> resultMap.merge(key, value, AggregatedAccessCounts::merge)); - } - - private Map aggregateAccessCounts(List events) { - return events.stream() - .collect(Collectors.toMap( - FileAccessEvent::getPath, - AggregatedAccessCounts::fromEvent, - AggregatedAccessCounts::merge - )); - } - - private Window assignWindow(long time) { - long start = time - (time % aggregationGranularity); - return new Window(start, start + aggregationGranularity); - } - - @Data - public static class Window { - private final long start; - private final long end; - - // [start, end) - public boolean contains(long time) { - return this.start <= time && this.end > time; - } - } - - public interface WindowClosedCallback { - void onWindowClosed(long windowStart, long windowEnd, - Collection aggregatedAccessCounts); - } -} diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/TableAddOpListener.java b/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/TableAddOpListener.java deleted file mode 100644 index 157fedb183c..00000000000 --- a/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/TableAddOpListener.java +++ /dev/null @@ -1,59 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.smartdata.metastore.dao.accesscount; - -import org.smartdata.metastore.model.AccessCountTable; -import org.smartdata.metastore.utils.Constants; - -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutorService; - -public interface TableAddOpListener { - CompletableFuture EMPTY_RESULT = CompletableFuture.completedFuture(null); - - CompletableFuture tableAdded( - AccessCountTableDeque fineGrainedTableDeque, AccessCountTable table); - - static TableAddOpListener perDay( - AccessCountTableDeque deque, - AccessCountTableHandler tableAggregator, - ExecutorService service) { - return new AggregatingTableAddOpListener( - deque, tableAggregator, service, Constants.ONE_DAY_IN_MILLIS); - } - - static TableAddOpListener perHour( - AccessCountTableDeque deque, - AccessCountTableHandler tableAggregator, - ExecutorService service) { - return new AggregatingTableAddOpListener( - deque, tableAggregator, service, Constants.ONE_HOUR_IN_MILLIS); - } - - static TableAddOpListener perMinute( - AccessCountTableDeque deque, - AccessCountTableHandler tableAggregator, - ExecutorService service) { - return new AggregatingTableAddOpListener( - deque, tableAggregator, service, Constants.ONE_MINUTE_IN_MILLIS); - } - - static TableAddOpListener noOp() { - return (deque, table) -> EMPTY_RESULT; - } -} diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/dao/impl/DefaultAccessCountEventDao.java b/smart-metastore/src/main/java/org/smartdata/metastore/dao/impl/DefaultAccessCountEventDao.java deleted file mode 100644 index 1706ddda10d..00000000000 --- a/smart-metastore/src/main/java/org/smartdata/metastore/dao/impl/DefaultAccessCountEventDao.java +++ /dev/null @@ -1,151 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.smartdata.metastore.dao.impl; - -import com.google.common.collect.ImmutableMap; -import org.smartdata.metastore.MetaStoreException; -import org.smartdata.metastore.SearchableAbstractDao; -import org.smartdata.metastore.dao.FileInfoDao; -import org.smartdata.metastore.dao.accesscount.AccessCountEventDao; -import org.smartdata.metastore.model.AccessCountTable; -import org.smartdata.metastore.model.AggregatedAccessCounts; -import org.smartdata.metastore.queries.MetastoreQuery; -import org.smartdata.metastore.queries.MetastoreQueryExecutor; -import org.smartdata.metastore.queries.sort.FileAccessInfoSortField; -import org.smartdata.model.FileAccessInfo; -import org.smartdata.model.request.FileAccessInfoSearchRequest; -import org.springframework.jdbc.core.ColumnMapRowMapper; -import org.springframework.jdbc.core.simple.SimpleJdbcInsert; -import org.springframework.transaction.PlatformTransactionManager; - -import javax.sql.DataSource; - -import java.sql.ResultSet; -import java.sql.SQLException; -import java.util.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import static org.smartdata.metastore.queries.MetastoreQuery.select; -import static org.smartdata.metastore.queries.expression.MetastoreQueryDsl.betweenEpochInclusive; -import static org.smartdata.metastore.queries.expression.MetastoreQueryDsl.in; -import static org.smartdata.metastore.queries.expression.MetastoreQueryDsl.like; - -public class DefaultAccessCountEventDao - extends - SearchableAbstractDao - implements AccessCountEventDao { - - private static final String TABLE_NAME = "access_count_table"; - private final MetastoreQueryExecutor queryExecutor; - - public DefaultAccessCountEventDao( - DataSource dataSource, PlatformTransactionManager transactionManager) { - super(dataSource, transactionManager, TABLE_NAME); - - this.queryExecutor = new MetastoreQueryExecutor(dataSource, transactionManager); - } - - @Override - public void insert(AccessCountTable table, - Collection aggregatedAccessCounts) { - insert(simpleJdbcInsert(table), aggregatedAccessCounts, this::toMap); - } - - protected Map toMap(AggregatedAccessCounts accessCounts) { - Map parameters = new HashMap<>(); - parameters.put(AccessCountEventDao.FILE_ID_FIELD, accessCounts.getFileId()); - parameters.put(AccessCountEventDao.ACCESS_COUNT_FIELD, accessCounts.getAccessCount()); - parameters.put(AccessCountEventDao.LAST_ACCESSED_TIME_FIELD, - accessCounts.getLastAccessedTimestamp()); - return parameters; - } - - @Override - public void validate(AccessCountTable table) throws MetaStoreException { - MetastoreQuery query = select(FILE_ID_FIELD, ACCESS_COUNT_FIELD, LAST_ACCESSED_TIME_FIELD) - .from(table.getTableName()) - .limit(1); - try { - queryExecutor.execute(query, new ColumnMapRowMapper()); - } catch (Exception exception) { - throw new MetaStoreException(exception); - } - } - - @Override - public void updateFileIds(List accessCountTables, - long srcFileId, long destFileId) throws MetaStoreException { - Exception lastException = null; - int failedNum = 0; - - for (AccessCountTable table : accessCountTables) { - String statement = "UPDATE " + table.getTableName() - + " SET fid = " + destFileId - + " WHERE fid = " + srcFileId; - - try { - jdbcTemplate.execute(statement); - } catch (Exception exception) { - lastException = exception; - failedNum++; - } - } - // Otherwise, ignore the exception because table evictor can evict access - // count tables, which are not synchronized. Even so, there is no impact on - // the measurement for data temperature. - if (failedNum == accessCountTables.size()) { - // Throw exception if all tables are not updated. - throw new MetaStoreException("Failed to update fid!", lastException); - } - } - - private SimpleJdbcInsert simpleJdbcInsert(AccessCountTable table) { - return new SimpleJdbcInsert(dataSource) - .withTableName(table.getTableName()); - } - - @Override - protected MetastoreQuery searchQuery(FileAccessInfoSearchRequest searchRequest) { - return select("access_counts.fid AS fid", - "SUM(access_counts.count) AS count", - "MAX(access_counts.last_accessed_time) as last_accessed_time", - "file.path as path") - .fromSubQuery(AccessCountEventDao.unionTablesQuery(searchRequest.getAccessCountTables()), - "access_counts") - .join("file", ImmutableMap.of("access_counts.fid", "file.fid")) - .where( - in("access_counts.fid", searchRequest.getIds()), - like("file.path", searchRequest.getPathLike()), - betweenEpochInclusive("access_counts.last_accessed_time", - searchRequest.getLastAccessedTime()) - ) - .groupBy("access_counts.fid", "file.path"); - } - - @Override - protected FileAccessInfo mapRow(ResultSet rs, int rowNum) throws SQLException { - return FileAccessInfo.builder() - .setFid(rs.getLong(AccessCountEventDao.FILE_ID_FIELD)) - .setPath(rs.getString(FileInfoDao.FILE_PATH_FIELD)) - .setAccessCount(rs.getInt(AccessCountEventDao.ACCESS_COUNT_FIELD)) - .setLastAccessedTime(rs.getLong(AccessCountEventDao.LAST_ACCESSED_TIME_FIELD)) - .build(); - } -} diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/dao/impl/DefaultAccessCountTableDao.java b/smart-metastore/src/main/java/org/smartdata/metastore/dao/impl/DefaultAccessCountTableDao.java deleted file mode 100644 index 04d0296b737..00000000000 --- a/smart-metastore/src/main/java/org/smartdata/metastore/dao/impl/DefaultAccessCountTableDao.java +++ /dev/null @@ -1,97 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.smartdata.metastore.dao.impl; - -import org.smartdata.metastore.dao.AbstractDao; -import org.smartdata.metastore.dao.accesscount.AccessCountTableDao; -import org.smartdata.metastore.model.AccessCountTable; -import org.smartdata.metastore.queries.MetastoreQuery; -import org.smartdata.metastore.queries.MetastoreQueryExecutor; -import org.springframework.transaction.PlatformTransactionManager; - -import javax.sql.DataSource; - -import java.sql.ResultSet; -import java.sql.SQLException; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import static org.smartdata.metastore.queries.MetastoreQuery.select; -import static org.smartdata.metastore.queries.MetastoreQuery.selectAll; -import static org.smartdata.metastore.queries.expression.MetastoreQueryDsl.equal; -import static org.smartdata.metastore.queries.sort.Sorting.ascending; - -public class DefaultAccessCountTableDao extends AbstractDao implements AccessCountTableDao { - - protected final MetastoreQueryExecutor queryExecutor; - - public DefaultAccessCountTableDao( - DataSource dataSource, PlatformTransactionManager transactionManager) { - super(dataSource, TABLE_NAME); - - this.queryExecutor = new MetastoreQueryExecutor(dataSource, transactionManager); - } - - @Override - public void insert(AccessCountTable table) { - if (!tableExists(table.getTableName())) { - insert(table, this::toMap); - } - } - - @Override - public boolean tableExists(String name) { - MetastoreQuery query = select("1") - .from(TABLE_NAME) - .where( - equal("table_name", name) - ) - .limit(1L); - - return queryExecutor.executeCount(query) != 0; - } - - @Override - public void delete(AccessCountTable table) { - final String sql = "DELETE FROM access_count_table WHERE table_name = ?"; - jdbcTemplate.update(sql, table.getTableName()); - } - - @Override - public List getAllSortedTables() { - MetastoreQuery query = selectAll() - .from(TABLE_NAME) - .orderBy(ascending("start_time")); - return queryExecutor.execute(query, this::mapRow); - } - - private Map toMap(AccessCountTable accessCountTable) { - Map parameters = new HashMap<>(); - parameters.put("table_name", accessCountTable.getTableName()); - parameters.put("start_time", accessCountTable.getStartTime()); - parameters.put("end_time", accessCountTable.getEndTime()); - return parameters; - } - - private AccessCountTable mapRow(ResultSet resultSet, int i) throws SQLException { - return new AccessCountTable( - resultSet.getLong("start_time"), - resultSet.getLong("end_time")); - } -} diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/dao/impl/DefaultCacheFileDao.java b/smart-metastore/src/main/java/org/smartdata/metastore/dao/impl/DefaultCacheFileDao.java index c2286575030..9dcbfe7bf0d 100644 --- a/smart-metastore/src/main/java/org/smartdata/metastore/dao/impl/DefaultCacheFileDao.java +++ b/smart-metastore/src/main/java/org/smartdata/metastore/dao/impl/DefaultCacheFileDao.java @@ -97,7 +97,7 @@ public void insert(List cachedFileStatusList) { } @Override - public int update(Long fid, Long lastAccessTime, Integer numAccessed) { + public int update(Long fid, Long lastAccessTime, long numAccessed) { String sql = "UPDATE cached_file SET last_access_time = ?, accessed_num = ? WHERE fid = ?"; return jdbcTemplate.update(sql, lastAccessTime, numAccessed, fid); } @@ -136,8 +136,7 @@ private void merge( long lastAccessTime = Math.max( cachedFileStatus.getLastAccessTime(), aggregatedAccessCounts.getLastAccessedTimestamp()); - - int accessCounts = cachedFileStatus.getNumAccessed() + long accessCounts = cachedFileStatus.getNumAccessed() + aggregatedAccessCounts.getAccessCount(); update(aggregatedAccessCounts.getFileId(), lastAccessTime, accessCounts); diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/dao/impl/DefaultDaoProvider.java b/smart-metastore/src/main/java/org/smartdata/metastore/dao/impl/DefaultDaoProvider.java index 18ea0331628..3f3c7632f5f 100644 --- a/smart-metastore/src/main/java/org/smartdata/metastore/dao/impl/DefaultDaoProvider.java +++ b/smart-metastore/src/main/java/org/smartdata/metastore/dao/impl/DefaultDaoProvider.java @@ -27,6 +27,8 @@ import org.smartdata.metastore.dao.CompressionFileDao; import org.smartdata.metastore.dao.DaoProvider; import org.smartdata.metastore.dao.ErasureCodingPolicyDao; +import org.smartdata.metastore.dao.FileAccessDao; +import org.smartdata.metastore.dao.FileAccessPartitionDao; import org.smartdata.metastore.dao.FileDiffDao; import org.smartdata.metastore.dao.FileInfoDao; import org.smartdata.metastore.dao.FileStateDao; @@ -39,8 +41,6 @@ import org.smartdata.metastore.dao.SystemInfoDao; import org.smartdata.metastore.dao.UserActivityDao; import org.smartdata.metastore.dao.WhitelistDao; -import org.smartdata.metastore.dao.accesscount.AccessCountEventDao; -import org.smartdata.metastore.dao.accesscount.AccessCountTableDao; import org.springframework.transaction.PlatformTransactionManager; import javax.sql.DataSource; @@ -90,13 +90,8 @@ public FileDiffDao fileDiffDao() { } @Override - public AccessCountTableDao accessCountDao() { - return new DefaultAccessCountTableDao(dataSource, transactionManager); - } - - @Override - public AccessCountEventDao accessCountEventDao() { - return new DefaultAccessCountEventDao(dataSource, transactionManager); + public FileAccessDao fileAccessDao() { + return new DefaultFileAccessDao(dataSource, transactionManager); } @Override @@ -163,4 +158,9 @@ public StoragePolicyDao storagePolicyDao() { public UserActivityDao userActivityDao() { return new DefaultUserActivityDao(dataSource, transactionManager); } + + @Override + public FileAccessPartitionDao fileAccessPartitionDao() { + return new DefaultFileAccessPartitionDao(dataSource); + } } diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/dao/impl/DefaultFileAccessDao.java b/smart-metastore/src/main/java/org/smartdata/metastore/dao/impl/DefaultFileAccessDao.java new file mode 100644 index 00000000000..e1b7d169020 --- /dev/null +++ b/smart-metastore/src/main/java/org/smartdata/metastore/dao/impl/DefaultFileAccessDao.java @@ -0,0 +1,109 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.smartdata.metastore.dao.impl; + +import org.smartdata.metastore.MetaStoreException; +import org.smartdata.metastore.SearchableAbstractDao; +import org.smartdata.metastore.dao.FileAccessDao; +import org.smartdata.metastore.dao.FileInfoDao; +import org.smartdata.metastore.model.AggregatedAccessCounts; +import org.smartdata.metastore.queries.MetastoreQuery; +import org.smartdata.metastore.queries.sort.FileAccessInfoSortField; +import org.smartdata.model.FileAccessInfo; +import org.smartdata.model.request.FileAccessInfoSearchRequest; +import org.springframework.jdbc.core.simple.SimpleJdbcInsert; +import org.springframework.transaction.PlatformTransactionManager; + +import javax.sql.DataSource; + +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; + +import static org.smartdata.metastore.queries.MetastoreQuery.select; +import static org.smartdata.metastore.queries.expression.MetastoreQueryDsl.betweenEpochInclusive; +import static org.smartdata.metastore.queries.expression.MetastoreQueryDsl.in; +import static org.smartdata.metastore.queries.expression.MetastoreQueryDsl.like; + +public class DefaultFileAccessDao + extends + SearchableAbstractDao + implements FileAccessDao { + + public DefaultFileAccessDao( + DataSource dataSource, PlatformTransactionManager transactionManager) { + super(dataSource, transactionManager, TABLE_NAME); + } + + @Override + public void insert(Collection aggregatedAccessCounts) { + insert(new SimpleJdbcInsert(dataSource).withTableName(TABLE_NAME), aggregatedAccessCounts, + this::toMap); + } + + protected Map toMap(AggregatedAccessCounts accessCounts) { + Map parameters = new HashMap<>(); + parameters.put(FileAccessDao.FILE_ID_FIELD, accessCounts.getFileId()); + parameters.put(FileAccessDao.ACCESS_TIME_FIELD, + accessCounts.getLastAccessedTimestamp()); + return parameters; + } + + @Override + public void updateFileIds(long srcFileId, long destFileId) throws MetaStoreException { + String statement = "UPDATE " + TABLE_NAME + + " SET fid = " + destFileId + + " WHERE fid = " + srcFileId; + try { + jdbcTemplate.execute(statement); + } catch (Exception exception) { + throw new MetaStoreException("Failed to update fid!", exception); + } + } + + @Override + protected MetastoreQuery searchQuery(FileAccessInfoSearchRequest searchRequest) { + return select("fid", + "count", + "access_time", + "path") + .fromSubQuery("SELECT file.fid, count(*) AS count,\n" + + "MAX(file_access.access_time) as access_time, file.path as path\n" + + "FROM file_access\n" + + " JOIN file ON file_access.fid = file.fid\n" + + "GROUP BY file.fid, file.path", "f") + .where( + in("fid", searchRequest.getIds()), + like("path", searchRequest.getPathLike()), + betweenEpochInclusive("access_time", + searchRequest.getLastAccessedTime()) + ); + } + + @Override + protected FileAccessInfo mapRow(ResultSet rs, int rowNum) throws SQLException { + return FileAccessInfo.builder() + .setFid(rs.getLong(FileAccessDao.FILE_ID_FIELD)) + .setPath(rs.getString(FileInfoDao.FILE_PATH_FIELD)) + .setAccessCount(rs.getInt(FileAccessDao.ACCESS_COUNT_FIELD)) + .setLastAccessedTime(rs.getLong(FileAccessDao.ACCESS_TIME_FIELD)) + .build(); + } +} diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/dao/impl/DefaultFileAccessPartitionDao.java b/smart-metastore/src/main/java/org/smartdata/metastore/dao/impl/DefaultFileAccessPartitionDao.java new file mode 100644 index 00000000000..b7bc37ea117 --- /dev/null +++ b/smart-metastore/src/main/java/org/smartdata/metastore/dao/impl/DefaultFileAccessPartitionDao.java @@ -0,0 +1,85 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.smartdata.metastore.dao.impl; + +import lombok.extern.slf4j.Slf4j; +import org.smartdata.metastore.MetaStoreException; +import org.smartdata.metastore.dao.AbstractDao; +import org.smartdata.metastore.dao.FileAccessPartitionDao; +import org.smartdata.metastore.model.FileAccessPartition; +import org.springframework.dao.EmptyResultDataAccessException; + +import javax.sql.DataSource; + +import java.time.LocalDateTime; +import java.util.Collections; +import java.util.List; + +@Slf4j +public class DefaultFileAccessPartitionDao extends AbstractDao implements FileAccessPartitionDao { + + private static final String CREATE_NEW_PARTITION_ERR_MSG = + "Failed to create new partition for file_access table"; + + public DefaultFileAccessPartitionDao(DataSource dataSource) { + super(dataSource, ""); + } + + @Override + public void create(LocalDateTime date) throws MetaStoreException { + try { + Integer result = + jdbcTemplate.queryForObject("select create_file_access_partition(?);", Integer.class, + date); + if (result == null) { + throw new MetaStoreException(CREATE_NEW_PARTITION_ERR_MSG); + } + if (result == 1) { + log.info("Created partition for file_access table for date {}", date); + } + } catch (Exception e) { + throw new MetaStoreException(CREATE_NEW_PARTITION_ERR_MSG, e); + } + } + + @Override + public List getAll() { + String query = "SELECT inhrelid AS id, inhrelid::regclass AS name, " + + "cast(REPLACE(REPLACE(inhrelid::regclass::text, 'file_access_', ''),'_','-') as date) " + + "as partition_date FROM pg_catalog.pg_inherits " + + "WHERE inhparent = 'file_access'::regclass " + + "ORDER BY partition_date ASC;"; + try { + return jdbcTemplate.query(query, + (rs, rowNum) -> new FileAccessPartition(rs.getLong("id"), + rs.getString("name"), + rs.getDate("partition_date").toLocalDate())); + } catch (EmptyResultDataAccessException e) { + return Collections.emptyList(); + } + } + + @Override + public void remove(FileAccessPartition partition) { + String query = String.format("DROP TABLE %s;", partition.getName()); + int result = jdbcTemplate.update(query); + if (result == 1) { + log.info("Dropped file access partition {}", partition); + } + } +} diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/dao/sqlite/SqliteDaoProvider.java b/smart-metastore/src/main/java/org/smartdata/metastore/dao/sqlite/SqliteDaoProvider.java deleted file mode 100644 index da67ca6ab8b..00000000000 --- a/smart-metastore/src/main/java/org/smartdata/metastore/dao/sqlite/SqliteDaoProvider.java +++ /dev/null @@ -1,44 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.smartdata.metastore.dao.sqlite; - -import org.smartdata.metastore.DBPool; -import org.smartdata.metastore.dao.FileInfoDao; -import org.smartdata.metastore.dao.FileStateDao; -import org.smartdata.metastore.dao.SmallFileDao; -import org.smartdata.metastore.dao.impl.DefaultDaoProvider; -import org.springframework.transaction.PlatformTransactionManager; - -public class SqliteDaoProvider extends DefaultDaoProvider { - public SqliteDaoProvider(DBPool dbPool, PlatformTransactionManager transactionManager) { - super(dbPool, transactionManager); - } - - @Override - public FileInfoDao fileInfoDao() { - return new SqliteFileInfoDao(dataSource); - } - - public FileStateDao fileStateDao() { - return new SqliteFileStateDao(dataSource); - } - - public SmallFileDao smallFileDao() { - return new SqliteSmallFileDao(dataSource); - } -} diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/dao/sqlite/SqliteFileStateDao.java b/smart-metastore/src/main/java/org/smartdata/metastore/dao/sqlite/SqliteFileStateDao.java deleted file mode 100644 index 0551dd980ef..00000000000 --- a/smart-metastore/src/main/java/org/smartdata/metastore/dao/sqlite/SqliteFileStateDao.java +++ /dev/null @@ -1,34 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.smartdata.metastore.dao.sqlite; - -import org.smartdata.metastore.dao.impl.DefaultFileStateDao; - -import javax.sql.DataSource; - -public class SqliteFileStateDao extends DefaultFileStateDao { - - public SqliteFileStateDao(DataSource dataSource) { - super(dataSource); - } - - protected void renameDirectoryFiles(String oldPath, String newPath) { - String sql = "UPDATE " + tableName + " SET path = ? || SUBSTR(path, ?) WHERE path LIKE ?"; - jdbcTemplate.update(sql, newPath, oldPath.length() + 1, oldPath + "/%"); - } -} diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/dao/sqlite/SqliteSmallFileDao.java b/smart-metastore/src/main/java/org/smartdata/metastore/dao/sqlite/SqliteSmallFileDao.java deleted file mode 100644 index 28729bd69f9..00000000000 --- a/smart-metastore/src/main/java/org/smartdata/metastore/dao/sqlite/SqliteSmallFileDao.java +++ /dev/null @@ -1,34 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.smartdata.metastore.dao.sqlite; - -import org.smartdata.metastore.dao.impl.DefaultSmallFileDao; - -import javax.sql.DataSource; - -public class SqliteSmallFileDao extends DefaultSmallFileDao { - - public SqliteSmallFileDao(DataSource dataSource) { - super(dataSource); - } - - protected void renameDirectoryFiles(String oldPath, String newPath) { - String sql = "UPDATE " + tableName + " SET path = ? || SUBSTR(path, ?) WHERE path LIKE ?"; - jdbcTemplate.update(sql, newPath, oldPath.length() + 1, oldPath + "/%"); - } -} diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/db/DBHandlersFactory.java b/smart-metastore/src/main/java/org/smartdata/metastore/db/DBHandlersFactory.java index 8c75436f92f..d9c1cc98ba8 100644 --- a/smart-metastore/src/main/java/org/smartdata/metastore/db/DBHandlersFactory.java +++ b/smart-metastore/src/main/java/org/smartdata/metastore/db/DBHandlersFactory.java @@ -22,46 +22,29 @@ import org.smartdata.metastore.DBType; import org.smartdata.metastore.MetaStoreException; import org.smartdata.metastore.db.metadata.DbMetadataProvider; -import org.smartdata.metastore.db.metadata.MySqlDbMetadataProvider; import org.smartdata.metastore.db.metadata.PostgresDbMetadataProvider; -import org.smartdata.metastore.db.metadata.SqliteDbMetadataProvider; import javax.sql.DataSource; -import static org.smartdata.conf.SmartConfKeys.SMART_METASTORE_LEGACY_MYSQL_SUPPORT_DEFAULT; -import static org.smartdata.conf.SmartConfKeys.SMART_METASTORE_LEGACY_MYSQL_SUPPORT_KEY; import static org.smartdata.conf.SmartConfKeys.SMART_METASTORE_MIGRATION_CHANGELOG_PATH_DEFAULT; import static org.smartdata.conf.SmartConfKeys.SMART_METASTORE_MIGRATION_CHANGELOG_PATH_KEY; public class DBHandlersFactory { - private static final String OLD_MYSQL_LABEL = "old_mysql"; public DbSchemaManager createDbManager(DBPool dbPool, Configuration conf) { String changelogPath = conf.get( SMART_METASTORE_MIGRATION_CHANGELOG_PATH_KEY, SMART_METASTORE_MIGRATION_CHANGELOG_PATH_DEFAULT); - - boolean legacyMysqlSupportEnabled = conf.getBoolean( - SMART_METASTORE_LEGACY_MYSQL_SUPPORT_KEY, - SMART_METASTORE_LEGACY_MYSQL_SUPPORT_DEFAULT); - - String labelsFilter = legacyMysqlSupportEnabled - ? OLD_MYSQL_LABEL - : "!" + OLD_MYSQL_LABEL; - - return new LiquibaseDbSchemaManager(dbPool, changelogPath, labelsFilter); + return new LiquibaseDbSchemaManager(dbPool, changelogPath); } public DbMetadataProvider createDbMetadataProvider(DBPool dbPool, DBType dbType) throws MetaStoreException { DataSource dataSource = dbPool.getDataSource(); switch (dbType) { - case MYSQL: - return new MySqlDbMetadataProvider(dataSource); case POSTGRES: - return new PostgresDbMetadataProvider(dataSource); default: - return new SqliteDbMetadataProvider(dataSource); + return new PostgresDbMetadataProvider(dataSource); } } } diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/db/LiquibaseDbSchemaManager.java b/smart-metastore/src/main/java/org/smartdata/metastore/db/LiquibaseDbSchemaManager.java index 58fb169f340..06a3d6d2072 100644 --- a/smart-metastore/src/main/java/org/smartdata/metastore/db/LiquibaseDbSchemaManager.java +++ b/smart-metastore/src/main/java/org/smartdata/metastore/db/LiquibaseDbSchemaManager.java @@ -35,14 +35,12 @@ public class LiquibaseDbSchemaManager implements DbSchemaManager { private final DBPool pool; private final String changelogPath; - private final String labelFilterExpr; private final Map scopeConfig; - public LiquibaseDbSchemaManager(DBPool pool, String changelogPath, String labelFilterExpr) { + public LiquibaseDbSchemaManager(DBPool pool, String changelogPath) { this.pool = pool; this.changelogPath = changelogPath; - this.labelFilterExpr = labelFilterExpr; this.scopeConfig = new HashMap<>(); scopeConfig.put(Scope.Attr.ui.name(), new LoggerUIService()); @@ -57,8 +55,7 @@ public void initializeDatabase() throws Exception { CommandScope updateCommand = new CommandScope(UpdateCommandStep.COMMAND_NAME) .addArgumentValue(DbUrlConnectionCommandStep.DATABASE_ARG, db) - .addArgumentValue(UpdateCommandStep.CHANGELOG_FILE_ARG, changelogPath) - .addArgumentValue(UpdateCommandStep.LABEL_FILTER_ARG, labelFilterExpr); + .addArgumentValue(UpdateCommandStep.CHANGELOG_FILE_ARG, changelogPath); executeWithLogging(updateCommand); } diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/model/AggregatedAccessCounts.java b/smart-metastore/src/main/java/org/smartdata/metastore/model/AggregatedAccessCounts.java index 9d6afae040a..c3c4ab4d0bb 100644 --- a/smart-metastore/src/main/java/org/smartdata/metastore/model/AggregatedAccessCounts.java +++ b/smart-metastore/src/main/java/org/smartdata/metastore/model/AggregatedAccessCounts.java @@ -27,7 +27,7 @@ @RequiredArgsConstructor public class AggregatedAccessCounts { private long fileId; - private final int accessCount; + private final long accessCount; private final long lastAccessedTimestamp; public AggregatedAccessCounts merge(AggregatedAccessCounts other) { diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/dao/sqlite/SqliteFileInfoDao.java b/smart-metastore/src/main/java/org/smartdata/metastore/model/FileAccessPartition.java similarity index 60% rename from smart-metastore/src/main/java/org/smartdata/metastore/dao/sqlite/SqliteFileInfoDao.java rename to smart-metastore/src/main/java/org/smartdata/metastore/model/FileAccessPartition.java index f3ec1508edb..5318f2214a4 100644 --- a/smart-metastore/src/main/java/org/smartdata/metastore/dao/sqlite/SqliteFileInfoDao.java +++ b/smart-metastore/src/main/java/org/smartdata/metastore/model/FileAccessPartition.java @@ -15,20 +15,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.smartdata.metastore.dao.sqlite; +package org.smartdata.metastore.model; -import org.smartdata.metastore.dao.impl.DefaultFileInfoDao; +import lombok.EqualsAndHashCode; +import lombok.Getter; +import lombok.RequiredArgsConstructor; -import javax.sql.DataSource; +import java.time.LocalDate; -public class SqliteFileInfoDao extends DefaultFileInfoDao { - - public SqliteFileInfoDao(DataSource dataSource) { - super(dataSource); - } - - protected void renameDirectoryFiles(String oldPath, String newPath) { - String sql = "UPDATE " + tableName + " SET path = ? || SUBSTR(path, ?) WHERE path LIKE ?"; - jdbcTemplate.update(sql, newPath, oldPath.length() + 1, oldPath + "/%"); - } +@EqualsAndHashCode +@Getter +@RequiredArgsConstructor +public class FileAccessPartition { + private final long id; + private final String name; + private final LocalDate date; } diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/partition/FileAccessPartitionManager.java b/smart-metastore/src/main/java/org/smartdata/metastore/partition/FileAccessPartitionManager.java new file mode 100644 index 00000000000..9c2cf1c891c --- /dev/null +++ b/smart-metastore/src/main/java/org/smartdata/metastore/partition/FileAccessPartitionManager.java @@ -0,0 +1,23 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.smartdata.metastore.partition; + +public interface FileAccessPartitionManager { + + void createNewPartitions(); +} diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/AccessCountTableEvictor.java b/smart-metastore/src/main/java/org/smartdata/metastore/partition/FileAccessPartitionManagerImpl.java similarity index 51% rename from smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/AccessCountTableEvictor.java rename to smart-metastore/src/main/java/org/smartdata/metastore/partition/FileAccessPartitionManagerImpl.java index d791dab3287..69307613741 100644 --- a/smart-metastore/src/main/java/org/smartdata/metastore/dao/accesscount/AccessCountTableEvictor.java +++ b/smart-metastore/src/main/java/org/smartdata/metastore/partition/FileAccessPartitionManagerImpl.java @@ -15,29 +15,34 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.smartdata.metastore.dao.accesscount; +package org.smartdata.metastore.partition; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import lombok.extern.slf4j.Slf4j; +import org.smartdata.metastore.MetaStore; import org.smartdata.metastore.MetaStoreException; -import org.smartdata.metastore.model.AccessCountTable; +import org.smartdata.metastore.dao.FileAccessPartitionDao; -public abstract class AccessCountTableEvictor { - public static final Logger LOG = LoggerFactory.getLogger(AccessCountTableEvictor.class); - private final AccessCountTableHandler tableDeleter; +import java.time.LocalDateTime; - public AccessCountTableEvictor(AccessCountTableHandler tableDeleter) { - this.tableDeleter = tableDeleter; +@Slf4j +public class FileAccessPartitionManagerImpl implements FileAccessPartitionManager { + + private final FileAccessPartitionDao fileAccessPartitionDao; + + public FileAccessPartitionManagerImpl(MetaStore metaStore) { + this.fileAccessPartitionDao = metaStore.fileAccessPartitionDao(); } - public void dropTable(AccessCountTable accessCountTable) { + @Override + public void createNewPartitions() { try { - tableDeleter.dropTable(accessCountTable); - LOG.debug("Dropped access count table " + accessCountTable.getTableName()); + //create partition for current and next months if they don't exist + LocalDateTime currentDate = LocalDateTime.now(); + fileAccessPartitionDao.create(currentDate); + fileAccessPartitionDao.create(currentDate.plusMonths(1)); } catch (MetaStoreException e) { - LOG.error("Drop access count table {} failed", accessCountTable.getTableName(), e); + log.error("Failed to create partitions", e); + throw new RuntimeException(e); } } - - abstract void evictTables(AccessCountTableDeque tables, long lastAggregatedIntervalEndTimestamp); } diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/partition/FileAccessPartitionService.java b/smart-metastore/src/main/java/org/smartdata/metastore/partition/FileAccessPartitionService.java new file mode 100644 index 00000000000..27794ab93fe --- /dev/null +++ b/smart-metastore/src/main/java/org/smartdata/metastore/partition/FileAccessPartitionService.java @@ -0,0 +1,90 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.smartdata.metastore.partition; + +import lombok.RequiredArgsConstructor; +import lombok.extern.slf4j.Slf4j; +import org.smartdata.metastore.partition.cleanup.FileAccessPartitionRetentionPolicyExecutor; + +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; + +@RequiredArgsConstructor +@Slf4j +public class FileAccessPartitionService { + private static final int CREATE_PARTITION_INTERVAL_DAYS = 30; + private static final int CLEANUP_PARTITION_INTERVAL_DAYS = 30; + private final ScheduledExecutorService scheduledExecutorService; + private final CreatePartitionTask createPartitionTask; + private final CleanupPartitionTask cleanupPartitionTask; + private ScheduledFuture createPartitionFuture; + private ScheduledFuture removePartitionFuture; + + public FileAccessPartitionService(ScheduledExecutorService service, + FileAccessPartitionManager fileAccessPartitionManager, + FileAccessPartitionRetentionPolicyExecutor retentionPolicyExecutor) { + this.scheduledExecutorService = service; + this.createPartitionTask = new CreatePartitionTask(fileAccessPartitionManager); + this.cleanupPartitionTask = new CleanupPartitionTask(retentionPolicyExecutor); + } + + public void start() { + this.createPartitionFuture = scheduledExecutorService.scheduleAtFixedRate( + createPartitionTask, 0, CREATE_PARTITION_INTERVAL_DAYS, TimeUnit.DAYS); + this.removePartitionFuture = scheduledExecutorService.scheduleAtFixedRate( + cleanupPartitionTask, 0, CLEANUP_PARTITION_INTERVAL_DAYS, TimeUnit.DAYS); + } + + public void stop() { + if (createPartitionFuture != null) { + createPartitionFuture.cancel(true); + } + if (removePartitionFuture != null) { + removePartitionFuture.cancel(true); + } + } + + @RequiredArgsConstructor + private static class CreatePartitionTask implements Runnable { + private final FileAccessPartitionManager partitionManager; + + @Override + public void run() { + try { + partitionManager.createNewPartitions(); + } catch (Exception e) { + log.error("CreatePartitionTask failed", e); + } + } + } + + @RequiredArgsConstructor + private static class CleanupPartitionTask implements Runnable { + private final FileAccessPartitionRetentionPolicyExecutor retentionPolicyExecutor; + + @Override + public void run() { + try { + retentionPolicyExecutor.cleanup(); + } catch (Exception e) { + log.error("CleanupPartitionTask failed", e); + } + } + } +} diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/partition/cleanup/FileAccessPartitionRetentionPolicyExecutor.java b/smart-metastore/src/main/java/org/smartdata/metastore/partition/cleanup/FileAccessPartitionRetentionPolicyExecutor.java new file mode 100644 index 00000000000..0899cb18783 --- /dev/null +++ b/smart-metastore/src/main/java/org/smartdata/metastore/partition/cleanup/FileAccessPartitionRetentionPolicyExecutor.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.smartdata.metastore.partition.cleanup; + +public interface FileAccessPartitionRetentionPolicyExecutor { + + void cleanup(); + + FileAccessPartitionRetentionPolicyType getPolicyType(); +} diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/partition/cleanup/FileAccessPartitionRetentionPolicyExecutorFactory.java b/smart-metastore/src/main/java/org/smartdata/metastore/partition/cleanup/FileAccessPartitionRetentionPolicyExecutorFactory.java new file mode 100644 index 00000000000..9032b066fdd --- /dev/null +++ b/smart-metastore/src/main/java/org/smartdata/metastore/partition/cleanup/FileAccessPartitionRetentionPolicyExecutorFactory.java @@ -0,0 +1,66 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.smartdata.metastore.partition.cleanup; + +import lombok.RequiredArgsConstructor; +import org.smartdata.conf.SmartConf; +import org.smartdata.metastore.MetaStore; +import org.smartdata.metastore.partition.cleanup.impl.MonthCountFileAccessPartitionRetentionPolicyExecutor; + +import static org.smartdata.conf.SmartConfKeys.SMART_FILE_ACCESS_PARTITIONS_RETENTION_COUNT_DEFAULT; +import static org.smartdata.conf.SmartConfKeys.SMART_FILE_ACCESS_PARTITIONS_RETENTION_COUNT_KEY; +import static org.smartdata.conf.SmartConfKeys.SMART_FILE_ACCESS_PARTITIONS_RETENTION_POLICY_KEY; + +@RequiredArgsConstructor +public class FileAccessPartitionRetentionPolicyExecutorFactory { + + private final MetaStore metaStore; + + public FileAccessPartitionRetentionPolicyExecutor createPolicyExecutor(SmartConf conf) { + FileAccessPartitionRetentionPolicyType policyType = + conf.getEnum(SMART_FILE_ACCESS_PARTITIONS_RETENTION_POLICY_KEY, + FileAccessPartitionRetentionPolicyType.MONTH_COUNT); + int retentionCount = conf.getInt(SMART_FILE_ACCESS_PARTITIONS_RETENTION_COUNT_KEY, + SMART_FILE_ACCESS_PARTITIONS_RETENTION_COUNT_DEFAULT); + switch (policyType) { + case MONTH_COUNT: + default: + return new MonthCountFileAccessPartitionRetentionPolicyExecutor( + metaStore.fileAccessPartitionDao(), + retentionCount); + } + } +} + + + + + + + + + + + + + + + + + + diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/partition/cleanup/FileAccessPartitionRetentionPolicyType.java b/smart-metastore/src/main/java/org/smartdata/metastore/partition/cleanup/FileAccessPartitionRetentionPolicyType.java new file mode 100644 index 00000000000..8282c490214 --- /dev/null +++ b/smart-metastore/src/main/java/org/smartdata/metastore/partition/cleanup/FileAccessPartitionRetentionPolicyType.java @@ -0,0 +1,22 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.smartdata.metastore.partition.cleanup; + +public enum FileAccessPartitionRetentionPolicyType { + MONTH_COUNT +} diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/partition/cleanup/impl/MonthCountFileAccessPartitionRetentionPolicyExecutor.java b/smart-metastore/src/main/java/org/smartdata/metastore/partition/cleanup/impl/MonthCountFileAccessPartitionRetentionPolicyExecutor.java new file mode 100644 index 00000000000..f976d738d26 --- /dev/null +++ b/smart-metastore/src/main/java/org/smartdata/metastore/partition/cleanup/impl/MonthCountFileAccessPartitionRetentionPolicyExecutor.java @@ -0,0 +1,61 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.smartdata.metastore.partition.cleanup.impl; + +import lombok.RequiredArgsConstructor; +import lombok.extern.slf4j.Slf4j; +import org.smartdata.metastore.dao.FileAccessPartitionDao; +import org.smartdata.metastore.model.FileAccessPartition; +import org.smartdata.metastore.partition.cleanup.FileAccessPartitionRetentionPolicyExecutor; +import org.smartdata.metastore.partition.cleanup.FileAccessPartitionRetentionPolicyType; + +import java.util.List; + +@Slf4j +@RequiredArgsConstructor +public class MonthCountFileAccessPartitionRetentionPolicyExecutor + implements FileAccessPartitionRetentionPolicyExecutor { + + private final FileAccessPartitionDao fileAccessPartitionDao; + private final int monthCount; + + @Override + public void cleanup() { + if (monthCount <= 0) { + return; + } + List fileAccessPartitions = fileAccessPartitionDao.getAll(); + //we should remove + 1 partition, because we also have partition for the next month + if (fileAccessPartitions.size() > monthCount + 1) { + List partitionsToRemove = + fileAccessPartitions.subList(monthCount, fileAccessPartitions.size()); + for (FileAccessPartition fileAccessPartition : partitionsToRemove) { + fileAccessPartitionDao.remove(fileAccessPartition); + } + } + log.info( + "File access partitions were cleanup successfully by retention policy: {}," + + " retention count: {}", + getPolicyType(), monthCount); + } + + @Override + public FileAccessPartitionRetentionPolicyType getPolicyType() { + return FileAccessPartitionRetentionPolicyType.MONTH_COUNT; + } +} diff --git a/smart-metastore/src/main/java/org/smartdata/metastore/utils/MetaStoreUtils.java b/smart-metastore/src/main/java/org/smartdata/metastore/utils/MetaStoreUtils.java index 11f0e25ed5f..f218f5ae471 100644 --- a/smart-metastore/src/main/java/org/smartdata/metastore/utils/MetaStoreUtils.java +++ b/smart-metastore/src/main/java/org/smartdata/metastore/utils/MetaStoreUtils.java @@ -54,11 +54,10 @@ public class MetaStoreUtils { static final Logger LOG = LoggerFactory.getLogger(MetaStoreUtils.class); public static final List SSM_TABLES = Lists.newArrayList( - "access_count_table", - "blank_access_count_info", "cached_file", "ec_policy", "file", + "file_access", "storage", "storage_hist", "storage_policy", @@ -134,13 +133,6 @@ private static Properties loadDruidConfig(SmartConf conf, File cpConfigFile) p.setProperty("url", url); } - String purl = p.getProperty("url"); - if (purl == null || purl.isEmpty()) { - purl = getDefaultSqliteDB(); // For testing - p.setProperty("url", purl); - LOG.warn("Database URL not specified, using " + purl); - } - try { conf.getPasswordFromHadoop(SmartConfKeys.SMART_METASTORE_PASSWORD) .filter(password -> !StringUtils.isBlank(password)) @@ -185,24 +177,11 @@ private static Properties loadDefaultDruidConfig(SmartConf conf, File cpConfigFi return properties; } - /** - * This default behavior provided here is mainly for convenience. - */ - private static String getDefaultSqliteDB() { - String absFilePath = System.getProperty("user.home") - + "/smart-test-default.db"; - return MetaStoreUtils.SQLITE_URL_PREFIX + absFilePath; - } - private static DBType getDbType(DBPool dbPool) throws MetaStoreException { try (Connection connection = dbPool.getConnection()) { String driver = connection.getMetaData().getDriverName(); driver = driver.toLowerCase(); - if (driver.contains("sqlite")) { - return DBType.SQLITE; - } else if (driver.contains("mysql")) { - return DBType.MYSQL; - } else if (driver.contains("postgres")) { + if (driver.contains("postgres")) { return DBType.POSTGRES; } else { throw new MetaStoreException("Unknown database: " + driver); diff --git a/smart-metastore/src/main/resources/db/changelog/changelog-1.init-db.xml b/smart-metastore/src/main/resources/db/changelog/changelog-1.init-db.xml index 0b0336904cf..23696dcc968 100644 --- a/smart-metastore/src/main/resources/db/changelog/changelog-1.init-db.xml +++ b/smart-metastore/src/main/resources/db/changelog/changelog-1.init-db.xml @@ -8,13 +8,6 @@ http://www.liquibase.org/xml/ns/dbchangelog/dbchangelog-latest.xsd http://www.liquibase.org/xml/ns/dbchangelog-ext http://www.liquibase.org/xml/ns/dbchangelog/dbchangelog-ext.xsd http://www.liquibase.org/xml/ns/pro http://www.liquibase.org/xml/ns/pro/liquibase-pro-latest.xsd"> - - - SET GLOBAL innodb_file_format=barracuda; - SET GLOBAL innodb_file_per_table=true; - SET GLOBAL innodb_large_prefix = ON; - - @@ -76,15 +69,6 @@ - - - - - - ALTER TABLE cached_file ROW_FORMAT= DYNAMIC ENGINE=INNODB; - - - @@ -170,9 +154,6 @@ - - - @@ -499,15 +480,6 @@ - - - - - - ALTER TABLE file_diff ROW_FORMAT= DYNAMIC ENGINE=INNODB; - - - @@ -517,9 +489,6 @@ - - - @@ -734,9 +703,6 @@ - - - diff --git a/smart-metastore/src/main/resources/db/changelog/changelog-3.create-user-activities-table.xml b/smart-metastore/src/main/resources/db/changelog/changelog-3.create-user-activities-table.xml index d39cdff6f2e..fc3e17355fd 100644 --- a/smart-metastore/src/main/resources/db/changelog/changelog-3.create-user-activities-table.xml +++ b/smart-metastore/src/main/resources/db/changelog/changelog-3.create-user-activities-table.xml @@ -40,13 +40,6 @@ - - - - \ No newline at end of file diff --git a/smart-metastore/src/main/resources/db/changelog/changelog-5.add-partitioning.xml b/smart-metastore/src/main/resources/db/changelog/changelog-5.add-partitioning.xml new file mode 100644 index 00000000000..b7cc1bda24b --- /dev/null +++ b/smart-metastore/src/main/resources/db/changelog/changelog-5.add-partitioning.xml @@ -0,0 +1,55 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + select count(*) from pg_proc where proname = 'create_file_access_partition' + + + + select create_file_access_partition(cast(now() as date)); + + + + + + + + + + + + + + \ No newline at end of file diff --git a/smart-metastore/src/main/resources/db/changelog/changelog-root.xml b/smart-metastore/src/main/resources/db/changelog/changelog-root.xml index 13aa64639bc..55021b0e158 100644 --- a/smart-metastore/src/main/resources/db/changelog/changelog-root.xml +++ b/smart-metastore/src/main/resources/db/changelog/changelog-root.xml @@ -12,4 +12,5 @@ + \ No newline at end of file diff --git a/smart-metastore/src/main/resources/db/changelog/sql/create_file_access_partition.sql b/smart-metastore/src/main/resources/db/changelog/sql/create_file_access_partition.sql new file mode 100644 index 00000000000..28bfbd23790 --- /dev/null +++ b/smart-metastore/src/main/resources/db/changelog/sql/create_file_access_partition.sql @@ -0,0 +1,30 @@ +CREATE OR REPLACE FUNCTION create_file_access_partition(input_date timestamp) + RETURNS INTEGER AS ' + DECLARE + current_date_part DATE; + current_date_part_text TEXT; + partition_table_name TEXT; + first_day_of_month DATE; + last_day_of_month DATE; + result INTEGER; + create_query TEXT; + BEGIN + result := 0; + current_date_part := CAST(DATE_TRUNC(''month'', input_date::date) AS DATE); + current_date_part_text := REGEXP_REPLACE(current_date_part::TEXT, ''-'',''_'',''g''); + partition_table_name := FORMAT(''file_access_%s'', current_date_part_text::TEXT); + IF (TO_REGCLASS(partition_table_name::TEXT) ISNULL) THEN + first_day_of_month := current_date_part; + last_day_of_month := current_date_part + ''1 month''::INTERVAL; + result := 1; + RAISE NOTICE ''table: %'', partition_table_name; + create_query := FORMAT( + ''CREATE TABLE %I PARTITION OF file_access FOR VALUES FROM (extract(epoch from %L::DATE) * 1000) TO (extract(epoch from %L::DATE) * 1000);'', + partition_table_name, first_day_of_month, last_day_of_month); + RAISE NOTICE ''query: %'', create_query; + EXECUTE create_query; + EXECUTE FORMAT(''CREATE INDEX %1$s__access_time ON %1$I (access_time);'', partition_table_name); + END IF; + RETURN result; + END; +' LANGUAGE plpgsql; diff --git a/smart-metastore/src/test/java/org/smartdata/metastore/SqliteTestDaoBase.java b/smart-metastore/src/test/java/org/smartdata/metastore/SqliteTestDaoBase.java deleted file mode 100644 index 8e932e4dbfa..00000000000 --- a/smart-metastore/src/test/java/org/smartdata/metastore/SqliteTestDaoBase.java +++ /dev/null @@ -1,36 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.smartdata.metastore; - -import org.junit.runners.Parameterized.Parameters; -import org.sqlite.JDBC; - -import static org.smartdata.metastore.TestDBUtil.getUniqueSqliteUrl; - -/** - * Limits the number of databases for testing to sqlite for tests - * that don't require a complete set of DBMS. - */ -public abstract class SqliteTestDaoBase extends TestDaoBase { - @Parameters(name = "{0}") - public static Object[] parameters() { - return new Object[][]{ - {DBType.SQLITE, JDBC.class.getName(), getUniqueSqliteUrl()} - }; - } -} diff --git a/smart-metastore/src/test/java/org/smartdata/metastore/TestDBUtil.java b/smart-metastore/src/test/java/org/smartdata/metastore/TestDBUtil.java index d7f7c10a65a..2b6f906340e 100644 --- a/smart-metastore/src/test/java/org/smartdata/metastore/TestDBUtil.java +++ b/smart-metastore/src/test/java/org/smartdata/metastore/TestDBUtil.java @@ -19,10 +19,8 @@ import lombok.extern.slf4j.Slf4j; import org.apache.hadoop.conf.Configuration; -import org.smartdata.metastore.dao.accesscount.AccessCountTableDeque; import org.smartdata.metastore.db.DBHandlersFactory; import org.smartdata.metastore.db.DbSchemaManager; -import org.smartdata.metastore.model.AccessCountTable; import javax.sql.DataSource; @@ -37,7 +35,6 @@ import java.sql.DriverManager; import java.sql.SQLException; import java.util.UUID; -import java.util.concurrent.TimeUnit; import static org.smartdata.metastore.utils.MetaStoreUtils.SQLITE_URL_PREFIX; @@ -60,20 +57,6 @@ public static String getUniqueDBFilePath() { return getUniqueFilePath() + ".db"; } - public static String getUniqueSqliteUrl() { - String dbFile = getUniqueDBFilePath(); - new File(dbFile).deleteOnExit(); - String url = SQLITE_URL_PREFIX + getUniqueDBFilePath(); - log.info("sqlite url: {}", url); - return url; - } - - public static void addAccessCountTableToDeque( - AccessCountTableDeque deque, AccessCountTable table) throws Exception { - deque.addAndNotifyListener(table) - .get(1, TimeUnit.SECONDS); - } - /** * Get an initialized empty Sqlite database file path. * diff --git a/smart-metastore/src/test/java/org/smartdata/metastore/TestDaoBase.java b/smart-metastore/src/test/java/org/smartdata/metastore/TestDaoBase.java index f9363ab84c8..f0b39d24702 100644 --- a/smart-metastore/src/test/java/org/smartdata/metastore/TestDaoBase.java +++ b/smart-metastore/src/test/java/org/smartdata/metastore/TestDaoBase.java @@ -33,7 +33,6 @@ import org.smartdata.metastore.db.metadata.DbMetadataProvider; import org.springframework.jdbc.support.JdbcTransactionManager; import org.springframework.transaction.PlatformTransactionManager; -import org.sqlite.JDBC; import org.testcontainers.jdbc.ContainerDatabaseDriver; import java.io.InputStream; @@ -41,7 +40,6 @@ import static com.alibaba.druid.pool.DruidDataSourceFactory.PROP_DRIVERCLASSNAME; import static com.alibaba.druid.pool.DruidDataSourceFactory.PROP_URL; -import static org.smartdata.metastore.TestDBUtil.getUniqueSqliteUrl; @RunWith(Parameterized.class) public abstract class TestDaoBase { @@ -62,12 +60,9 @@ public abstract class TestDaoBase { @Parameters(name = "{0}") public static Object[] parameters() { - return new Object[][] { - {DBType.SQLITE, JDBC.class.getName(), getUniqueSqliteUrl()}, - {DBType.MYSQL, ContainerDatabaseDriver.class.getName(), - "jdbc:tc:mysql:5.7.34:///ssm_mysql"}, + return new Object[][]{ {DBType.POSTGRES, ContainerDatabaseDriver.class.getName(), - "jdbc:tc:postgresql:9.6.8:///ssm_postgres"}, + "jdbc:tc:postgresql:12.19:///ssm_postgres"}, }; } diff --git a/smart-metastore/src/test/java/org/smartdata/metastore/TestDruid.java b/smart-metastore/src/test/java/org/smartdata/metastore/TestDruid.java index 10357ab1ccb..5696f794f33 100644 --- a/smart-metastore/src/test/java/org/smartdata/metastore/TestDruid.java +++ b/smart-metastore/src/test/java/org/smartdata/metastore/TestDruid.java @@ -21,20 +21,22 @@ import org.junit.Assert; import org.junit.Test; import org.smartdata.metastore.dao.DaoProvider; -import org.smartdata.metastore.dao.sqlite.SqliteDaoProvider; +import org.smartdata.metastore.dao.postgres.PostgresDaoProvider; import org.smartdata.metastore.db.DBHandlersFactory; import org.smartdata.metastore.db.DbSchemaManager; import org.smartdata.metastore.db.metadata.DbMetadataProvider; -import org.smartdata.metastore.db.metadata.SqliteDbMetadataProvider; -import org.smartdata.metastore.utils.MetaStoreUtils; import org.smartdata.model.RuleInfo; import org.smartdata.model.RuleState; import org.springframework.jdbc.support.JdbcTransactionManager; import org.springframework.transaction.PlatformTransactionManager; +import org.testcontainers.jdbc.ContainerDatabaseDriver; import java.io.InputStream; import java.util.Properties; +import static com.alibaba.druid.pool.DruidDataSourceFactory.PROP_DRIVERCLASSNAME; +import static com.alibaba.druid.pool.DruidDataSourceFactory.PROP_URL; + public class TestDruid { @Test @@ -43,22 +45,20 @@ public void test() throws Exception { .getResourceAsStream("druid-template.xml"); Properties p = new Properties(); p.loadFromXML(in); - - String dbFile = TestDBUtil.getUniqueEmptySqliteDBFile(); - String url = MetaStoreUtils.SQLITE_URL_PREFIX + dbFile; - p.setProperty("url", url); - + p.setProperty(PROP_URL, "jdbc:tc:postgresql:12.19:///ssm_postgres"); + p.setProperty(PROP_DRIVERCLASSNAME, ContainerDatabaseDriver.class.getName()); DruidPool druidPool = new DruidPool(p); - DbSchemaManager dbSchemaManager = new DBHandlersFactory() + DBHandlersFactory dbHandlersFactory = new DBHandlersFactory(); + DbSchemaManager dbSchemaManager = dbHandlersFactory .createDbManager(druidPool, new Configuration()); PlatformTransactionManager transactionManager = new JdbcTransactionManager(druidPool.getDataSource()); - DaoProvider daoProvider = new SqliteDaoProvider(druidPool, transactionManager); - DbMetadataProvider dbMetadataProvider = new SqliteDbMetadataProvider( - druidPool.getDataSource()); + DaoProvider daoProvider = new PostgresDaoProvider(druidPool, transactionManager); + DbMetadataProvider dbMetadataProvider = + dbHandlersFactory.createDbMetadataProvider(druidPool, DBType.POSTGRES); MetaStore adapter = new MetaStore( druidPool, dbSchemaManager, daoProvider, dbMetadataProvider, transactionManager); - + dbSchemaManager.initializeDatabase(); String rule = "file : accessCount(10m) > 20 \n\n" + "and length() > 3 | cache"; long submitTime = System.currentTimeMillis(); diff --git a/smart-metastore/src/test/java/org/smartdata/metastore/TestRulesTable.java b/smart-metastore/src/test/java/org/smartdata/metastore/TestRulesTable.java index ebce1b0ad4f..223b6378321 100644 --- a/smart-metastore/src/test/java/org/smartdata/metastore/TestRulesTable.java +++ b/smart-metastore/src/test/java/org/smartdata/metastore/TestRulesTable.java @@ -27,7 +27,7 @@ /** * Tests for table 'rules'. */ -public class TestRulesTable extends SqliteTestDaoBase { +public class TestRulesTable extends TestDaoBase { /** * Insert rules into table and retrieve them back. diff --git a/smart-metastore/src/test/java/org/smartdata/metastore/TestSqliteDB.java b/smart-metastore/src/test/java/org/smartdata/metastore/TestSqliteDB.java deleted file mode 100644 index f68d0994ed3..00000000000 --- a/smart-metastore/src/test/java/org/smartdata/metastore/TestSqliteDB.java +++ /dev/null @@ -1,119 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.smartdata.metastore; - -import org.junit.Assert; -import org.junit.Test; -import org.smartdata.metastore.utils.MetaStoreUtils; - -import java.sql.Connection; -import java.sql.ResultSet; -import java.sql.Statement; -import java.util.ArrayList; -import java.util.List; - -/** Test operations with sqlite database. */ -public class TestSqliteDB extends SqliteTestDaoBase { - @Test - public void testDropTables() throws Exception { - Connection conn = druidPool.getConnection(); - Statement s = conn.createStatement(); - metaStore.dropAllTables(); - for (int i = 0; i < 10; i++) { - metaStore.execute("DROP TABLE IF EXISTS tb_" + i + ";"); - metaStore.execute("CREATE TABLE tb_" + i + " (a INT(11));"); - } - String dbUrl = conn.getMetaData().getURL(); - if (dbUrl.startsWith(MetaStoreUtils.SQLITE_URL_PREFIX)) { - ResultSet rs = s.executeQuery("select tbl_name from sqlite_master;"); - List list = new ArrayList<>(); - while (rs.next()) { - list.add(rs.getString(1)); - } - metaStore.dropAllTables(); - rs = s.executeQuery("select tbl_name from sqlite_master;"); - List list1 = new ArrayList<>(); - while (rs.next()) { - list1.add(rs.getString(1)); - } - Assert.assertEquals(10, list.size() - list1.size()); - } else { - String dbName; - if (dbUrl.contains("?")) { - dbName = dbUrl.substring(dbUrl.indexOf("/", 13) + 1, dbUrl.indexOf("?")); - } else { - dbName = dbUrl.substring(dbUrl.lastIndexOf("/") + 1, dbUrl.length()); - } - ResultSet rs = - s.executeQuery( - "SELECT TABLE_NAME FROM " - + "INFORMATION_SCHEMA.TABLES WHERE TABLE_SCHEMA = '" - + dbName - + "';"); - List list = new ArrayList<>(); - while (rs.next()) { - list.add(rs.getString(1)); - } - metaStore.dropAllTables(); - rs = - s.executeQuery( - "SELECT TABLE_NAME FROM " - + "INFORMATION_SCHEMA.TABLES WHERE TABLE_SCHEMA = '" - + dbName - + "';"); - List list1 = new ArrayList<>(); - while (rs.next()) { - list1.add(rs.getString(1)); - } - Assert.assertEquals(10, list.size() - list1.size()); - } - conn.close(); - } - - @Test - public void testDBBlankStatements() throws Exception { - String[] presqls = - new String[]{ - "INSERT INTO rule (state, rule_text, submit_time, checked_count, " - + "generated_cmdlets) VALUES (0, 'file: every 1s \n" - + " | " - + "accessCount(5s) > 3 | cache', 1494903787619, 0, 0);" - }; - - for (int i = 0; i < presqls.length; i++) { - String sql = presqls[i]; - metaStore.execute(sql); - } - - String[] sqls = - new String[]{ - "DROP TABLE IF EXISTS VIR_ACC_CNT_TAB_1_accessCount_5000;", - "CREATE TABLE VIR_ACC_CNT_TAB_1_accessCount_5000 " - + "AS SELECT * FROM blank_access_count_info;", - "SELECT fid from VIR_ACC_CNT_TAB_1_accessCount_5000;", - "SELECT path FROM file WHERE (fid IN (SELECT fid FROM " - + "VIR_ACC_CNT_TAB_1_accessCount_5000 WHERE ((count > 3))));" - }; - - for (int i = 0; i < sqls.length * 3; i++) { - int idx = i % sqls.length; - String sql = sqls[idx]; - metaStore.execute(sql); - } - } -} diff --git a/smart-metastore/src/test/java/org/smartdata/metastore/accesscount/failover/impl/FailAccessCountFailoverTest.java b/smart-metastore/src/test/java/org/smartdata/metastore/accesscount/failover/impl/FailAccessCountFailoverTest.java new file mode 100644 index 00000000000..cee0428da70 --- /dev/null +++ b/smart-metastore/src/test/java/org/smartdata/metastore/accesscount/failover/impl/FailAccessCountFailoverTest.java @@ -0,0 +1,47 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.smartdata.metastore.accesscount.failover.impl; + +import org.junit.Test; +import org.smartdata.metastore.accesscount.failover.AccessCountContext; +import org.smartdata.metastore.accesscount.failover.Failover; +import org.smartdata.metastore.model.AggregatedAccessCounts; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import static org.junit.Assert.assertThrows; + +public class FailAccessCountFailoverTest { + private Failover accessCountFailover; + + @Test + public void testExecuteWithoutExceedingOverMaxRetries() { + long currentTimeMillis = System.currentTimeMillis(); + accessCountFailover = new Failover() {}; + List accessCounts = new ArrayList<>(Collections.singletonList( + new AggregatedAccessCounts(1, 1, currentTimeMillis))); + AccessCountContext context = new AccessCountContext(accessCounts); + assertThrows(RuntimeException.class, () -> { + accessCountFailover.execute(ctx -> { + throw new RuntimeException("error"); + }, context); + }); + } +} diff --git a/smart-metastore/src/test/java/org/smartdata/metastore/accesscount/failover/impl/RetryAccessCountFailoverTest.java b/smart-metastore/src/test/java/org/smartdata/metastore/accesscount/failover/impl/RetryAccessCountFailoverTest.java new file mode 100644 index 00000000000..e9a39ffb548 --- /dev/null +++ b/smart-metastore/src/test/java/org/smartdata/metastore/accesscount/failover/impl/RetryAccessCountFailoverTest.java @@ -0,0 +1,70 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.smartdata.metastore.accesscount.failover.impl; + +import org.junit.Test; +import org.smartdata.metastore.accesscount.failover.AccessCountContext; +import org.smartdata.metastore.accesscount.failover.Failover; +import org.smartdata.metastore.model.AggregatedAccessCounts; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; + +public class RetryAccessCountFailoverTest { + private Failover accessCountFailover; + + @Test + public void testExecuteWithoutExceedingOverMaxRetries() { + int maxRetries = 3; + int attemptsCount = 0; + long currentTimeMillis = System.currentTimeMillis(); + accessCountFailover = new RetryAccessCountFailover(maxRetries); + List accessCounts = new ArrayList<>(Collections.singletonList( + new AggregatedAccessCounts(1, 1, currentTimeMillis))); + AccessCountContext context = new AccessCountContext(accessCounts); + for (int i = 1; i <= maxRetries; i++) { + accessCountFailover.execute(ctx -> { + throw new RuntimeException("error"); + }, context); + attemptsCount = i; + } + assertEquals(maxRetries, attemptsCount); + } + + @Test + public void testExecuteWithExceedingOverMaxRetries() { + RuntimeException error = new RuntimeException("error"); + int maxRetries = 3; + long currentTimeMillis = System.currentTimeMillis(); + accessCountFailover = new RetryAccessCountFailover(maxRetries); + List accessCounts = new ArrayList<>(Collections.singletonList( + new AggregatedAccessCounts(1, 1, currentTimeMillis))); + AccessCountContext context = new AccessCountContext(accessCounts); + assertThrows(RuntimeException.class, () -> { + for (int i = 1; i <= maxRetries + 1; i++) { + accessCountFailover.execute(ctx -> { + throw error; + }, context); + } + }); + } +} diff --git a/smart-metastore/src/test/java/org/smartdata/metastore/dao/TestCacheFileDao.java b/smart-metastore/src/test/java/org/smartdata/metastore/dao/TestCacheFileDao.java index 6615aedafec..1cabcd4e314 100644 --- a/smart-metastore/src/test/java/org/smartdata/metastore/dao/TestCacheFileDao.java +++ b/smart-metastore/src/test/java/org/smartdata/metastore/dao/TestCacheFileDao.java @@ -56,8 +56,8 @@ public void testUpdateCachedFiles() { "testPath2", 2000L, 3000L, 200); cacheFileDao.insert(second); List accessCounts = new ArrayList<>(); - accessCounts.add(new AggregatedAccessCounts(80L, 2, 4000L)); - accessCounts.add(new AggregatedAccessCounts(90L, 2, 5000L)); + accessCounts.add(new AggregatedAccessCounts(80L, 2, 4000L)); + accessCounts.add(new AggregatedAccessCounts(90L, 2, 5000L)); accessCounts.add(new AggregatedAccessCounts(100L, 2, 9000L)); // Sync status diff --git a/smart-metastore/src/test/java/org/smartdata/metastore/dao/accesscount/TestAccessCountTableEvictor.java b/smart-metastore/src/test/java/org/smartdata/metastore/dao/accesscount/TestAccessCountTableEvictor.java deleted file mode 100644 index efdb4f71bf9..00000000000 --- a/smart-metastore/src/test/java/org/smartdata/metastore/dao/accesscount/TestAccessCountTableEvictor.java +++ /dev/null @@ -1,129 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.smartdata.metastore.dao.accesscount; - -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.smartdata.metastore.model.AccessCountTable; - -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; - -import static org.smartdata.metastore.TestDBUtil.addAccessCountTableToDeque; -import static org.smartdata.metastore.utils.Constants.ONE_SECOND_IN_MILLIS; - -public class TestAccessCountTableEvictor { - private ExecutorService executorService; - private AccessCountTableHandler tableHandler; - - @Before - public void setUp() { - tableHandler = new NoOpAccessCountTableHandler(); - executorService = Executors.newSingleThreadExecutor(); - } - - @After - public void shutdown() { - executorService.shutdown(); - } - - @Test - public void testCountEvictor() { - CountTableEvictor countEvictor = new CountTableEvictor(tableHandler, 2); - AccessCountTableDeque tableDeque = new AccessCountTableDeque(countEvictor); - - tableDeque.add(new AccessCountTable(0L, 1L)); - countEvictor.evictTables(tableDeque, 0L); - Assert.assertEquals(1, tableDeque.size()); - - tableDeque.add(new AccessCountTable(1L, 2L)); - countEvictor.evictTables(tableDeque, 0L); - Assert.assertEquals(2, tableDeque.size()); - - tableDeque.add(new AccessCountTable(3L, 4L)); - countEvictor.evictTables(tableDeque, 0L); - Assert.assertEquals(3, tableDeque.size()); - - AccessCountTable firstExpectedTable = new AccessCountTable(4L, 59 * ONE_SECOND_IN_MILLIS); - tableDeque.add(firstExpectedTable); - countEvictor.evictTables(tableDeque, 0L); - Assert.assertEquals(4, tableDeque.size()); - - AccessCountTable secondExpectedTable = new AccessCountTable( - 59 * ONE_SECOND_IN_MILLIS, 60 * ONE_SECOND_IN_MILLIS); - tableDeque.add(secondExpectedTable); - countEvictor.evictTables(tableDeque, ONE_SECOND_IN_MILLIS); - Assert.assertEquals(2, tableDeque.size()); - - Assert.assertTrue(tableDeque.contains(firstExpectedTable)); - Assert.assertTrue(tableDeque.contains(secondExpectedTable)); - } - - @Test - public void testDontEvictIfNotAggregatedYet() throws Exception { - AccessCountTableDeque secondDeque = buildSecondDeque(1); - - addAccessCountTableToDeque(secondDeque, new AccessCountTable(0L, 1L)); - Assert.assertEquals(1, secondDeque.size()); - - addAccessCountTableToDeque(secondDeque, new AccessCountTable(1L, 2L)); - Assert.assertEquals(2, secondDeque.size()); - - addAccessCountTableToDeque(secondDeque, new AccessCountTable(2L, 3L)); - Assert.assertEquals(3, secondDeque.size()); - - AccessCountTable lastFirstMinuteTable = new AccessCountTable( - 59 * ONE_SECOND_IN_MILLIS, 60 * ONE_SECOND_IN_MILLIS); - addAccessCountTableToDeque(secondDeque, lastFirstMinuteTable); - - Assert.assertEquals(1, secondDeque.size()); - Assert.assertTrue(secondDeque.contains(lastFirstMinuteTable)); - } - - @Test - public void testDontEvictDuringAggregationIfThresholdNotMet() throws Exception { - AccessCountTableDeque secondDeque = buildSecondDeque(10); - - addAccessCountTableToDeque(secondDeque, new AccessCountTable(0L, 1L)); - Assert.assertEquals(1, secondDeque.size()); - - addAccessCountTableToDeque(secondDeque, new AccessCountTable(1L, 2L)); - Assert.assertEquals(2, secondDeque.size()); - - addAccessCountTableToDeque(secondDeque, new AccessCountTable(2L, 3L)); - Assert.assertEquals(3, secondDeque.size()); - - AccessCountTable lastFirstMinuteTable = new AccessCountTable( - 59 * ONE_SECOND_IN_MILLIS, 60 * ONE_SECOND_IN_MILLIS); - addAccessCountTableToDeque(secondDeque, lastFirstMinuteTable); - - Assert.assertEquals(4, secondDeque.size()); - } - - private AccessCountTableDeque buildSecondDeque(int evictThreshold) { - AccessCountTableDeque minuteDeque = new AccessCountTableDeque( - new CountTableEvictor(tableHandler, 999)); - TableAddOpListener minuteTableListener = - TableAddOpListener.perMinute(minuteDeque, tableHandler, executorService); - - AccessCountTableEvictor secondEvictor = new CountTableEvictor(tableHandler, evictThreshold); - return new AccessCountTableDeque(secondEvictor, minuteTableListener); - } -} diff --git a/smart-metastore/src/test/java/org/smartdata/metastore/dao/accesscount/TestAccessCountTableManager.java b/smart-metastore/src/test/java/org/smartdata/metastore/dao/accesscount/TestAccessCountTableManager.java deleted file mode 100644 index 9682291660c..00000000000 --- a/smart-metastore/src/test/java/org/smartdata/metastore/dao/accesscount/TestAccessCountTableManager.java +++ /dev/null @@ -1,1161 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.smartdata.metastore.dao.accesscount; - -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.smartdata.conf.SmartConf; -import org.smartdata.metastore.MetaStoreException; -import org.smartdata.metastore.dao.Searchable; -import org.smartdata.metastore.dao.TestSearchableDao; -import org.smartdata.metastore.model.AccessCountTable; -import org.smartdata.metastore.model.AggregatedAccessCounts; -import org.smartdata.metastore.queries.PageRequest; -import org.smartdata.metastore.queries.sort.FileAccessInfoSortField; -import org.smartdata.metastore.queries.sort.Sorting; -import org.smartdata.metastore.utils.Constants; -import org.smartdata.metastore.utils.TimeGranularity; -import org.smartdata.metrics.FileAccessEvent; -import org.smartdata.model.FileAccessInfo; -import org.smartdata.model.FileInfo; -import org.smartdata.model.TimeInterval; -import org.smartdata.model.request.FileAccessInfoSearchRequest; - -import java.time.Instant; -import java.util.Arrays; -import java.util.Collections; -import java.util.Comparator; -import java.util.Deque; -import java.util.List; -import java.util.NoSuchElementException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.stream.IntStream; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.smartdata.conf.SmartConfKeys.SMART_ACCESS_COUNT_AGGREGATION_INTERVAL_MS_DEFAULT; -import static org.smartdata.metastore.utils.Constants.ONE_DAY_IN_MILLIS; -import static org.smartdata.metastore.utils.Constants.ONE_HOUR_IN_MILLIS; -import static org.smartdata.metastore.utils.Constants.ONE_MINUTE_IN_MILLIS; -import static org.smartdata.metastore.utils.Constants.ONE_SECOND_IN_MILLIS; - - -public class TestAccessCountTableManager - extends TestSearchableDao { - - private static final List TEST_FILES = Arrays.asList( - "/file0", - "/file1", - "/file2", - "/file3", - "/file4", - "/file5" - ); - - private AccessCountTableManager accessCountTableManager; - private InMemoryAccessCountTableManager inMemoryTableManager; - private ExecutorService executorService; - - @Before - public void setup() { - executorService = Executors.newFixedThreadPool(4); - accessCountTableManager = new AccessCountTableManager( - metaStore, executorService, new SmartConf() - ); - inMemoryTableManager = accessCountTableManager.getInMemoryTableManager(); - } - - @After - public void shutDown() { - executorService.shutdownNow(); - } - - @Override - protected Searchable searchable() { - return accessCountTableManager; - } - - @Override - protected Long getIdentifier(FileAccessInfo fileAccessInfo) { - return fileAccessInfo.getFid(); - } - - @Override - protected FileAccessInfoSortField defaultSortField() { - return FileAccessInfoSortField.FID; - } - - @Test - public void testSearchByFilePath() throws MetaStoreException { - createTestFiles(); - - List accessEvents = Arrays.asList( - new FileAccessEvent("/file1", 1), - new FileAccessEvent("/file2", 3), - new FileAccessEvent("/file4", 4), - new FileAccessEvent("/file2", 5), - new FileAccessEvent("", 5001) - ); - accessCountTableManager.getAccessEventAggregator().aggregate(accessEvents); - - FileAccessInfoSearchRequest searchRequest = FileAccessInfoSearchRequest.builder() - .pathLike("/file") - .build(); - - testSearch(searchRequest, 1L, 2L, 4L); - - searchRequest = FileAccessInfoSearchRequest.builder() - .pathLike("/file2") - .build(); - - testSearch(searchRequest, 2L); - - searchRequest = FileAccessInfoSearchRequest.builder() - .pathLike("another_path") - .build(); - - testSearch(searchRequest); - } - - @Test - public void testCreateTable() throws Exception { - long firstDayEnd = 24 * 60 * 60 * 1000L; - AccessCountTable accessCountTable = - new AccessCountTable(firstDayEnd - 5 * 1000, firstDayEnd); - accessCountTableManager.createTable(accessCountTable); - - Thread.sleep(5000); - - Deque second = - inMemoryTableManager.getTablesOfGranularity(TimeGranularity.SECOND); - - assertEquals(1, second.size()); - assertEquals(accessCountTable, second.peek()); - - Deque minute = - inMemoryTableManager.getTablesOfGranularity(TimeGranularity.MINUTE); - AccessCountTable minuteTable = - new AccessCountTable(firstDayEnd - 60 * 1000, firstDayEnd); - assertEquals(1, minute.size()); - assertEquals(minuteTable, minute.peek()); - - Deque hour = - inMemoryTableManager.getTablesOfGranularity(TimeGranularity.HOUR); - AccessCountTable hourTable = - new AccessCountTable(firstDayEnd - 60 * 60 * 1000, firstDayEnd); - assertEquals(1, hour.size()); - assertEquals(hourTable, hour.peek()); - - Deque day = - inMemoryTableManager.getTablesOfGranularity(TimeGranularity.DAY); - AccessCountTable dayTable = new AccessCountTable(0, firstDayEnd); - assertEquals(1, day.size()); - assertEquals(dayTable, day.peek()); - } - - @Test - public void testGetTables() throws MetaStoreException { - AccessCountTable firstDay = new AccessCountTable(0L, Constants.ONE_DAY_IN_MILLIS); - - AccessCountTable firstHour = - new AccessCountTable(23 * Constants.ONE_HOUR_IN_MILLIS, - 24 * Constants.ONE_HOUR_IN_MILLIS); - AccessCountTable secondHour = - new AccessCountTable(24 * Constants.ONE_HOUR_IN_MILLIS, - 25 * Constants.ONE_HOUR_IN_MILLIS); - - int numMins = 25 * 60; - AccessCountTable firstMin = - new AccessCountTable( - (numMins - 1) * ONE_MINUTE_IN_MILLIS, - numMins * ONE_MINUTE_IN_MILLIS); - AccessCountTable secondMin = - new AccessCountTable( - numMins * ONE_MINUTE_IN_MILLIS, - (numMins + 1) * ONE_MINUTE_IN_MILLIS); - - int numSeconds = (25 * 60 + 1) * 60; - AccessCountTable firstFiveSeconds = - new AccessCountTable( - (numSeconds - 5) * Constants.ONE_SECOND_IN_MILLIS, - numSeconds * Constants.ONE_SECOND_IN_MILLIS); - AccessCountTable secondFiveSeconds = - new AccessCountTable( - numSeconds * Constants.ONE_SECOND_IN_MILLIS, - (numSeconds + 5) * Constants.ONE_SECOND_IN_MILLIS); - - accessCountTableManager.getDbTableManager().createTable(firstDay); - - List tablesToRecover = Arrays.asList( - firstDay, firstHour, secondHour, firstMin, secondMin, firstFiveSeconds, secondFiveSeconds); - inMemoryTableManager.recoverTables(tablesToRecover); - /* - |------------------------interval----------------------------| - |-s-|-s-| - |---m---|---m---| - |-------h-------|-------h------| - |---------------d-----------------| - */ - List firstResult = - accessCountTableManager.getTablesForLast( - (numSeconds + 5) * Constants.ONE_SECOND_IN_MILLIS); - assertEquals(4, firstResult.size()); - assertEquals(firstDay, firstResult.get(0)); - assertEquals(secondHour, firstResult.get(1)); - assertEquals(secondMin, firstResult.get(2)); - assertEquals(secondFiveSeconds, firstResult.get(3)); - - /* - |--------------------interval----------------------------| - |-s-|-s-| - |---m---|---m---| - |-------h-------|-------h------| - |---------------d-----------------| - */ - List secondResult = - accessCountTableManager.getTablesForLast( - numSeconds * Constants.ONE_SECOND_IN_MILLIS); - assertEquals(5, secondResult.size()); - - AccessCountTable firstTable = secondResult.get(0); - assertTrue(firstTable.getStartTime() == 5 * Constants.ONE_SECOND_IN_MILLIS - && firstTable.getEndTime() == 23 * Constants.ONE_HOUR_IN_MILLIS); - - /* - |--------------interval--------------------| - |-s-|-s-| - |---m---|---m---| - |-------h-------|-------h------| - |---------------d-----------------| - */ - List thirdResult = - accessCountTableManager.getTablesForLast( - secondFiveSeconds.getEndTime() - 23 * Constants.ONE_HOUR_IN_MILLIS); - assertEquals(4, thirdResult.size()); - assertEquals(firstHour, thirdResult.get(0)); - - /* - |--------interval----------| - |-s-|-s-| - |---m---|---m---| - |-------h-------|-------h------| - |---------------d-----------------| - */ - List fourthResult = - accessCountTableManager.getTablesForLast( - secondFiveSeconds.getEndTime() - 24 * Constants.ONE_HOUR_IN_MILLIS); - assertEquals(3, fourthResult.size()); - assertEquals(secondHour, fourthResult.get(0)); - } - - @Test - public void testGetTablesCornerCase() throws MetaStoreException { - AccessCountTable firstFiveSeconds = - new AccessCountTable(0L, 5 * Constants.ONE_SECOND_IN_MILLIS); - AccessCountTable secondFiveSeconds = - new AccessCountTable(5 * Constants.ONE_SECOND_IN_MILLIS, - 10 * Constants.ONE_SECOND_IN_MILLIS); - - List tablesToRecover = - Arrays.asList(firstFiveSeconds, secondFiveSeconds); - inMemoryTableManager.recoverTables(tablesToRecover); - - List result = accessCountTableManager.getTablesForLast( - 2 * ONE_MINUTE_IN_MILLIS); - assertEquals(2, result.size()); - assertEquals(firstFiveSeconds, result.get(0)); - assertEquals(secondFiveSeconds, result.get(1)); - } - - @Test - public void testGetTablesCornerCaseMinutes() throws MetaStoreException { - /* - |--interval-| - |-s-|-s-|-s-| - |----m-----| - */ - AccessCountTable firstMinute = - new AccessCountTable(0L, ONE_MINUTE_IN_MILLIS); - AccessCountTable firstFiveSeconds = - new AccessCountTable( - 55 * Constants.ONE_SECOND_IN_MILLIS, 60 * Constants.ONE_SECOND_IN_MILLIS); - AccessCountTable secondFiveSeconds = - new AccessCountTable(60 * Constants.ONE_SECOND_IN_MILLIS, - 65 * Constants.ONE_SECOND_IN_MILLIS); - AccessCountTable thirdFiveSeconds = - new AccessCountTable(110 * Constants.ONE_SECOND_IN_MILLIS, - 115 * Constants.ONE_SECOND_IN_MILLIS); - - List tablesToRecover = Arrays.asList( - firstMinute, firstFiveSeconds, secondFiveSeconds, thirdFiveSeconds); - - inMemoryTableManager.recoverTables(tablesToRecover); - - List result = accessCountTableManager.getTablesForLast( - ONE_MINUTE_IN_MILLIS); - - assertEquals(3, result.size()); - assertEquals(firstFiveSeconds, result.get(0)); - Assert.assertFalse(result.get(0).isEphemeral()); - assertEquals(secondFiveSeconds, result.get(1)); - assertEquals(thirdFiveSeconds, result.get(2)); - } - - @Test - public void testGetAllHotFiles() throws MetaStoreException { - createTestFiles(); - /* - create access count tables for seconds, day and hours intervals - |-------------------------------------interval-----------------------------------------------| - |-s-| |-s-| |-s-| |-s-| |-s-| |-s-| |-s-| |-s-| |-s-| |-s-| |-s-| |-s-| - |---h---|---h----| |---h---|---h---| - |-----------------------d-----------------| - */ - DbAccessCountTableManager tableManager = accessCountTableManager.getDbTableManager(); - AccessCountTable t1 = new AccessCountTable(0, 5 * ONE_SECOND_IN_MILLIS); - AccessCountTable t2 = new AccessCountTable(2 * ONE_MINUTE_IN_MILLIS, - 2 * ONE_MINUTE_IN_MILLIS + 5 * ONE_SECOND_IN_MILLIS); - AccessCountTable t3 = new AccessCountTable(4 * ONE_MINUTE_IN_MILLIS + 10 * ONE_SECOND_IN_MILLIS, - 4 * ONE_MINUTE_IN_MILLIS + 15 * ONE_SECOND_IN_MILLIS); - AccessCountTable t4 = new AccessCountTable(ONE_HOUR_IN_MILLIS + 5 * ONE_MINUTE_IN_MILLIS, - ONE_HOUR_IN_MILLIS + 5 * ONE_MINUTE_IN_MILLIS + 5 * ONE_SECOND_IN_MILLIS); - AccessCountTable t5 = new AccessCountTable(ONE_HOUR_IN_MILLIS + 9 * ONE_MINUTE_IN_MILLIS, - ONE_HOUR_IN_MILLIS + 9 * ONE_MINUTE_IN_MILLIS + 5 * ONE_SECOND_IN_MILLIS); - AccessCountTable t6 = new AccessCountTable(8 * ONE_HOUR_IN_MILLIS, - 8 * ONE_HOUR_IN_MILLIS + 5 * ONE_SECOND_IN_MILLIS); - AccessCountTable t7 = new AccessCountTable(ONE_DAY_IN_MILLIS + 12 * ONE_HOUR_IN_MILLIS, - ONE_DAY_IN_MILLIS + 12 * ONE_HOUR_IN_MILLIS + 5 * ONE_SECOND_IN_MILLIS); - AccessCountTable t8 = new AccessCountTable( - ONE_DAY_IN_MILLIS + 13 * ONE_HOUR_IN_MILLIS + 10 * ONE_SECOND_IN_MILLIS, - ONE_DAY_IN_MILLIS + 13 * ONE_HOUR_IN_MILLIS + 15 * ONE_SECOND_IN_MILLIS); - AccessCountTable t9 = new AccessCountTable(ONE_DAY_IN_MILLIS + 17 * ONE_HOUR_IN_MILLIS, - ONE_DAY_IN_MILLIS + 17 * ONE_HOUR_IN_MILLIS + 5 * ONE_SECOND_IN_MILLIS); - AccessCountTable t10 = new AccessCountTable( - ONE_DAY_IN_MILLIS + 18 * ONE_HOUR_IN_MILLIS + 10 * ONE_SECOND_IN_MILLIS, - ONE_DAY_IN_MILLIS + 18 * ONE_HOUR_IN_MILLIS + 15 * ONE_SECOND_IN_MILLIS); - AccessCountTable t11 = - new AccessCountTable(ONE_DAY_IN_MILLIS + 23 * ONE_HOUR_IN_MILLIS + 59 * ONE_MINUTE_IN_MILLIS - + 58 * ONE_SECOND_IN_MILLIS, 2 * ONE_DAY_IN_MILLIS); - AccessCountTable t12 = new AccessCountTable(0, ONE_DAY_IN_MILLIS); - AccessCountTable t13 = new AccessCountTable(ONE_DAY_IN_MILLIS + 12 * ONE_HOUR_IN_MILLIS, - ONE_DAY_IN_MILLIS + 13 * ONE_HOUR_IN_MILLIS); - AccessCountTable t14 = new AccessCountTable(ONE_DAY_IN_MILLIS + 13 * ONE_HOUR_IN_MILLIS, - ONE_DAY_IN_MILLIS + 14 * ONE_HOUR_IN_MILLIS); - AccessCountTable t15 = new AccessCountTable(ONE_DAY_IN_MILLIS + 17 * ONE_HOUR_IN_MILLIS, - ONE_DAY_IN_MILLIS + 18 * ONE_HOUR_IN_MILLIS); - AccessCountTable t16 = new AccessCountTable(ONE_DAY_IN_MILLIS + 18 * ONE_HOUR_IN_MILLIS, - ONE_DAY_IN_MILLIS + 19 * ONE_HOUR_IN_MILLIS); - - List tables = - Arrays.asList(t1, t2, t3, t4, t5, t6, t7, t8, t9, t10, t11, t12, t13, t14, t15, t16); - - for (AccessCountTable t : tables) { - tableManager.createTable(t); - } - tableManager.handleAggregatedEvents(t1, - Arrays.asList(new AggregatedAccessCounts(0, 1, 0), - new AggregatedAccessCounts(1, 1, 1))); - tableManager.handleAggregatedEvents(t2, - Arrays.asList(new AggregatedAccessCounts(2, 1, 2 * ONE_MINUTE_IN_MILLIS + 1))); - tableManager.handleAggregatedEvents(t3, Arrays.asList( - new AggregatedAccessCounts(1, 1, 4 * ONE_MINUTE_IN_MILLIS + 10 * ONE_SECOND_IN_MILLIS))); - tableManager.handleAggregatedEvents(t4, Arrays.asList( - new AggregatedAccessCounts(2, 1, ONE_HOUR_IN_MILLIS + 5 * ONE_MINUTE_IN_MILLIS))); - tableManager.handleAggregatedEvents(t5, Arrays.asList( - new AggregatedAccessCounts(3, 1, ONE_HOUR_IN_MILLIS + 9 * ONE_MINUTE_IN_MILLIS))); - tableManager.handleAggregatedEvents(t6, - Arrays.asList(new AggregatedAccessCounts(3, 1, 8 * ONE_HOUR_IN_MILLIS))); - tableManager.handleAggregatedEvents(t7, Arrays.asList( - new AggregatedAccessCounts(3, 1, ONE_DAY_IN_MILLIS + 12 * ONE_HOUR_IN_MILLIS))); - tableManager.handleAggregatedEvents(t7, Arrays.asList( - new AggregatedAccessCounts(2, 1, ONE_DAY_IN_MILLIS + 12 * ONE_HOUR_IN_MILLIS + 1))); - tableManager.handleAggregatedEvents(t8, Arrays.asList(new AggregatedAccessCounts(1, 1, - ONE_DAY_IN_MILLIS + 13 * ONE_HOUR_IN_MILLIS + 10 * ONE_SECOND_IN_MILLIS))); - tableManager.handleAggregatedEvents(t9, Arrays.asList( - new AggregatedAccessCounts(4, 1, ONE_DAY_IN_MILLIS + 17 * ONE_HOUR_IN_MILLIS))); - tableManager.handleAggregatedEvents(t10, Arrays.asList( - new AggregatedAccessCounts(5, 1, ONE_DAY_IN_MILLIS + 18 * ONE_HOUR_IN_MILLIS + 10))); - tableManager.handleAggregatedEvents(t11, Arrays.asList( - new AggregatedAccessCounts(3, 1, - ONE_DAY_IN_MILLIS + 23 * ONE_HOUR_IN_MILLIS + 59 * ONE_MINUTE_IN_MILLIS - + 58 * ONE_SECOND_IN_MILLIS))); - tableManager.handleAggregatedEvents(t11, Arrays.asList( - new AggregatedAccessCounts(3, 1, - ONE_DAY_IN_MILLIS + 23 * ONE_HOUR_IN_MILLIS + 59 * ONE_MINUTE_IN_MILLIS - + 59 * ONE_SECOND_IN_MILLIS))); - tableManager.handleAggregatedEvents(t12, - Arrays.asList( - new AggregatedAccessCounts(0, 1, 0), - new AggregatedAccessCounts(1, 2, 4 * ONE_MINUTE_IN_MILLIS + 10 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(2, 2, ONE_HOUR_IN_MILLIS + 5 * ONE_MINUTE_IN_MILLIS), - new AggregatedAccessCounts(3, 2, 8 * ONE_HOUR_IN_MILLIS) - )); - tableManager.handleAggregatedEvents(t13, Arrays.asList( - new AggregatedAccessCounts(3, 1, ONE_DAY_IN_MILLIS + 12 * ONE_HOUR_IN_MILLIS), - new AggregatedAccessCounts(2, 1, ONE_DAY_IN_MILLIS + 12 * ONE_HOUR_IN_MILLIS + 1) - )); - tableManager.handleAggregatedEvents(t14, Arrays.asList(new AggregatedAccessCounts(1, 1, - ONE_DAY_IN_MILLIS + 13 * ONE_HOUR_IN_MILLIS + 10 * ONE_SECOND_IN_MILLIS))); - tableManager.handleAggregatedEvents(t15, Arrays.asList( - new AggregatedAccessCounts(4, 1, ONE_DAY_IN_MILLIS + 17 * ONE_HOUR_IN_MILLIS))); - tableManager.handleAggregatedEvents(t16, Arrays.asList( - new AggregatedAccessCounts(5, 1, ONE_DAY_IN_MILLIS + 18 * ONE_HOUR_IN_MILLIS + 10))); - - inMemoryTableManager.recoverTables(tables); - - List expectedFiles = Arrays.asList( - new FileAccessInfo(0, TEST_FILES.get(0), 1, 0), - new FileAccessInfo(1, TEST_FILES.get(1), 3, - ONE_DAY_IN_MILLIS + 13 * ONE_HOUR_IN_MILLIS + 10 * ONE_SECOND_IN_MILLIS), - new FileAccessInfo(2, TEST_FILES.get(2), 3, - ONE_DAY_IN_MILLIS + 12 * ONE_HOUR_IN_MILLIS + 1), - new FileAccessInfo(3, TEST_FILES.get(3), 5, - ONE_DAY_IN_MILLIS + 23 * ONE_HOUR_IN_MILLIS + 59 * ONE_MINUTE_IN_MILLIS - + 59 * ONE_SECOND_IN_MILLIS), - new FileAccessInfo(4, TEST_FILES.get(4), 1, ONE_DAY_IN_MILLIS - + 17 * ONE_HOUR_IN_MILLIS), - new FileAccessInfo(5, TEST_FILES.get(5), 1, - ONE_DAY_IN_MILLIS + 18 * ONE_HOUR_IN_MILLIS + 10) - ); - - Long latestAccessedTime = expectedFiles.stream() - .min(Comparator.comparing(FileAccessInfo::getLastAccessedTime)) - .map(FileAccessInfo::getLastAccessedTime) - .orElseThrow(NoSuchElementException::new); - - List hotFiles = - accessCountTableManager.search(FileAccessInfoSearchRequest.builder() - .lastAccessedTime(TimeInterval.builder() - .from(Instant.ofEpochMilli(latestAccessedTime)) - .build()) - .build(), - PageRequest.builder() - .addSorting(FileAccessInfoSortField.FID, Sorting.Order.ASC) - .build()).getItems(); - - assertEquals(expectedFiles, hotFiles); - } - - @Test - public void testGetAllHotFilesDuringLastSeconds() throws MetaStoreException { - createTestFiles(); - submitAccessEvents(); - - List expectedFiles = Arrays.asList( - new FileAccessInfo(3, TEST_FILES.get(3), 2, - ONE_DAY_IN_MILLIS + 23 * ONE_HOUR_IN_MILLIS + 59 * ONE_MINUTE_IN_MILLIS - + 59 * ONE_SECOND_IN_MILLIS) - ); - Long latestAccessedTime = expectedFiles.stream() - .max(Comparator.comparing(FileAccessInfo::getLastAccessedTime)) - .map(FileAccessInfo::getLastAccessedTime) - .orElseThrow(NoSuchElementException::new); - - List hotFiles = - accessCountTableManager.search(FileAccessInfoSearchRequest.builder() - .lastAccessedTime(TimeInterval.builder() - .from(Instant.ofEpochMilli(latestAccessedTime - 5 * ONE_SECOND_IN_MILLIS)) - .to(Instant.ofEpochMilli(latestAccessedTime)) - .build()) - .build()); - - assertEquals(expectedFiles, hotFiles); - } - - @Test - public void testGetHotFilesDuringLastHour() throws MetaStoreException { - createTestFiles(); - submitAccessEvents(); - - List expectedFiles = Collections.singletonList( - new FileAccessInfo(3, TEST_FILES.get(3), 2, - ONE_DAY_IN_MILLIS + 23 * ONE_HOUR_IN_MILLIS + 59 * ONE_MINUTE_IN_MILLIS - + 59 * ONE_SECOND_IN_MILLIS) - ); - - Long latestAccessedTime = expectedFiles.stream() - .max(Comparator.comparing(FileAccessInfo::getLastAccessedTime)) - .map(FileAccessInfo::getLastAccessedTime) - .orElseThrow(NoSuchElementException::new); - - List hotFiles = - accessCountTableManager.search(FileAccessInfoSearchRequest.builder() - .lastAccessedTime(TimeInterval.builder() - .from(Instant.ofEpochMilli(latestAccessedTime - ONE_HOUR_IN_MILLIS)) - .to(Instant.ofEpochMilli(latestAccessedTime)) - .build()) - .build()); - - assertEquals(expectedFiles, hotFiles); - } - - @Test - public void testGetHotFilesFromPartialTable() throws MetaStoreException { - createTestFiles(); - submitAccessEvents(); - - List expectedFiles = Collections.singletonList( - new FileAccessInfo(3, TEST_FILES.get(3), 1, - ONE_DAY_IN_MILLIS + 23 * ONE_HOUR_IN_MILLIS + 59 * ONE_MINUTE_IN_MILLIS - + 59 * ONE_SECOND_IN_MILLIS) - ); - - Long latestAccessedTime = expectedFiles.stream() - .max(Comparator.comparing(FileAccessInfo::getLastAccessedTime)) - .map(FileAccessInfo::getLastAccessedTime) - .orElseThrow(NoSuchElementException::new); - - List hotFiles = - accessCountTableManager.search(FileAccessInfoSearchRequest.builder() - .lastAccessedTime(TimeInterval.builder() - .from(Instant.ofEpochMilli( - (latestAccessedTime - SMART_ACCESS_COUNT_AGGREGATION_INTERVAL_MS_DEFAULT / 2))) - .to(Instant.ofEpochMilli(latestAccessedTime)) - .build()) - .build()); - - assertEquals(expectedFiles, hotFiles); - } - - @Test - public void testGetHotFilesWhenIntervalBiggerThanExistedTables() - throws MetaStoreException { - /* - |---------interval-------------| - |-s-| |-s-| |-s-| - |--------min--------------| - */ - createTestFiles(); - DbAccessCountTableManager tableManager = accessCountTableManager.getDbTableManager(); - AccessCountTable t1 = new AccessCountTable(0, 5 * ONE_SECOND_IN_MILLIS); - AccessCountTable t2 = new AccessCountTable(ONE_MINUTE_IN_MILLIS + 5 * ONE_SECOND_IN_MILLIS, - ONE_MINUTE_IN_MILLIS + 10 * ONE_SECOND_IN_MILLIS); - AccessCountTable t3 = new AccessCountTable(ONE_MINUTE_IN_MILLIS + 20 * ONE_SECOND_IN_MILLIS, - ONE_MINUTE_IN_MILLIS + 25 * ONE_SECOND_IN_MILLIS); - AccessCountTable t4 = new AccessCountTable(ONE_MINUTE_IN_MILLIS + 35 * ONE_SECOND_IN_MILLIS, - ONE_MINUTE_IN_MILLIS + 40 * ONE_SECOND_IN_MILLIS); - AccessCountTable t5 = new AccessCountTable(ONE_MINUTE_IN_MILLIS + 45 * ONE_SECOND_IN_MILLIS, - ONE_MINUTE_IN_MILLIS + 50 * ONE_SECOND_IN_MILLIS); - AccessCountTable t6 = new AccessCountTable(ONE_MINUTE_IN_MILLIS, 2 * ONE_MINUTE_IN_MILLIS); - List tables = Arrays.asList(t1, t2, t3, t4, t5, t6); - tables.forEach(t -> { - try { - tableManager.createTable(t); - } catch (MetaStoreException e) { - throw new RuntimeException(e); - } - }); - tableManager.handleAggregatedEvents(t1, Arrays.asList(new AggregatedAccessCounts(3, 1, 0))); - tableManager.handleAggregatedEvents(t2, Arrays.asList( - new AggregatedAccessCounts(3, 1, ONE_MINUTE_IN_MILLIS + 10 * ONE_SECOND_IN_MILLIS))); - tableManager.handleAggregatedEvents(t3, Arrays.asList( - new AggregatedAccessCounts(2, 1, ONE_MINUTE_IN_MILLIS + 21 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(4, 1, ONE_MINUTE_IN_MILLIS + 21 * ONE_SECOND_IN_MILLIS))); - tableManager.handleAggregatedEvents(t4, Arrays.asList( - new AggregatedAccessCounts(5, 1, ONE_MINUTE_IN_MILLIS + 38 * ONE_SECOND_IN_MILLIS))); - tableManager.handleAggregatedEvents(t5, Arrays.asList( - new AggregatedAccessCounts(1, 2, ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(3, 2, ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(2, 1, ONE_MINUTE_IN_MILLIS + 46 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(4, 1, ONE_MINUTE_IN_MILLIS + 46 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(5, 1, ONE_MINUTE_IN_MILLIS + 46 * ONE_SECOND_IN_MILLIS) - )); - tableManager.handleAggregatedEvents(t6, Arrays.asList( - new AggregatedAccessCounts(1, 4, ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(2, 3, ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(3, 3, ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(4, 2, ONE_MINUTE_IN_MILLIS + 46 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(5, 2, ONE_MINUTE_IN_MILLIS + 46 * ONE_SECOND_IN_MILLIS) - )); - inMemoryTableManager.recoverTables(tables); - - List expectedFiles = Arrays.asList( - new FileAccessInfo(1, TEST_FILES.get(1), 4, - ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new FileAccessInfo(2, TEST_FILES.get(2), 3, - ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new FileAccessInfo(3, TEST_FILES.get(3), 3, - ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new FileAccessInfo(4, TEST_FILES.get(4), 2, - ONE_MINUTE_IN_MILLIS + 46 * ONE_SECOND_IN_MILLIS), - new FileAccessInfo(5, TEST_FILES.get(5), 2, - ONE_MINUTE_IN_MILLIS + 46 * ONE_SECOND_IN_MILLIS) - ); - - List hotFiles = - accessCountTableManager.search(FileAccessInfoSearchRequest.builder() - .lastAccessedTime(TimeInterval.builder() - .from(Instant.ofEpochMilli(55 * ONE_SECOND_IN_MILLIS)) - .to(Instant.ofEpochMilli(2 * ONE_MINUTE_IN_MILLIS + 5 * ONE_SECOND_IN_MILLIS)) - .build()) - .build()); - assertEquals(expectedFiles, hotFiles); - } - - @Test - public void testGetHotFilesWhenIntervalEndTimeLessThanRightBorderOfParentTable() - throws MetaStoreException { - /* - |---------interval------| - |-s-| |-s-| |-s-| - |--------min--------------| - */ - createTestFiles(); - DbAccessCountTableManager tableManager = accessCountTableManager.getDbTableManager(); - AccessCountTable t1 = new AccessCountTable(0, 5 * ONE_SECOND_IN_MILLIS); - AccessCountTable t2 = new AccessCountTable(ONE_MINUTE_IN_MILLIS + 5 * ONE_SECOND_IN_MILLIS, - ONE_MINUTE_IN_MILLIS + 10 * ONE_SECOND_IN_MILLIS); - AccessCountTable t3 = new AccessCountTable(ONE_MINUTE_IN_MILLIS + 20 * ONE_SECOND_IN_MILLIS, - ONE_MINUTE_IN_MILLIS + 25 * ONE_SECOND_IN_MILLIS); - AccessCountTable t4 = new AccessCountTable(ONE_MINUTE_IN_MILLIS + 35 * ONE_SECOND_IN_MILLIS, - ONE_MINUTE_IN_MILLIS + 40 * ONE_SECOND_IN_MILLIS); - AccessCountTable t5 = new AccessCountTable(ONE_MINUTE_IN_MILLIS + 45 * ONE_SECOND_IN_MILLIS, - ONE_MINUTE_IN_MILLIS + 50 * ONE_SECOND_IN_MILLIS); - AccessCountTable t6 = new AccessCountTable(ONE_MINUTE_IN_MILLIS, 2 * ONE_MINUTE_IN_MILLIS); - List tables = Arrays.asList(t1, t2, t3, t4, t5, t6); - tables.forEach(t -> { - try { - tableManager.createTable(t); - } catch (MetaStoreException e) { - throw new RuntimeException(e); - } - }); - tableManager.handleAggregatedEvents(t1, Arrays.asList(new AggregatedAccessCounts(3, 1, 0))); - tableManager.handleAggregatedEvents(t2, Arrays.asList( - new AggregatedAccessCounts(3, 1, ONE_MINUTE_IN_MILLIS + 10 * ONE_SECOND_IN_MILLIS))); - tableManager.handleAggregatedEvents(t3, Arrays.asList( - new AggregatedAccessCounts(2, 1, ONE_MINUTE_IN_MILLIS + 21 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(4, 1, ONE_MINUTE_IN_MILLIS + 21 * ONE_SECOND_IN_MILLIS))); - tableManager.handleAggregatedEvents(t4, Arrays.asList( - new AggregatedAccessCounts(5, 1, ONE_MINUTE_IN_MILLIS + 38 * ONE_SECOND_IN_MILLIS))); - tableManager.handleAggregatedEvents(t5, Arrays.asList( - new AggregatedAccessCounts(1, 2, ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(3, 2, ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(2, 1, ONE_MINUTE_IN_MILLIS + 46 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(4, 1, ONE_MINUTE_IN_MILLIS + 46 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(5, 1, ONE_MINUTE_IN_MILLIS + 46 * ONE_SECOND_IN_MILLIS) - )); - tableManager.handleAggregatedEvents(t6, Arrays.asList( - new AggregatedAccessCounts(1, 4, ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(2, 3, ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(3, 3, ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(4, 2, ONE_MINUTE_IN_MILLIS + 46 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(5, 2, ONE_MINUTE_IN_MILLIS + 46 * ONE_SECOND_IN_MILLIS) - )); - inMemoryTableManager.recoverTables(tables); - - List expectedFiles = Arrays.asList( - new FileAccessInfo(1, TEST_FILES.get(1), 4, - ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new FileAccessInfo(2, TEST_FILES.get(2), 3, - ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new FileAccessInfo(3, TEST_FILES.get(3), 3, - ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new FileAccessInfo(4, TEST_FILES.get(4), 2, - ONE_MINUTE_IN_MILLIS + 46 * ONE_SECOND_IN_MILLIS), - new FileAccessInfo(5, TEST_FILES.get(5), 2, - ONE_MINUTE_IN_MILLIS + 46 * ONE_SECOND_IN_MILLIS) - ); - List hotFiles = - accessCountTableManager.search(FileAccessInfoSearchRequest.builder() - .lastAccessedTime(TimeInterval.builder() - .from(Instant.ofEpochMilli(55 * ONE_SECOND_IN_MILLIS)) - .to(Instant.ofEpochMilli(ONE_MINUTE_IN_MILLIS + 56 * ONE_SECOND_IN_MILLIS)) - .build()) - .build()); - assertEquals(expectedFiles, hotFiles); - } - - @Test - public void testGetHotFilesWhenIntervalStartTimeIncludesIntoChildTable() - throws MetaStoreException { - /* - |--interval-| - |-s-| |-s-| |-s-| - |--------min-------------------| - */ - createTestFiles(); - DbAccessCountTableManager tableManager = accessCountTableManager.getDbTableManager(); - AccessCountTable t4 = new AccessCountTable(ONE_MINUTE_IN_MILLIS + 35 * ONE_SECOND_IN_MILLIS, - ONE_MINUTE_IN_MILLIS + 40 * ONE_SECOND_IN_MILLIS); - AccessCountTable t5 = new AccessCountTable(ONE_MINUTE_IN_MILLIS + 45 * ONE_SECOND_IN_MILLIS, - ONE_MINUTE_IN_MILLIS + 50 * ONE_SECOND_IN_MILLIS); - AccessCountTable t6 = new AccessCountTable(ONE_MINUTE_IN_MILLIS, 2 * ONE_MINUTE_IN_MILLIS); - List tables = Arrays.asList(t4, t5, t6); - tables.forEach(t -> { - try { - tableManager.createTable(t); - } catch (MetaStoreException e) { - throw new RuntimeException(e); - } - }); - tableManager.handleAggregatedEvents(t4, Arrays.asList( - new AggregatedAccessCounts(5, 1, ONE_MINUTE_IN_MILLIS + 38 * ONE_SECOND_IN_MILLIS))); - tableManager.handleAggregatedEvents(t5, Arrays.asList( - new AggregatedAccessCounts(1, 4, ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(2, 3, ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(3, 3, ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(4, 3, ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(5, 2, ONE_MINUTE_IN_MILLIS + 46 * ONE_SECOND_IN_MILLIS) - )); - tableManager.handleAggregatedEvents(t6, Arrays.asList( - new AggregatedAccessCounts(1, 4, ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(2, 3, ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(3, 3, ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(4, 3, ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(5, 3, ONE_MINUTE_IN_MILLIS + 46 * ONE_SECOND_IN_MILLIS) - )); - inMemoryTableManager.recoverTables(tables); - - List expectedFiles = Arrays.asList( - new FileAccessInfo(1, TEST_FILES.get(1), 2, - ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS), - new FileAccessInfo(2, TEST_FILES.get(2), 1, - ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new FileAccessInfo(3, TEST_FILES.get(3), 1, - ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS), - new FileAccessInfo(4, TEST_FILES.get(4), 1, - ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS) - ); - - List hotFiles = - accessCountTableManager.search(FileAccessInfoSearchRequest.builder() - .lastAccessedTime(TimeInterval.builder() - .from(Instant.ofEpochMilli(ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS)) - .to(Instant.ofEpochMilli(ONE_MINUTE_IN_MILLIS + 56 * ONE_SECOND_IN_MILLIS)) - .build()) - .build()); - assertEquals(expectedFiles, hotFiles); - } - - @Test - public void testGetHotFilesWhenIntervalStartAndEndTimeIncludesIntoChildTables() - throws MetaStoreException { - /* - |--interval-| - |-s-| |-s-| |-s-| - |--------min-----------| - */ - createTestFiles(); - DbAccessCountTableManager tableManager = accessCountTableManager.getDbTableManager(); - AccessCountTable t1 = new AccessCountTable(ONE_MINUTE_IN_MILLIS + 5 * ONE_SECOND_IN_MILLIS, - ONE_MINUTE_IN_MILLIS + 10 * ONE_SECOND_IN_MILLIS); - AccessCountTable t2 = new AccessCountTable(ONE_MINUTE_IN_MILLIS + 20 * ONE_SECOND_IN_MILLIS, - ONE_MINUTE_IN_MILLIS + 25 * ONE_SECOND_IN_MILLIS); - AccessCountTable t4 = new AccessCountTable(ONE_MINUTE_IN_MILLIS + 35 * ONE_SECOND_IN_MILLIS, - ONE_MINUTE_IN_MILLIS + 40 * ONE_SECOND_IN_MILLIS); - AccessCountTable t5 = new AccessCountTable(ONE_MINUTE_IN_MILLIS + 45 * ONE_SECOND_IN_MILLIS, - ONE_MINUTE_IN_MILLIS + 50 * ONE_SECOND_IN_MILLIS); - AccessCountTable t6 = new AccessCountTable(ONE_MINUTE_IN_MILLIS, 2 * ONE_MINUTE_IN_MILLIS); - List tables = Arrays.asList(t1, t2, t4, t5, t6); - tables.forEach(t -> { - try { - tableManager.createTable(t); - } catch (MetaStoreException e) { - throw new RuntimeException(e); - } - }); - tableManager.handleAggregatedEvents(t1, Arrays.asList( - new AggregatedAccessCounts(1, 4, ONE_MINUTE_IN_MILLIS + 9 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(2, 2, ONE_MINUTE_IN_MILLIS + 8 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(3, 1, ONE_MINUTE_IN_MILLIS + 5 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(4, 2, ONE_MINUTE_IN_MILLIS + 9 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(5, 1, ONE_MINUTE_IN_MILLIS + 7 * ONE_SECOND_IN_MILLIS))); - tableManager.handleAggregatedEvents(t2, Arrays.asList( - new AggregatedAccessCounts(2, 1, ONE_MINUTE_IN_MILLIS + 21 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(4, 1, ONE_MINUTE_IN_MILLIS + 21 * ONE_SECOND_IN_MILLIS))); - tableManager.handleAggregatedEvents(t4, Arrays.asList( - new AggregatedAccessCounts(5, 1, ONE_MINUTE_IN_MILLIS + 38 * ONE_SECOND_IN_MILLIS))); - tableManager.handleAggregatedEvents(t5, Arrays.asList( - new AggregatedAccessCounts(1, 4, ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(2, 3, ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(3, 3, ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(4, 3, ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(5, 2, ONE_MINUTE_IN_MILLIS + 46 * ONE_SECOND_IN_MILLIS) - )); - tableManager.handleAggregatedEvents(t6, Arrays.asList( - new AggregatedAccessCounts(1, 4, ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(2, 3, ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(3, 3, ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(4, 3, ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(5, 3, ONE_MINUTE_IN_MILLIS + 46 * ONE_SECOND_IN_MILLIS) - )); - inMemoryTableManager.recoverTables(tables); - - List expectedFiles = Arrays.asList( - new FileAccessInfo(1, TEST_FILES.get(1), 4, - ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS), - new FileAccessInfo(2, TEST_FILES.get(2), 2, - ONE_MINUTE_IN_MILLIS + 21 * ONE_SECOND_IN_MILLIS), - new FileAccessInfo(3, TEST_FILES.get(3), 2, - ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS), - new FileAccessInfo(4, TEST_FILES.get(4), 2, - ONE_MINUTE_IN_MILLIS + 21 * ONE_SECOND_IN_MILLIS), - new FileAccessInfo(5, TEST_FILES.get(5), 2, - ONE_MINUTE_IN_MILLIS + 46 * ONE_SECOND_IN_MILLIS) - ); - - List hotFiles = - accessCountTableManager.search(FileAccessInfoSearchRequest.builder() - .lastAccessedTime(TimeInterval.builder() - .from(Instant.ofEpochMilli(ONE_MINUTE_IN_MILLIS + 8 * ONE_SECOND_IN_MILLIS)) - .to(Instant.ofEpochMilli(ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS)) - .build()) - .build()); - assertEquals(expectedFiles, hotFiles); - } - - @Test - public void testGetHotFilesWhenIntervalBiggerThanExistedTablesCombineGranularity() - throws MetaStoreException { - /* - |---------interval------------------------| - |-s-| |-s-| |-s-| - |-----min-----------|-------------------| - */ - createTestFiles(); - DbAccessCountTableManager tableManager = accessCountTableManager.getDbTableManager(); - - AccessCountTable t0 = new AccessCountTable(0, ONE_MINUTE_IN_MILLIS); - AccessCountTable t1 = new AccessCountTable(ONE_MINUTE_IN_MILLIS + 5 * ONE_SECOND_IN_MILLIS, - ONE_MINUTE_IN_MILLIS + 10 * ONE_SECOND_IN_MILLIS); - AccessCountTable t2 = new AccessCountTable(ONE_MINUTE_IN_MILLIS + 20 * ONE_SECOND_IN_MILLIS, - ONE_MINUTE_IN_MILLIS + 25 * ONE_SECOND_IN_MILLIS); - AccessCountTable t4 = new AccessCountTable(ONE_MINUTE_IN_MILLIS + 35 * ONE_SECOND_IN_MILLIS, - ONE_MINUTE_IN_MILLIS + 40 * ONE_SECOND_IN_MILLIS); - AccessCountTable t5 = new AccessCountTable(ONE_MINUTE_IN_MILLIS + 45 * ONE_SECOND_IN_MILLIS, - ONE_MINUTE_IN_MILLIS + 50 * ONE_SECOND_IN_MILLIS); - AccessCountTable t6 = new AccessCountTable(ONE_MINUTE_IN_MILLIS, 2 * ONE_MINUTE_IN_MILLIS); - List tables = Arrays.asList(t0, t1, t2, t4, t5, t6); - tables.forEach(t -> { - try { - tableManager.createTable(t); - } catch (MetaStoreException e) { - throw new RuntimeException(e); - } - }); - tableManager.handleAggregatedEvents(t0, Arrays.asList( - new AggregatedAccessCounts(1, 2, 38 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(3, 5, 44 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(4, 1, 45 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(5, 2, 51 * ONE_SECOND_IN_MILLIS))); - tableManager.handleAggregatedEvents(t1, Arrays.asList( - new AggregatedAccessCounts(1, 4, ONE_MINUTE_IN_MILLIS + 9 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(2, 2, ONE_MINUTE_IN_MILLIS + 8 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(3, 1, ONE_MINUTE_IN_MILLIS + 5 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(4, 2, ONE_MINUTE_IN_MILLIS + 9 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(5, 1, ONE_MINUTE_IN_MILLIS + 7 * ONE_SECOND_IN_MILLIS))); - tableManager.handleAggregatedEvents(t2, Arrays.asList( - new AggregatedAccessCounts(2, 1, ONE_MINUTE_IN_MILLIS + 21 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(4, 1, ONE_MINUTE_IN_MILLIS + 21 * ONE_SECOND_IN_MILLIS))); - tableManager.handleAggregatedEvents(t4, Arrays.asList( - new AggregatedAccessCounts(5, 1, ONE_MINUTE_IN_MILLIS + 38 * ONE_SECOND_IN_MILLIS))); - tableManager.handleAggregatedEvents(t5, Arrays.asList( - new AggregatedAccessCounts(1, 4, ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(2, 3, ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(3, 3, ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(4, 3, ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(5, 2, ONE_MINUTE_IN_MILLIS + 46 * ONE_SECOND_IN_MILLIS) - )); - tableManager.handleAggregatedEvents(t6, Arrays.asList( - new AggregatedAccessCounts(1, 4, ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(2, 3, ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(3, 3, ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(4, 3, ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(5, 3, ONE_MINUTE_IN_MILLIS + 46 * ONE_SECOND_IN_MILLIS) - )); - inMemoryTableManager.recoverTables(tables); - - List expectedFiles = Arrays.asList( - new FileAccessInfo(1, TEST_FILES.get(1), 6, - ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS), - new FileAccessInfo(2, TEST_FILES.get(2), 3, - ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new FileAccessInfo(3, TEST_FILES.get(3), 8, - ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS), - new FileAccessInfo(4, TEST_FILES.get(4), 4, - ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new FileAccessInfo(5, TEST_FILES.get(5), 5, - ONE_MINUTE_IN_MILLIS + 46 * ONE_SECOND_IN_MILLIS) - ); - - List hotFiles = - accessCountTableManager.search(FileAccessInfoSearchRequest.builder() - .lastAccessedTime(TimeInterval.builder() - .from(Instant.ofEpochMilli(0)) - .to(Instant.ofEpochMilli(2 * ONE_MINUTE_IN_MILLIS + 4 * ONE_SECOND_IN_MILLIS)) - .build()) - .build()); - assertEquals(expectedFiles, hotFiles); - } - - @Test - public void testGetHotFilesWhenIntervalEndsWithinSmallestTableCombinedGranularity() - throws MetaStoreException { - /* - |---------interval------------| - |-s-| |-s-| - |-----min----------|--------------| - |-----------------hour--------...-----| - */ - createTestFiles(); - DbAccessCountTableManager tableManager = accessCountTableManager.getDbTableManager(); - AccessCountTable t0 = new AccessCountTable(0, ONE_MINUTE_IN_MILLIS); - AccessCountTable t1 = new AccessCountTable(ONE_MINUTE_IN_MILLIS + 5 * ONE_SECOND_IN_MILLIS, - ONE_MINUTE_IN_MILLIS + 10 * ONE_SECOND_IN_MILLIS); - AccessCountTable t2 = new AccessCountTable(ONE_MINUTE_IN_MILLIS + 20 * ONE_SECOND_IN_MILLIS, - ONE_MINUTE_IN_MILLIS + 25 * ONE_SECOND_IN_MILLIS); - AccessCountTable t4 = new AccessCountTable(ONE_MINUTE_IN_MILLIS + 35 * ONE_SECOND_IN_MILLIS, - ONE_MINUTE_IN_MILLIS + 40 * ONE_SECOND_IN_MILLIS); - AccessCountTable t5 = new AccessCountTable(ONE_MINUTE_IN_MILLIS + 45 * ONE_SECOND_IN_MILLIS, - ONE_MINUTE_IN_MILLIS + 50 * ONE_SECOND_IN_MILLIS); - AccessCountTable t6 = new AccessCountTable(ONE_MINUTE_IN_MILLIS, 2 * ONE_MINUTE_IN_MILLIS); - AccessCountTable t7 = new AccessCountTable(0, ONE_HOUR_IN_MILLIS); - List tables = Arrays.asList(t0, t1, t2, t4, t5, t6, t7); - tables.forEach(t -> { - try { - tableManager.createTable(t); - } catch (MetaStoreException e) { - throw new RuntimeException(e); - } - }); - tableManager.handleAggregatedEvents(t0, Arrays.asList( - new AggregatedAccessCounts(1, 2, 38 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(3, 5, 44 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(4, 1, 45 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(5, 2, 51 * ONE_SECOND_IN_MILLIS))); - tableManager.handleAggregatedEvents(t1, Arrays.asList( - new AggregatedAccessCounts(1, 4, ONE_MINUTE_IN_MILLIS + 9 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(2, 2, ONE_MINUTE_IN_MILLIS + 8 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(3, 1, ONE_MINUTE_IN_MILLIS + 5 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(4, 2, ONE_MINUTE_IN_MILLIS + 9 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(5, 1, ONE_MINUTE_IN_MILLIS + 7 * ONE_SECOND_IN_MILLIS))); - tableManager.handleAggregatedEvents(t2, Arrays.asList( - new AggregatedAccessCounts(2, 1, ONE_MINUTE_IN_MILLIS + 21 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(4, 1, ONE_MINUTE_IN_MILLIS + 21 * ONE_SECOND_IN_MILLIS))); - tableManager.handleAggregatedEvents(t4, Arrays.asList( - new AggregatedAccessCounts(5, 1, ONE_MINUTE_IN_MILLIS + 38 * ONE_SECOND_IN_MILLIS))); - tableManager.handleAggregatedEvents(t5, Arrays.asList( - new AggregatedAccessCounts(1, 4, ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(2, 3, ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(3, 3, ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(4, 3, ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(5, 2, ONE_MINUTE_IN_MILLIS + 46 * ONE_SECOND_IN_MILLIS) - )); - tableManager.handleAggregatedEvents(t6, Arrays.asList( - new AggregatedAccessCounts(1, 4, ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(2, 3, ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(3, 3, ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(4, 3, ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(5, 3, ONE_MINUTE_IN_MILLIS + 46 * ONE_SECOND_IN_MILLIS) - )); - tableManager.handleAggregatedEvents(t7, Arrays.asList( - new AggregatedAccessCounts(1, 24, 50 * ONE_MINUTE_IN_MILLIS), - new AggregatedAccessCounts(2, 18, 49 * ONE_MINUTE_IN_MILLIS), - new AggregatedAccessCounts(3, 18, 30 * ONE_MINUTE_IN_MILLIS), - new AggregatedAccessCounts(4, 15, 55 * ONE_MINUTE_IN_MILLIS), - new AggregatedAccessCounts(5, 27, 57 * ONE_MINUTE_IN_MILLIS) - )); - inMemoryTableManager.recoverTables(tables); - - List expectedFiles = Arrays.asList( - new FileAccessInfo(1, TEST_FILES.get(1), 8, - ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS), - new FileAccessInfo(2, TEST_FILES.get(2), 3, - ONE_MINUTE_IN_MILLIS + 21 * ONE_SECOND_IN_MILLIS), - new FileAccessInfo(3, TEST_FILES.get(3), 8, - ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS), - new FileAccessInfo(4, TEST_FILES.get(4), 4, - ONE_MINUTE_IN_MILLIS + 21 * ONE_SECOND_IN_MILLIS), - new FileAccessInfo(5, TEST_FILES.get(5), 5, - ONE_MINUTE_IN_MILLIS + 46 * ONE_SECOND_IN_MILLIS) - ); - - List hotFiles = - accessCountTableManager.search(FileAccessInfoSearchRequest.builder() - .lastAccessedTime(TimeInterval.builder() - .from(Instant.ofEpochMilli(0)) - .to(Instant.ofEpochMilli(ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS)) - .build()) - .build()); - assertEquals(expectedFiles, hotFiles); - } - - @Test - public void testGetHotFilesWhenIntervalStartsInParentTableAndEndsInChildTable() - throws MetaStoreException { - /* - |---interval--| - |-s-| - |---------m------------| - */ - createTestFiles(); - DbAccessCountTableManager tableManager = accessCountTableManager.getDbTableManager(); - - AccessCountTable t5 = new AccessCountTable(ONE_MINUTE_IN_MILLIS + 45 * ONE_SECOND_IN_MILLIS, - ONE_MINUTE_IN_MILLIS + 50 * ONE_SECOND_IN_MILLIS); - AccessCountTable t6 = new AccessCountTable(ONE_MINUTE_IN_MILLIS, 2 * ONE_MINUTE_IN_MILLIS); - List tables = Arrays.asList(t5, t6); - tables.forEach(t -> { - try { - tableManager.createTable(t); - } catch (MetaStoreException e) { - throw new RuntimeException(e); - } - }); - tableManager.handleAggregatedEvents(t5, Arrays.asList( - new AggregatedAccessCounts(1, 4, ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(2, 3, ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(3, 3, ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(4, 3, ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(5, 2, ONE_MINUTE_IN_MILLIS + 46 * ONE_SECOND_IN_MILLIS) - )); - tableManager.handleAggregatedEvents(t6, Arrays.asList( - new AggregatedAccessCounts(1, 10, ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(2, 5, ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(3, 7, ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(4, 8, ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(5, 4, ONE_MINUTE_IN_MILLIS + 46 * ONE_SECOND_IN_MILLIS) - )); - inMemoryTableManager.recoverTables(tables); - - List expectedFiles = Arrays.asList( - new FileAccessInfo(1, TEST_FILES.get(1), 2, - ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS), - new FileAccessInfo(3, TEST_FILES.get(3), 2, - ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS), - new FileAccessInfo(5, TEST_FILES.get(5), 1, - ONE_MINUTE_IN_MILLIS + 46 * ONE_SECOND_IN_MILLIS) - ); - - List hotFiles = - accessCountTableManager.search(FileAccessInfoSearchRequest.builder() - .lastAccessedTime(TimeInterval.builder() - .from(Instant.ofEpochMilli(ONE_MINUTE_IN_MILLIS + 10 * ONE_SECOND_IN_MILLIS)) - .to(Instant.ofEpochMilli(ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS)) - .build()) - .build()); - assertEquals(expectedFiles, hotFiles); - } - - @Test - public void testGetHotFilesWhenIntervalIncludesOnlyInParentTable() - throws MetaStoreException { - /* - |--interval--| - |-s-| - |---------m------------| - */ - createTestFiles(); - DbAccessCountTableManager tableManager = accessCountTableManager.getDbTableManager(); - - AccessCountTable t5 = new AccessCountTable(ONE_MINUTE_IN_MILLIS + 45 * ONE_SECOND_IN_MILLIS, - ONE_MINUTE_IN_MILLIS + 50 * ONE_SECOND_IN_MILLIS); - AccessCountTable t6 = new AccessCountTable(ONE_MINUTE_IN_MILLIS, 2 * ONE_MINUTE_IN_MILLIS); - List tables = Arrays.asList(t5, t6); - tables.forEach(t -> { - try { - tableManager.createTable(t); - } catch (MetaStoreException e) { - throw new RuntimeException(e); - } - }); - tableManager.handleAggregatedEvents(t5, Arrays.asList( - new AggregatedAccessCounts(1, 4, ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(3, 3, ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(4, 3, ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS) - )); - tableManager.handleAggregatedEvents(t6, Arrays.asList( - new AggregatedAccessCounts(1, 10, ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(2, 15, ONE_MINUTE_IN_MILLIS + 20 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(3, 7, ONE_MINUTE_IN_MILLIS + 48 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(4, 8, ONE_MINUTE_IN_MILLIS + 49 * ONE_SECOND_IN_MILLIS), - new AggregatedAccessCounts(5, 4, ONE_MINUTE_IN_MILLIS + 30 * ONE_SECOND_IN_MILLIS) - )); - inMemoryTableManager.recoverTables(tables); - - List expectedFiles = Arrays.asList( - new FileAccessInfo(2, TEST_FILES.get(2), 8, - ONE_MINUTE_IN_MILLIS + 20 * ONE_SECOND_IN_MILLIS), - new FileAccessInfo(5, TEST_FILES.get(5), 2, - ONE_MINUTE_IN_MILLIS + 30 * ONE_SECOND_IN_MILLIS) - ); - - List hotFiles = - accessCountTableManager.search(FileAccessInfoSearchRequest.builder() - .lastAccessedTime(TimeInterval.builder() - .from(Instant.ofEpochMilli(ONE_MINUTE_IN_MILLIS + 10 * ONE_SECOND_IN_MILLIS)) - .to(Instant.ofEpochMilli(ONE_MINUTE_IN_MILLIS + 40 * ONE_SECOND_IN_MILLIS)) - .build()) - .build()); - assertEquals(expectedFiles, hotFiles); - } - - private void submitAccessEvents() { - InMemoryAccessEventAggregator accessEventAggregator = - accessCountTableManager.getAccessEventAggregator(); - List accessEvents = Arrays.asList( - new FileAccessEvent(TEST_FILES.get(0), 0), - new FileAccessEvent(TEST_FILES.get(1), 1), - new FileAccessEvent(TEST_FILES.get(2), - 2 * ONE_MINUTE_IN_MILLIS + 1), - new FileAccessEvent("/unknown", - 3 * ONE_MINUTE_IN_MILLIS), - new FileAccessEvent(TEST_FILES.get(1), - 4 * ONE_MINUTE_IN_MILLIS + 10 * ONE_SECOND_IN_MILLIS), - new FileAccessEvent(TEST_FILES.get(2), - ONE_HOUR_IN_MILLIS + 5 * ONE_MINUTE_IN_MILLIS), - new FileAccessEvent(TEST_FILES.get(3), - ONE_HOUR_IN_MILLIS + 9 * ONE_MINUTE_IN_MILLIS), - new FileAccessEvent(TEST_FILES.get(3), - 8 * ONE_HOUR_IN_MILLIS), - new FileAccessEvent(TEST_FILES.get(3), - ONE_DAY_IN_MILLIS + 12 * ONE_HOUR_IN_MILLIS), - new FileAccessEvent(TEST_FILES.get(2), - ONE_DAY_IN_MILLIS + 12 * ONE_HOUR_IN_MILLIS + 1), - new FileAccessEvent(TEST_FILES.get(1), - ONE_DAY_IN_MILLIS + 13 * ONE_HOUR_IN_MILLIS + 10 * ONE_SECOND_IN_MILLIS), - new FileAccessEvent(TEST_FILES.get(4), - ONE_DAY_IN_MILLIS + 17 * ONE_HOUR_IN_MILLIS), - new FileAccessEvent(TEST_FILES.get(5), - ONE_DAY_IN_MILLIS + 18 * ONE_HOUR_IN_MILLIS + 10), - new FileAccessEvent(TEST_FILES.get(3), - ONE_DAY_IN_MILLIS + 23 * ONE_HOUR_IN_MILLIS + 59 * ONE_MINUTE_IN_MILLIS - + 58 * ONE_SECOND_IN_MILLIS), - new FileAccessEvent(TEST_FILES.get(3), - ONE_DAY_IN_MILLIS + 23 * ONE_HOUR_IN_MILLIS + 59 * ONE_MINUTE_IN_MILLIS - + 59 * ONE_SECOND_IN_MILLIS), - new FileAccessEvent("", 2 * ONE_DAY_IN_MILLIS) - ); - accessEventAggregator.aggregate(accessEvents); - } - - private void createTestFiles() throws MetaStoreException { - FileInfo[] fileInfos = IntStream.range(0, TEST_FILES.size()) - .mapToObj(id -> FileInfo.newBuilder() - .setFileId(id) - .setPath(TEST_FILES.get(id)) - .build()) - .toArray(FileInfo[]::new); - - metaStore.insertFiles(fileInfos); - } -} diff --git a/smart-metastore/src/test/java/org/smartdata/metastore/dao/accesscount/TestAccessEventAggregator.java b/smart-metastore/src/test/java/org/smartdata/metastore/dao/accesscount/TestAccessEventAggregator.java index e50a504313a..8b5fab634b1 100644 --- a/smart-metastore/src/test/java/org/smartdata/metastore/dao/accesscount/TestAccessEventAggregator.java +++ b/smart-metastore/src/test/java/org/smartdata/metastore/dao/accesscount/TestAccessEventAggregator.java @@ -17,108 +17,85 @@ */ package org.smartdata.metastore.dao.accesscount; -import lombok.Getter; import org.junit.Before; import org.junit.Test; import org.smartdata.metastore.TestDaoBase; -import org.smartdata.metastore.dao.accesscount.InMemoryAccessEventAggregator.WindowClosedCallback; -import org.smartdata.metastore.model.AccessCountTable; -import org.smartdata.metastore.model.AggregatedAccessCounts; +import org.smartdata.metastore.accesscount.DbAccessEventAggregator; +import org.smartdata.metastore.accesscount.FileAccessManager; +import org.smartdata.metastore.accesscount.failover.AccessCountContext; +import org.smartdata.metastore.accesscount.failover.Failover; +import org.smartdata.metastore.model.SearchResult; +import org.smartdata.metastore.queries.PageRequest; +import org.smartdata.metastore.queries.sort.FileAccessInfoSortField; +import org.smartdata.metastore.queries.sort.Sorting; +import org.smartdata.metastore.transaction.TransactionRunner; import org.smartdata.metrics.FileAccessEvent; +import org.smartdata.model.FileAccessInfo; import org.smartdata.model.FileInfo; +import org.smartdata.model.request.FileAccessInfoSearchRequest; import org.testcontainers.shaded.com.google.common.collect.ImmutableMap; -import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; -import java.util.Comparator; -import java.util.List; import java.util.Map; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; public class TestAccessEventAggregator extends TestDaoBase { - private static final long AGGREGATION_GRANULARITY_MS = 5000; private static final Map FILE_PATH_IDS = ImmutableMap.of( "/file1", 1L, "/file2", 2L, "/file3", 3L, "/file4", 4L ); - - private InMemoryAccessEventAggregator aggregator; - private MockWindowClosedCallback windowCallback; + private FileAccessManager dbTableManager; + private DbAccessEventAggregator aggregator; @Before public void setup() { - windowCallback = new MockWindowClosedCallback(); - aggregator = new InMemoryAccessEventAggregator( - metaStore.fileInfoDao(), - windowCallback, - AccessCountEventAggregatorFailover.dropEvents(), - AGGREGATION_GRANULARITY_MS); - + dbTableManager = + new FileAccessManager(new TransactionRunner(metaStore.transactionManager()), + metaStore.accessCountEventDao(), + metaStore.cacheFileDao()); + aggregator = + new DbAccessEventAggregator(metaStore.fileInfoDao(), + dbTableManager, new Failover(){}); metaStore.fileInfoDao().insert(testFileInfos()); } @Test public void testAggregateEvents() { - List createdTables = windowCallback.getCreatedTables(); - List collectedAccessCounts = - windowCallback.getCollectedAccessCounts(); - - aggregator.aggregate(Collections.singletonList(new FileAccessEvent("", 3000))); - assertTrue(createdTables.isEmpty()); - assertTrue(collectedAccessCounts.isEmpty()); - + long currentTimeMs = System.currentTimeMillis(); + aggregator.aggregate(Collections.singletonList(new FileAccessEvent("", currentTimeMs))); aggregator.aggregate(Collections.singletonList - (new FileAccessEvent("/file1", 4999))); + (new FileAccessEvent("/file1", currentTimeMs + 1))); aggregator.aggregate(Collections.singletonList( - new FileAccessEvent("", 6000))); - - assertEquals( - Collections.singletonList(new AccessCountTable(0, 5000)), - createdTables); - assertEquals( - Collections.singletonList( - new AggregatedAccessCounts(FILE_PATH_IDS.get("/file1"), 1, 4999)), - collectedAccessCounts); - + new FileAccessEvent("", currentTimeMs + 1000))); aggregator.aggregate( Arrays.asList( - new FileAccessEvent("/file1", 7900), - new FileAccessEvent("/file1", 7999), - new FileAccessEvent("/file1", 8000), - new FileAccessEvent("/file2", 14000), - new FileAccessEvent("/file3", 14000), - new FileAccessEvent("/file3", 14001), - new FileAccessEvent("/file3", 14002), - new FileAccessEvent("/unknown_file", 14003), - new FileAccessEvent("/file4", 16000), - new FileAccessEvent("", 22000))); - - List expectedTables = Arrays.asList( - new AccessCountTable(0, 5000), - new AccessCountTable(5000, 10000), - new AccessCountTable(10000, 15000), - new AccessCountTable(15000, 20000) - ); - assertEquals(expectedTables, createdTables); - - List expectedAccessCounts = Arrays.asList( - new AggregatedAccessCounts(FILE_PATH_IDS.get("/file1"), 1, 4999), - new AggregatedAccessCounts(FILE_PATH_IDS.get("/file1"), 3, 8000), - new AggregatedAccessCounts(FILE_PATH_IDS.get("/file2"), 1, 14000), - new AggregatedAccessCounts(FILE_PATH_IDS.get("/file3"), 3, 14002), - new AggregatedAccessCounts(FILE_PATH_IDS.get("/file4"), 1, 16000) - ); - - collectedAccessCounts.sort( - Comparator.comparingLong(AggregatedAccessCounts::getLastAccessedTimestamp)); - assertEquals(expectedAccessCounts, collectedAccessCounts); + new FileAccessEvent("/file1", currentTimeMs + 2000), + new FileAccessEvent("/file1", currentTimeMs + 7999), + new FileAccessEvent("/file1", currentTimeMs + 8000), + new FileAccessEvent("/file2", currentTimeMs + 14000), + new FileAccessEvent("/file3", currentTimeMs + 14000), + new FileAccessEvent("/file3", currentTimeMs + 14001), + new FileAccessEvent("/file3", currentTimeMs + 14002), + new FileAccessEvent("/unknown_file", currentTimeMs + 14003), + new FileAccessEvent("/file4", currentTimeMs + 16000), + new FileAccessEvent("", currentTimeMs + 22000))); + + SearchResult fileAccessInfos = + dbTableManager.search(FileAccessInfoSearchRequest.noFilters(), + PageRequest.builder() + .addSorting(FileAccessInfoSortField.FID, Sorting.Order.ASC) + .build()); + assertEquals(Arrays.asList( + new FileAccessInfo(FILE_PATH_IDS.get("/file1"), "/file1", 4, currentTimeMs + 8000), + new FileAccessInfo(FILE_PATH_IDS.get("/file2"), "/file2", 1, currentTimeMs + 14000), + new FileAccessInfo(FILE_PATH_IDS.get("/file3"), "/file3", 3, currentTimeMs + 14002), + new FileAccessInfo(FILE_PATH_IDS.get("/file4"), "/file4", 1, currentTimeMs + 16000)), + fileAccessInfos.getItems()); } private FileInfo[] testFileInfos() { @@ -134,20 +111,4 @@ private FileInfo dummyFileInfo(String path, long fileId) { .setFileId(fileId) .build(); } - - @Getter - private static class MockWindowClosedCallback implements WindowClosedCallback { - - private final List createdTables = new ArrayList<>(); - private final List - collectedAccessCounts = new ArrayList<>(); - - @Override - public void onWindowClosed(long windowStart, long windowEnd, - Collection aggregatedAccessCounts) { - AccessCountTable table = new AccessCountTable(windowStart, windowEnd); - createdTables.add(table); - collectedAccessCounts.addAll(aggregatedAccessCounts); - } - } } diff --git a/smart-metastore/src/test/java/org/smartdata/metastore/dao/accesscount/TestAddTableOpListener.java b/smart-metastore/src/test/java/org/smartdata/metastore/dao/accesscount/TestAddTableOpListener.java deleted file mode 100644 index 4002c1ff259..00000000000 --- a/smart-metastore/src/test/java/org/smartdata/metastore/dao/accesscount/TestAddTableOpListener.java +++ /dev/null @@ -1,137 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.smartdata.metastore.dao.accesscount; - -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.smartdata.metastore.model.AccessCountTable; - -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; - -import static org.smartdata.metastore.TestDBUtil.addAccessCountTableToDeque; - -public class TestAddTableOpListener { - - private ExecutorService executorService; - - private AccessCountTableHandler tableHandler; - - @Before - public void setup() { - executorService = Executors.newFixedThreadPool(4); - tableHandler = new NoOpAccessCountTableHandler(); - } - - @After - public void shutDown() { - executorService.shutdownNow(); - } - - @Test - public void testMinuteTableListener() throws Exception { - long oneSec = 1000L; - AccessCountTableEvictor tableEvictor = new CountTableEvictor(tableHandler, 10); - AccessCountTableDeque minuteTableDeque = new AccessCountTableDeque(tableEvictor); - TableAddOpListener minuteTableListener = - TableAddOpListener.perMinute(minuteTableDeque, tableHandler, executorService); - AccessCountTableDeque secondTableDeque = - new AccessCountTableDeque(tableEvictor, minuteTableListener); - - AccessCountTable table1 = - new AccessCountTable(45 * oneSec, 50 * oneSec); - AccessCountTable table2 = - new AccessCountTable(50 * oneSec, 55 * oneSec); - AccessCountTable table3 = - new AccessCountTable(55 * oneSec, 60 * oneSec); - - addAccessCountTableToDeque(secondTableDeque, table1); - Assert.assertTrue(minuteTableDeque.isEmpty()); - - addAccessCountTableToDeque(secondTableDeque, table2); - Assert.assertTrue(minuteTableDeque.isEmpty()); - - addAccessCountTableToDeque(secondTableDeque, table3); - Assert.assertEquals(1, minuteTableDeque.size()); - - AccessCountTable expected = new AccessCountTable(0L, 60 * oneSec); - Assert.assertEquals(minuteTableDeque.poll(), expected); - } - - @Test - public void testHourTableListener() throws Exception { - long oneMin = 60 * 1000L; - AccessCountTableEvictor tableEvictor = new CountTableEvictor(tableHandler, 10); - AccessCountTableDeque hourTableDeque = new AccessCountTableDeque(tableEvictor); - TableAddOpListener hourTableListener = - TableAddOpListener.perHour(hourTableDeque, tableHandler, executorService); - AccessCountTableDeque minuteTableDeque = - new AccessCountTableDeque(tableEvictor, hourTableListener); - - AccessCountTable table1 = - new AccessCountTable(57 * oneMin, 58 * oneMin); - AccessCountTable table2 = - new AccessCountTable(58 * oneMin, 59 * oneMin); - AccessCountTable table3 = - new AccessCountTable(59 * oneMin, 60 * oneMin); - - addAccessCountTableToDeque(minuteTableDeque, table1); - Assert.assertTrue(hourTableDeque.isEmpty()); - - addAccessCountTableToDeque(minuteTableDeque, table2); - Assert.assertTrue(hourTableDeque.isEmpty()); - - addAccessCountTableToDeque(minuteTableDeque, table3); - Assert.assertEquals(1, hourTableDeque.size()); - - AccessCountTable expected = new AccessCountTable(0L, 60 * oneMin); - Assert.assertEquals(hourTableDeque.poll(), expected); - } - - @Test - public void testDayTableListener() throws Exception { - long oneHour = 60 * 60 * 1000L; - AccessCountTableEvictor tableEvictor = new CountTableEvictor(tableHandler, 10); - AccessCountTableDeque dayTableDeque = new AccessCountTableDeque(tableEvictor); - TableAddOpListener dayTableListener = - TableAddOpListener.perDay(dayTableDeque, tableHandler, executorService); - AccessCountTableDeque hourTableDeque = - new AccessCountTableDeque(tableEvictor, dayTableListener); - - AccessCountTable table1 = - new AccessCountTable(21 * oneHour, 22 * oneHour); - AccessCountTable table2 = - new AccessCountTable(22 * oneHour, 23 * oneHour); - AccessCountTable table3 = - new AccessCountTable(23 * oneHour, 24 * oneHour); - - addAccessCountTableToDeque(hourTableDeque, table1); - Assert.assertTrue(dayTableDeque.isEmpty()); - - addAccessCountTableToDeque(hourTableDeque, table2); - Assert.assertTrue(dayTableDeque.isEmpty()); - - addAccessCountTableToDeque(hourTableDeque, table3); - Assert.assertEquals(1, dayTableDeque.size()); - - AccessCountTable today = new AccessCountTable(0L, 24 * oneHour); - Assert.assertEquals(dayTableDeque.poll(), today); - } -} diff --git a/smart-metastore/src/test/java/org/smartdata/metastore/dao/accesscount/TestDbAccessCountTableManager.java b/smart-metastore/src/test/java/org/smartdata/metastore/dao/accesscount/TestDbAccessCountTableManager.java deleted file mode 100644 index a223bd97fcf..00000000000 --- a/smart-metastore/src/test/java/org/smartdata/metastore/dao/accesscount/TestDbAccessCountTableManager.java +++ /dev/null @@ -1,127 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.smartdata.metastore.dao.accesscount; - -import org.junit.Before; -import org.junit.Test; -import org.smartdata.metastore.TestDaoBase; -import org.smartdata.metastore.db.metadata.DbMetadataProvider; -import org.smartdata.metastore.model.AccessCountTable; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -public class TestDbAccessCountTableManager extends TestDaoBase { - - private DbAccessCountTableManager dbAccessCountTableManager; - private DbMetadataProvider dbMetadataProvider; - private AccessCountTableDao accessCountTableDao; - - @Before - public void setUp() { - dbAccessCountTableManager = new DbAccessCountTableManager(metaStore); - accessCountTableDao = metaStore.accessCountTableDao(); - dbMetadataProvider = metaStore.dbMetadataProvider(); - } - - private List createTables(String... tables) throws Exception { - List createdTables = new ArrayList<>(); - - for (String tableName : tables) { - AccessCountTable table = dummyTable(tableName); - dbAccessCountTableManager.createTable(table); - createdTables.add(table); - } - - return createdTables; - } - - private AccessCountTable dummyTable(String tableName) { - return new AccessCountTable(tableName, 0L, 0L, false); - } - - @Test - public void testCreateTable() throws Exception { - createTables("table1", "table2", "table3") - .stream() - .map(AccessCountTable::getTableName) - .forEach(this::assertTableExists); - } - - @Test - public void testAggregateTables() throws Exception { - List tablesToAggregate = - createTables("table1", "table2", "table3"); - - AccessCountTable destTable = dummyTable("dest"); - dbAccessCountTableManager.aggregate(destTable, tablesToAggregate); - - assertTableExists(destTable.getTableName()); - } - - @Test - public void testDropTable() throws Exception { - List tablesToAggregate = - createTables("table1", "table2"); - - dbAccessCountTableManager.dropTable(tablesToAggregate.get(0)); - assertFalse(accessCountTableDao.tableExists("table1")); - assertFalse(dbMetadataProvider.tableExists("table1")); - assertTableExists("table2"); - - dbAccessCountTableManager.dropTable(dummyTable("another")); - assertTableExists("table2"); - } - - @Test - public void testRecoverOnlyValidTables() throws Exception { - // table from previous SSM version without last access time column - AccessCountTable oldTable = new AccessCountTable(0, 5000); - metaStore.execute( - "CREATE TABLE " + oldTable.getTableName() - + "(fid BIGINT NOT NULL, " - + "count INTEGER NOT NULL)"); - - accessCountTableDao.insert(oldTable); - - // non-existing table - AccessCountTable deletedTable = new AccessCountTable(15000, 20000); - accessCountTableDao.insert(deletedTable); - - List validTables = Arrays.asList( - new AccessCountTable(5000, 10000), - new AccessCountTable(10000, 15000), - new AccessCountTable(20000, 25000)); - for (AccessCountTable table: validTables) { - dbAccessCountTableManager.createTable(table); - } - - List validatedTables = dbAccessCountTableManager.getTables(); - assertEquals(validTables, validatedTables); - } - - private void assertTableExists(String tableName) { - assertTrue(accessCountTableDao.tableExists(tableName)); - assertTrue(dbMetadataProvider.tableExists(tableName)); - } -} diff --git a/smart-metastore/src/test/java/org/smartdata/metastore/dao/accesscount/TestFileAccessManager.java b/smart-metastore/src/test/java/org/smartdata/metastore/dao/accesscount/TestFileAccessManager.java new file mode 100644 index 00000000000..cb589fb8785 --- /dev/null +++ b/smart-metastore/src/test/java/org/smartdata/metastore/dao/accesscount/TestFileAccessManager.java @@ -0,0 +1,161 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.smartdata.metastore.dao.accesscount; + +import org.junit.Before; +import org.junit.Test; +import org.smartdata.metastore.MetaStoreException; +import org.smartdata.metastore.accesscount.FileAccessManager; +import org.smartdata.metastore.dao.Searchable; +import org.smartdata.metastore.dao.TestSearchableDao; +import org.smartdata.metastore.model.AggregatedAccessCounts; +import org.smartdata.metastore.model.SearchResult; +import org.smartdata.metastore.partition.FileAccessPartitionManager; +import org.smartdata.metastore.partition.FileAccessPartitionManagerImpl; +import org.smartdata.metastore.queries.PageRequest; +import org.smartdata.metastore.queries.sort.FileAccessInfoSortField; +import org.smartdata.metastore.queries.sort.Sorting; +import org.smartdata.metastore.transaction.TransactionRunner; +import org.smartdata.model.FileAccessInfo; +import org.smartdata.model.FileInfo; +import org.smartdata.model.TimeInterval; +import org.smartdata.model.request.FileAccessInfoSearchRequest; + +import java.time.Instant; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.stream.IntStream; + +import static org.junit.Assert.assertEquals; + +public class TestFileAccessManager extends + TestSearchableDao { + + private static final List TEST_FILES = Arrays.asList( + "/file0", + "/file1", + "/file2", + "/file3", + "/file4", + "/file5" + ); + private FileAccessManager fileAccessManager; + + @Before + public void setUp() { + FileAccessPartitionManager fileAccessPartitionManager = + new FileAccessPartitionManagerImpl(metaStore); + fileAccessPartitionManager.createNewPartitions(); + fileAccessManager = new FileAccessManager( + new TransactionRunner(metaStore.transactionManager()), + metaStore.accessCountEventDao(), + metaStore.cacheFileDao()); + } + + @Override + protected Searchable searchable() { + return fileAccessManager; + } + + @Override + protected Long getIdentifier(FileAccessInfo fileAccessInfo) { + return fileAccessInfo.getFid(); + } + + @Override + protected FileAccessInfoSortField defaultSortField() { + return FileAccessInfoSortField.FID; + } + + @Test + public void testSaveAccessCounts() throws MetaStoreException { + long currentTimeMillis = System.currentTimeMillis(); + insertFileAccessCounts(currentTimeMillis); + + SearchResult fileAccessInfos = + fileAccessManager.search(FileAccessInfoSearchRequest.noFilters(), + PageRequest.builder() + .addSorting(FileAccessInfoSortField.FID, Sorting.Order.ASC) + .build()); + assertEquals(Arrays.asList(new FileAccessInfo(1, TEST_FILES.get(1), 3, currentTimeMillis + 2), + new FileAccessInfo(2, TEST_FILES.get(2), 1, currentTimeMillis + 2), + new FileAccessInfo(3, TEST_FILES.get(3), 2, currentTimeMillis + 3)), + fileAccessInfos.getItems()); + } + + @Test + public void testSearchByIds() throws MetaStoreException { + List ids = Arrays.asList(1L, 2L); + long currentTimeMillis = System.currentTimeMillis(); + insertFileAccessCounts(currentTimeMillis); + testSearch(FileAccessInfoSearchRequest.builder().ids(ids).build(), 1L, 2L); + } + + @Test + public void testSearchByPath() throws MetaStoreException { + long currentTimeMillis = System.currentTimeMillis(); + insertFileAccessCounts(currentTimeMillis); + testSearch(FileAccessInfoSearchRequest.builder() + .pathLike("/file3") + .build(), + 3L); + testSearch(FileAccessInfoSearchRequest.builder() + .pathLike("/file") + .build(), + 1L, 2L, 3L); + } + + @Test + public void testSearchByLastAccessTime() throws MetaStoreException { + long currentTimeMillis = System.currentTimeMillis(); + insertFileAccessCounts(currentTimeMillis); + + testSearch(FileAccessInfoSearchRequest.builder() + .lastAccessedTime(TimeInterval.builder() + .from(Instant.ofEpochMilli(currentTimeMillis)) + .to(Instant.ofEpochMilli(currentTimeMillis + 2)) + .build()) + .build(), + 1L, 2L); + } + + private void insertFileAccessCounts(long currentTimeMillis) throws MetaStoreException { + createTestFiles(); + Collection accessCounts = Arrays.asList( + new AggregatedAccessCounts(1, 1, currentTimeMillis), + new AggregatedAccessCounts(1, 1, currentTimeMillis + 1), + new AggregatedAccessCounts(1, 1, currentTimeMillis + 2), + new AggregatedAccessCounts(2, 1, currentTimeMillis + 2), + new AggregatedAccessCounts(3, 1, currentTimeMillis + 2), + new AggregatedAccessCounts(3, 1, currentTimeMillis + 3) + ); + metaStore.accessCountEventDao().insert(accessCounts); + } + + private void createTestFiles() throws MetaStoreException { + FileInfo[] fileInfos = IntStream.range(0, TEST_FILES.size()) + .mapToObj(id -> FileInfo.newBuilder() + .setFileId(id) + .setPath(TEST_FILES.get(id)) + .build()) + .toArray(FileInfo[]::new); + + metaStore.insertFiles(fileInfos); + } +} diff --git a/smart-metastore/src/test/java/org/smartdata/metastore/partition/FileAccessPartitionManagerImplTest.java b/smart-metastore/src/test/java/org/smartdata/metastore/partition/FileAccessPartitionManagerImplTest.java new file mode 100644 index 00000000000..eef65b21329 --- /dev/null +++ b/smart-metastore/src/test/java/org/smartdata/metastore/partition/FileAccessPartitionManagerImplTest.java @@ -0,0 +1,57 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.smartdata.metastore.partition; + +import org.junit.Before; +import org.junit.Test; +import org.smartdata.metastore.TestDaoBase; +import org.smartdata.metastore.model.FileAccessPartition; + +import java.time.LocalDate; +import java.time.format.DateTimeFormatter; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +import static org.junit.Assert.assertEquals; + +public class FileAccessPartitionManagerImplTest extends TestDaoBase { + private static final String FILE_ACCESS_PARTITION_NAME_TEMPLATE = "file_access_%s"; + private FileAccessPartitionManager fileAccessPartitionManager; + private static final DateTimeFormatter PARTITION_DATE_TIME_FORMAT = + DateTimeFormatter.ofPattern("yyyy_MM_dd"); + + @Before + public void setUp() throws Exception { + fileAccessPartitionManager = new FileAccessPartitionManagerImpl(metaStore); + } + + @Test + public void testCreatePartitions() { + LocalDate currentDate = LocalDate.now().withDayOfMonth(1); + String currentMonthPartition = + String.format(FILE_ACCESS_PARTITION_NAME_TEMPLATE, + currentDate.format(PARTITION_DATE_TIME_FORMAT)); + String nextMonthPartition = String.format(FILE_ACCESS_PARTITION_NAME_TEMPLATE, + currentDate.plusMonths(1).withDayOfMonth(1).format(PARTITION_DATE_TIME_FORMAT)); + fileAccessPartitionManager.createNewPartitions(); + List partitions = metaStore.fileAccessPartitionDao().getAll(); + assertEquals(Arrays.asList(currentMonthPartition, nextMonthPartition), partitions.stream().map( + FileAccessPartition::getName).collect(Collectors.toList())); + } +} diff --git a/smart-metastore/src/test/java/org/smartdata/metastore/partition/cleanup/impl/MonthCountFileAccessPartitionRetentionPolicyExecutorTest.java b/smart-metastore/src/test/java/org/smartdata/metastore/partition/cleanup/impl/MonthCountFileAccessPartitionRetentionPolicyExecutorTest.java new file mode 100644 index 00000000000..3dd4a21adfa --- /dev/null +++ b/smart-metastore/src/test/java/org/smartdata/metastore/partition/cleanup/impl/MonthCountFileAccessPartitionRetentionPolicyExecutorTest.java @@ -0,0 +1,104 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.smartdata.metastore.partition.cleanup.impl; + +import org.junit.Test; +import org.smartdata.metastore.dao.FileAccessPartitionDao; +import org.smartdata.metastore.model.FileAccessPartition; +import org.smartdata.metastore.partition.cleanup.FileAccessPartitionRetentionPolicyExecutor; + +import java.time.LocalDate; +import java.util.Arrays; +import java.util.List; + +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class MonthCountFileAccessPartitionRetentionPolicyExecutorTest { + + private FileAccessPartitionRetentionPolicyExecutor retentionPolicyExecutor; + private final FileAccessPartitionDao mockFileAccessPartitionDao = + mock(FileAccessPartitionDao.class); + + @Test + public void cleanupRedundantPartitions() { + int retentionCount = 2; + retentionPolicyExecutor = + new MonthCountFileAccessPartitionRetentionPolicyExecutor(mockFileAccessPartitionDao, + retentionCount); + List partitions = Arrays.asList( + new FileAccessPartition(1, "2024-07-01", + LocalDate.of(2024, 7, 1)), + new FileAccessPartition(2, "2024-08-01", + LocalDate.of(2024, 8, 1)), + new FileAccessPartition(3, "2024-09-01", + LocalDate.of(2024, 9, 1)), + new FileAccessPartition(4, "2024-10-01", + LocalDate.of(2024, 10, 1)), + new FileAccessPartition(5, "2024-11-01", + LocalDate.of(2024, 11, 1)) + ); + when(mockFileAccessPartitionDao.getAll()).thenReturn(partitions); + retentionPolicyExecutor.cleanup(); + verify(mockFileAccessPartitionDao, times(1)).remove(eq(partitions.get(3))); + verify(mockFileAccessPartitionDao, times(1)).remove(eq(partitions.get(4))); + } + + @Test + public void testCleanupWithoutRedundantPartitions() { + int retentionCount = 2; + retentionPolicyExecutor = + new MonthCountFileAccessPartitionRetentionPolicyExecutor(mockFileAccessPartitionDao, + retentionCount); + List partitions = Arrays.asList( + new FileAccessPartition(1, "2024-07-01", + LocalDate.of(2024, 7, 1)), + new FileAccessPartition(2, "2024-08-01", + LocalDate.of(2024, 8, 1)), + new FileAccessPartition(3, "2024-09-01", + LocalDate.of(2024, 9, 1)) + ); + when(mockFileAccessPartitionDao.getAll()).thenReturn(partitions); + retentionPolicyExecutor.cleanup(); + verify(mockFileAccessPartitionDao, never()).remove(any()); + } + + @Test + public void testCleanupWithIncorrectRetentionCount() { + int retentionCount = -1; + retentionPolicyExecutor = + new MonthCountFileAccessPartitionRetentionPolicyExecutor(mockFileAccessPartitionDao, + retentionCount); + List partitions = Arrays.asList( + new FileAccessPartition(1, "2024-07-01", + LocalDate.of(2024, 7, 1)), + new FileAccessPartition(2, "2024-08-01", + LocalDate.of(2024, 8, 1)), + new FileAccessPartition(3, "2024-09-01", + LocalDate.of(2024, 9, 1)) + ); + retentionPolicyExecutor.cleanup(); + verify(mockFileAccessPartitionDao, never()).getAll(); + verify(mockFileAccessPartitionDao, never()).remove(any()); + } +} diff --git a/smart-metastore/src/test/resources/druid-template.xml b/smart-metastore/src/test/resources/druid-template.xml index 7e2e89d5ccc..dee3700feee 100644 --- a/smart-metastore/src/test/resources/druid-template.xml +++ b/smart-metastore/src/test/resources/druid-template.xml @@ -1,14 +1,12 @@ - sqlite - jdbc:sqlite:sqlit-db-path "" "" - 1 - 1 - 1 + 8 + 4 + 16 60000 90000 diff --git a/smart-rule/src/main/java/org/smartdata/rule/objects/FileObject.java b/smart-rule/src/main/java/org/smartdata/rule/objects/FileObject.java index e880035bfd5..6713e859310 100644 --- a/smart-rule/src/main/java/org/smartdata/rule/objects/FileObject.java +++ b/smart-rule/src/main/java/org/smartdata/rule/objects/FileObject.java @@ -59,18 +59,10 @@ public class FileObject extends SmartObject { new Property("acBot", ValueType.LONG, Arrays.asList(ValueType.TIMEINTVAL, ValueType.LONG), "VIRTUAL_ACCESS_COUNT_TABLE", "", false, "count")); - PROPERTIES.put("accessCountTopOnStoragePolicy", - new Property("accessCountTopOnStoragePolicy", ValueType.LONG, - Arrays.asList(ValueType.TIMEINTVAL, ValueType.LONG, ValueType.STRING), - "VIRTUAL_ACCESS_COUNT_TABLE", "", false, "count")); PROPERTIES.put("acTopSp", new Property("acTopSp", ValueType.LONG, Arrays.asList(ValueType.TIMEINTVAL, ValueType.LONG, ValueType.STRING), "VIRTUAL_ACCESS_COUNT_TABLE", "", false, "count")); - PROPERTIES.put("accessCountBottomOnStoragePolicy", - new Property("accessCountBottomOnStoragePolicy", ValueType.LONG, - Arrays.asList(ValueType.TIMEINTVAL, ValueType.LONG, ValueType.STRING), - "VIRTUAL_ACCESS_COUNT_TABLE", "", false, "count")); PROPERTIES.put("acBotSp", new Property("acBotSp", ValueType.LONG, Arrays.asList(ValueType.TIMEINTVAL, ValueType.LONG, ValueType.STRING), diff --git a/smart-rule/src/main/java/org/smartdata/rule/parser/SmartRuleVisitTranslator.java b/smart-rule/src/main/java/org/smartdata/rule/parser/SmartRuleVisitTranslator.java index 27c3fe6abb5..af45e2522ff 100644 --- a/smart-rule/src/main/java/org/smartdata/rule/parser/SmartRuleVisitTranslator.java +++ b/smart-rule/src/main/java/org/smartdata/rule/parser/SmartRuleVisitTranslator.java @@ -914,28 +914,6 @@ public NodeTransResult doGenerateSql(TreeNode root, String tableName) throws IOE return new NodeTransResult(null, "$" + mStrValue); } - if (p.getPropertyName().equals("accessCountTopOnStoragePolicy") - || p.getPropertyName().equals("accessCountBottomOnStoragePolicy") - || p.getPropertyName().equals("acTopSp") - || p.getPropertyName().equals("acBotSp")) { - boolean topFlag = p.getPropertyName().equals("accessCountTopOnStoragePolicy") - || p.getPropertyName().equals("acTopSp"); - String virTab = genAccessCountTable(transCtx == null ? 0 : transCtx.getRuleId(), - (Long) realParas.getValues().get(0)); - String func = "$@genVirtualAccessCountTable" + (topFlag ? "Top" : "Bottom") - + "ValueOnStoragePolicy"; - String mStr = virTab + (topFlag ? "_top_" : "_bottom_") - + realParas.getValues().get(1).toString() + "_on_storage_policy_" - + realParas.getValues().get(2).toString(); - String mStrValue = mStr + "_value"; - if (!sqlStatements.contains(func + "(" + mStr + ")")) { - sqlStatements.add(func + "(" + mStr + ")"); - dynamicParameters.put(mStr, Arrays.asList(realParas.getValues(), virTab, mStrValue)); - } - procAcc = true; - return new NodeTransResult(null, "$" + mStrValue); - } - return new NodeTransResult(p.getTableName(), realParas.formatParameters()); } } diff --git a/smart-rule/src/test/java/org/smartdata/rule/TestSmartRuleStringParser.java b/smart-rule/src/test/java/org/smartdata/rule/TestSmartRuleStringParser.java index 15e0885ce00..5aba7d066c1 100644 --- a/smart-rule/src/test/java/org/smartdata/rule/TestSmartRuleStringParser.java +++ b/smart-rule/src/test/java/org/smartdata/rule/TestSmartRuleStringParser.java @@ -36,11 +36,7 @@ public void testRuleTranslate() throws Exception { rules.add("file : ac(10min) > acTop(10min, 10) | sleep -ms 0"); rules.add("file : accessCount(10min) > accessCountBottom(10min, 10) | sleep -ms 0"); rules.add("file : ac(10min) > acBot(10min, 10) | sleep -ms 0"); - rules.add("file : ac(10min) > accessCountTopOnStoragePolicy(10min, 10, \"ALL_SSD\") " - + "| sleep -ms 0"); rules.add("file : ac(10min) > acTopSp(10min, 10, \"ALL_SSD\") | sleep -ms 0"); - rules.add("file : ac(10min) > accessCountBottomOnStoragePolicy(10min, 10, \"CACHE\") " - + "| sleep -ms 0"); rules.add("file : ac(10min) > acBotSp(10min, 10, \"CACHE\") | sleep -ms 0"); rules.add("file : ac(10min) > acBotSp(10min, 10, \"HOT\") and acBotSp(10min, 10, \"HOT\") > 0 " + "| sleep -ms 0"); diff --git a/smart-server/src/test/java/org/smartdata/server/MiniSmartClusterHarness.java b/smart-server/src/test/java/org/smartdata/server/MiniSmartClusterHarness.java index c9a83975844..6a1c4d0edf3 100644 --- a/smart-server/src/test/java/org/smartdata/server/MiniSmartClusterHarness.java +++ b/smart-server/src/test/java/org/smartdata/server/MiniSmartClusterHarness.java @@ -25,8 +25,6 @@ import org.smartdata.conf.SmartConf; import org.smartdata.conf.SmartConfKeys; import org.smartdata.hdfs.MiniClusterWithStoragesHarness; -import org.smartdata.metastore.TestDBUtil; -import org.smartdata.metastore.utils.MetaStoreUtils; import java.io.IOException; import java.net.URI; @@ -49,11 +47,7 @@ public void init() throws Exception { List uriList = new ArrayList<>(namenodes); conf.set(DFS_NAMENODE_HTTP_ADDRESS_KEY, uriList.get(0).toString()); conf.set(SmartConfKeys.SMART_DFS_NAMENODE_RPCSERVER_KEY, - uriList.get(0).toString()); - - String dbFile = TestDBUtil.getUniqueEmptySqliteDBFile(); - String dbUrl = MetaStoreUtils.SQLITE_URL_PREFIX + dbFile; - smartContext.getConf().set(SmartConfKeys.SMART_METASTORE_DB_URL_KEY, dbUrl); + uriList.get(0).toString()); // rpcServer start in SmartServer ssm = SmartServer.launchWith(conf); diff --git a/smart-server/src/test/java/org/smartdata/server/TestSmartServer.java b/smart-server/src/test/java/org/smartdata/server/TestSmartServer.java index 107793580db..a0edebe4218 100644 --- a/smart-server/src/test/java/org/smartdata/server/TestSmartServer.java +++ b/smart-server/src/test/java/org/smartdata/server/TestSmartServer.java @@ -23,15 +23,10 @@ import org.junit.Before; import org.junit.Test; import org.smartdata.conf.SmartConf; -import org.smartdata.conf.SmartConfKeys; -import org.smartdata.metastore.TestDBUtil; -import org.smartdata.metastore.utils.MetaStoreUtils; public class TestSmartServer { protected SmartConf conf; protected SmartServer ssm; - protected String dbFile; - protected String dbUrl; private static final int DEFAULT_BLOCK_SIZE = 100; @@ -44,11 +39,6 @@ public void setUp() throws Exception { conf = new SmartConf(); initConf(conf); - // Set db used - dbFile = TestDBUtil.getUniqueEmptySqliteDBFile(); - dbUrl = MetaStoreUtils.SQLITE_URL_PREFIX + dbFile; - conf.set(SmartConfKeys.SMART_METASTORE_DB_URL_KEY, dbUrl); - // rpcServer start in SmartServer ssm = SmartServer.launchWith(conf); } diff --git a/smart-server/src/test/java/org/smartdata/server/TestSmartServerCli.java b/smart-server/src/test/java/org/smartdata/server/TestSmartServerCli.java index 2edb87ef074..8bcb7f7b6f6 100644 --- a/smart-server/src/test/java/org/smartdata/server/TestSmartServerCli.java +++ b/smart-server/src/test/java/org/smartdata/server/TestSmartServerCli.java @@ -23,8 +23,6 @@ import org.smartdata.conf.SmartConf; import org.smartdata.conf.SmartConfKeys; import org.smartdata.hdfs.MiniClusterHarness; -import org.smartdata.metastore.TestDBUtil; -import org.smartdata.metastore.utils.MetaStoreUtils; import java.net.URI; import java.util.ArrayList; @@ -38,12 +36,7 @@ public void testConfNameNodeRPCAddr() throws Exception { try { Collection namenodes = DFSUtil.getInternalNsRpcUris(smartContext.getConf()); List uriList = new ArrayList<>(namenodes); - SmartConf conf = new SmartConf(); - // Set db used - String dbFile = TestDBUtil.getUniqueEmptySqliteDBFile(); - String dbUrl = MetaStoreUtils.SQLITE_URL_PREFIX + dbFile; - conf.set(SmartConfKeys.SMART_METASTORE_DB_URL_KEY, dbUrl); // rpcServer start in SmartServer SmartServer ssm = null; @@ -71,7 +64,7 @@ public void testConfNameNodeRPCAddr() throws Exception { Thread.sleep(1000); regServer.shutdown(); - args = new String[] { + args = new String[]{ "-h" }; SmartServer.launchWith(args, conf); diff --git a/smart-server/src/test/java/org/smartdata/server/TestSmartServerLogin.java b/smart-server/src/test/java/org/smartdata/server/TestSmartServerLogin.java index f1e2db82606..ed0eec1c9ed 100644 --- a/smart-server/src/test/java/org/smartdata/server/TestSmartServerLogin.java +++ b/smart-server/src/test/java/org/smartdata/server/TestSmartServerLogin.java @@ -27,8 +27,6 @@ import org.smartdata.conf.SmartConf; import org.smartdata.conf.SmartConfKeys; import org.smartdata.hdfs.MiniClusterFactory; -import org.smartdata.metastore.TestDBUtil; -import org.smartdata.metastore.utils.MetaStoreUtils; import java.io.File; import java.net.URI; @@ -47,8 +45,6 @@ public class TestSmartServerLogin { private int serverPort = -1; private SmartConf conf; private MiniDFSCluster cluster; - private String dbFile; - private String dbUrl; private SmartServer ssm; private final String keytabFileName = "smart.keytab"; @@ -74,12 +70,6 @@ private void initConf() throws Exception { conf.set(DFS_NAMENODE_HTTP_ADDRESS_KEY, uriList.get(0).toString()); conf.set(SmartConfKeys.SMART_DFS_NAMENODE_RPCSERVER_KEY, uriList.get(0).toString()); - - // Set db used - dbFile = TestDBUtil.getUniqueEmptySqliteDBFile(); - dbUrl = MetaStoreUtils.SQLITE_URL_PREFIX + dbFile; - conf.set(SmartConfKeys.SMART_METASTORE_DB_URL_KEY, dbUrl); - conf.setBoolean(SmartConfKeys.SMART_SECURITY_ENABLE, true); conf.set(SmartConfKeys.SMART_SERVER_KEYTAB_FILE_KEY, keytabFileName); conf.set(SmartConfKeys.SMART_SERVER_KERBEROS_PRINCIPAL_KEY, principal); diff --git a/smart-server/src/test/java/org/smartdata/server/engine/audit/TestCmdletLifecycleLogger.java b/smart-server/src/test/java/org/smartdata/server/engine/audit/TestCmdletLifecycleLogger.java index 862de859842..13e6759d6d6 100644 --- a/smart-server/src/test/java/org/smartdata/server/engine/audit/TestCmdletLifecycleLogger.java +++ b/smart-server/src/test/java/org/smartdata/server/engine/audit/TestCmdletLifecycleLogger.java @@ -22,7 +22,7 @@ import org.junit.Before; import org.junit.Test; import org.smartdata.conf.SmartConf; -import org.smartdata.metastore.SqliteTestDaoBase; +import org.smartdata.metastore.TestDaoBase; import org.smartdata.model.CmdletInfo; import org.smartdata.model.audit.UserActivityEvent; import org.smartdata.model.request.AuditSearchRequest; @@ -46,7 +46,7 @@ import static org.smartdata.model.audit.UserActivityResult.FAILURE; import static org.smartdata.model.audit.UserActivityResult.SUCCESS; -public class TestCmdletLifecycleLogger extends SqliteTestDaoBase { +public class TestCmdletLifecycleLogger extends TestDaoBase { private CmdletManager cmdletManager; private AuditService auditService; private SmartPrincipalManager principalManager; diff --git a/smart-server/src/test/java/org/smartdata/server/engine/audit/TestRuleLifecycleLogger.java b/smart-server/src/test/java/org/smartdata/server/engine/audit/TestRuleLifecycleLogger.java index edd75ddfb06..8583c98f8d6 100644 --- a/smart-server/src/test/java/org/smartdata/server/engine/audit/TestRuleLifecycleLogger.java +++ b/smart-server/src/test/java/org/smartdata/server/engine/audit/TestRuleLifecycleLogger.java @@ -22,7 +22,7 @@ import org.junit.Before; import org.junit.Test; import org.smartdata.conf.SmartConf; -import org.smartdata.metastore.SqliteTestDaoBase; +import org.smartdata.metastore.TestDaoBase; import org.smartdata.model.RuleInfo; import org.smartdata.model.RuleState; import org.smartdata.model.audit.UserActivityEvent; @@ -47,7 +47,7 @@ import static org.smartdata.model.audit.UserActivityResult.FAILURE; import static org.smartdata.model.audit.UserActivityResult.SUCCESS; -public class TestRuleLifecycleLogger extends SqliteTestDaoBase { +public class TestRuleLifecycleLogger extends TestDaoBase { private RuleManager ruleManager; private AuditService auditService; private SmartPrincipalManager principalManager; diff --git a/smart-server/src/test/resources/druid-template.xml b/smart-server/src/test/resources/druid-template.xml index 7e2e89d5ccc..278346ba9d6 100644 --- a/smart-server/src/test/resources/druid-template.xml +++ b/smart-server/src/test/resources/druid-template.xml @@ -1,14 +1,14 @@ - sqlite - jdbc:sqlite:sqlit-db-path + jdbc:tc:postgresql:12.19:///ssm_postgres + org.testcontainers.jdbc.ContainerDatabaseDriver "" "" - 1 - 1 - 1 + 8 + 4 + 16 60000 90000 diff --git a/smart-web-server/src/main/java/org/smartdata/server/config/SsmContextInitializer.java b/smart-web-server/src/main/java/org/smartdata/server/config/SsmContextInitializer.java index 0dcf353eee2..6f0fd80fae1 100644 --- a/smart-web-server/src/main/java/org/smartdata/server/config/SsmContextInitializer.java +++ b/smart-web-server/src/main/java/org/smartdata/server/config/SsmContextInitializer.java @@ -51,7 +51,7 @@ public void initialize(ConfigurableApplicationContext applicationContext) { "cachedFilesManager", smartEngine.getStatesManager().getCachedFilesManager()); beanFactory.registerSingleton( "smartPrincipalManager", smartEngine.getSmartPrincipalManager()); - beanFactory.registerSingleton("accessCountTableManager", - smartEngine.getStatesManager().getAccessCountTableManager()); + beanFactory.registerSingleton("dbFileAccessManager", + smartEngine.getStatesManager().getFileAccessManager()); } } diff --git a/smart-web-server/src/main/java/org/smartdata/server/controller/FilesControllerDelegate.java b/smart-web-server/src/main/java/org/smartdata/server/controller/FilesControllerDelegate.java index d9371e25d88..b52affc6276 100644 --- a/smart-web-server/src/main/java/org/smartdata/server/controller/FilesControllerDelegate.java +++ b/smart-web-server/src/main/java/org/smartdata/server/controller/FilesControllerDelegate.java @@ -18,7 +18,7 @@ package org.smartdata.server.controller; import lombok.RequiredArgsConstructor; -import org.smartdata.metastore.dao.accesscount.AccessCountTableManager; +import org.smartdata.metastore.accesscount.FileAccessManager; import org.smartdata.metastore.model.SearchResult; import org.smartdata.metastore.queries.PageRequest; import org.smartdata.metastore.queries.sort.FileAccessInfoSortField; @@ -44,7 +44,7 @@ @RequiredArgsConstructor public class FilesControllerDelegate implements FilesApiDelegate { - private final AccessCountTableManager accessCountTableManager; + private final FileAccessManager fileAccessManager; private final FileAccessInfoMapper fileInfoMapper; private final FileAccessInfoPageRequestMapper pageRequestMapper; private final CachedFilesControllerDelegate cachedFilesControllerDelegate; @@ -72,7 +72,7 @@ public FileAccessCountsDto getAccessCounts(PageRequestDto pageRequestDto, lastAccessedTime); SearchResult searchResult = - accessCountTableManager.search(searchRequest, pageRequest); + fileAccessManager.search(searchRequest, pageRequest); return fileInfoMapper.toFileAccessCountsDto(searchResult); } } diff --git a/smart-web-server/src/main/java/org/smartdata/server/mappers/FileAccessInfoMapper.java b/smart-web-server/src/main/java/org/smartdata/server/mappers/FileAccessInfoMapper.java index bf9b58d6594..299beb11a2e 100644 --- a/smart-web-server/src/main/java/org/smartdata/server/mappers/FileAccessInfoMapper.java +++ b/smart-web-server/src/main/java/org/smartdata/server/mappers/FileAccessInfoMapper.java @@ -37,7 +37,6 @@ public interface FileAccessInfoMapper extends SmartMapper { @Mapping(target = "ids", ignore = true) @Mapping(target = "id", ignore = true) - @Mapping(target = "accessCountTables", ignore = true) FileAccessInfoSearchRequest toSearchRequest(String pathLike, LastAccessedTimeIntervalDto lastAccessedTime); diff --git a/smart-zeppelin/zeppelin-web/src/app/dashboard/views/rules/submit/help.html b/smart-zeppelin/zeppelin-web/src/app/dashboard/views/rules/submit/help.html new file mode 100755 index 00000000000..e69de29bb2d