diff --git a/conf/config.properties b/conf/config.properties index 910cd6c0f..055809dd7 100644 --- a/conf/config.properties +++ b/conf/config.properties @@ -56,6 +56,12 @@ maxCachedPhysicalTaskPerStorage=500 policyClassName=cn.edu.tsinghua.iginx.policy.naive.NaivePolicy #policyClassName=cn.edu.tsinghua.iginx.policy.simple.SimplePolicy +# 是否允许监视器,如果需要动态负载均衡策略和分片清理则必须允许 +enableMonitor=true + +# 监视器数据同步间隔(单位为s),默认为3s +loadBalanceCheckInterval=3 + # 重分片时,新分片的结束时间多加的间距,单位为秒 reshardFragmentTimeMargin=60 @@ -88,6 +94,19 @@ storageGroupValueLimit=200.0 # 是否允许通过环境变量设置参数 enableEnvParameter=false +########################## +### 分片清理配置 +########################## + +# 分片合并的写入负载阈值(小于该阈值的分片将被删除或合并) +fragmentCompactionWriteThreshold=1000 + +# 分片合并的查询负载阈值(小于该阈值的分片将被合并) +fragmentCompactionReadThreshold=1000 + +# 分片合并的查询负载阈值比例(小于该阈值的分片将被合并) +fragmentCompactionReadRatioThreshold=0.1 + #################### ### Rest 服务配置 #################### diff --git a/core/src/main/java/cn/edu/tsinghua/iginx/Iginx.java b/core/src/main/java/cn/edu/tsinghua/iginx/Iginx.java index ff5b1034f..149210435 100644 --- a/core/src/main/java/cn/edu/tsinghua/iginx/Iginx.java +++ b/core/src/main/java/cn/edu/tsinghua/iginx/Iginx.java @@ -20,6 +20,7 @@ import cn.edu.tsinghua.iginx.conf.Config; import cn.edu.tsinghua.iginx.conf.ConfigDescriptor; +import cn.edu.tsinghua.iginx.monitor.MonitorManager; import cn.edu.tsinghua.iginx.mqtt.MQTTService; import cn.edu.tsinghua.iginx.rest.RestServer; import cn.edu.tsinghua.iginx.thrift.IService; @@ -45,6 +46,9 @@ public static void main(String[] args) throws Exception { if (config.isEnableMQTT()) { new Thread(MQTTService.getInstance()).start(); } + if (config.isEnableMonitor()) { + new Thread(MonitorManager.getInstance()).start(); + } Iginx iginx = new Iginx(); iginx.startServer(); } diff --git a/core/src/main/java/cn/edu/tsinghua/iginx/compaction/Compaction.java b/core/src/main/java/cn/edu/tsinghua/iginx/compaction/Compaction.java new file mode 100644 index 000000000..394113e64 --- /dev/null +++ b/core/src/main/java/cn/edu/tsinghua/iginx/compaction/Compaction.java @@ -0,0 +1,124 @@ +package cn.edu.tsinghua.iginx.compaction; + +import cn.edu.tsinghua.iginx.engine.physical.PhysicalEngine; +import cn.edu.tsinghua.iginx.engine.physical.PhysicalEngineImpl; +import cn.edu.tsinghua.iginx.engine.physical.exception.PhysicalException; +import cn.edu.tsinghua.iginx.engine.shared.TimeRange; +import cn.edu.tsinghua.iginx.engine.shared.data.read.RowStream; +import cn.edu.tsinghua.iginx.engine.shared.operator.Delete; +import cn.edu.tsinghua.iginx.engine.shared.operator.Migration; +import cn.edu.tsinghua.iginx.engine.shared.operator.ShowTimeSeries; +import cn.edu.tsinghua.iginx.engine.shared.source.FragmentSource; +import cn.edu.tsinghua.iginx.engine.shared.source.GlobalSource; +import cn.edu.tsinghua.iginx.metadata.DefaultMetaManager; +import cn.edu.tsinghua.iginx.metadata.IMetaManager; +import cn.edu.tsinghua.iginx.metadata.entity.FragmentMeta; +import cn.edu.tsinghua.iginx.metadata.entity.StorageUnitMeta; + +import java.util.*; + +public abstract class Compaction { + protected final static PhysicalEngine physicalEngine = PhysicalEngineImpl.getInstance(); + protected final IMetaManager metaManager = DefaultMetaManager.getInstance(); + + public abstract boolean needCompaction() throws Exception; + + public abstract void compact() throws Exception; + + protected List> packFragmentsByGroup(List fragmentMetas) { + // 排序以减少算法时间复杂度 + fragmentMetas.sort((o1, o2) -> { + // 先按照时间维度排序,再按照时间序列维度排序 + if (o1.getTimeInterval().getStartTime() == o2.getTimeInterval().getStartTime()) { + return o1.getTsInterval().getStartTimeSeries().compareTo(o2.getTsInterval().getEndTimeSeries()); + } else { + // 所有分片在时间维度上是统一的,因此只需要根据起始时间排序即可 + return Long.compare(o1.getTimeInterval().getStartTime(), o2.getTimeInterval().getStartTime()); + } + }); + + // 对筛选出来要合并的所有分片按连通性进行分组(同一组中的分片可以合并) + List> result = new ArrayList<>(); + List lastFragmentGroup = new ArrayList<>(); + FragmentMeta lastFragment = null; + for (FragmentMeta fragmentMeta : fragmentMetas) { + if (lastFragment == null) { + lastFragmentGroup.add(fragmentMeta); + } else { + if (isNext(lastFragment, fragmentMeta)) { + lastFragmentGroup.add(fragmentMeta); + } else { + if (lastFragmentGroup.size() > 1) { + result.add(lastFragmentGroup); + } + lastFragmentGroup = new ArrayList<>(); + } + } + lastFragment = fragmentMeta; + } + return result; + } + + private boolean isNext(FragmentMeta firstFragment, FragmentMeta secondFragment) { + return firstFragment.getTimeInterval().equals(secondFragment.getTimeInterval()) && firstFragment.getTsInterval().getEndTimeSeries().equals(secondFragment.getTsInterval().getStartTimeSeries()); + } + + protected void compactFragmentGroupToTargetStorageUnit(List fragmentGroup, StorageUnitMeta targetStorageUnit, long totalPoints) throws PhysicalException { + String startTimeseries = fragmentGroup.get(0).getTsInterval().getStartTimeSeries(); + String endTimeseries = fragmentGroup.get(0).getTsInterval().getEndTimeSeries(); + long startTime = fragmentGroup.get(0).getTimeInterval().getStartTime(); + long endTime = fragmentGroup.get(0).getTimeInterval().getEndTime(); + + for (FragmentMeta fragmentMeta : fragmentGroup) { + String storageUnitId = fragmentMeta.getMasterStorageUnitId(); + if (!storageUnitId.equals(targetStorageUnit.getId())) { + // 重写该分片的数据 + Set pathRegexSet = new HashSet<>(); + ShowTimeSeries showTimeSeries = new ShowTimeSeries(new GlobalSource(), pathRegexSet, null, + Integer.MAX_VALUE, 0); + RowStream rowStream = physicalEngine.execute(showTimeSeries); + SortedSet pathSet = new TreeSet<>(); + rowStream.getHeader().getFields().forEach(field -> { + String timeSeries = field.getName(); + if (timeSeries.contains("{") && timeSeries.contains("}")) { + timeSeries = timeSeries.split("\\{")[0]; + } + if (fragmentMeta.getTsInterval().isContain(timeSeries)) { + pathSet.add(timeSeries); + } + }); + Migration migration = new Migration(new GlobalSource(), fragmentMeta, new ArrayList<>(pathSet), targetStorageUnit); + physicalEngine.execute(migration); + // 更新存储点数信息 + metaManager.updateFragmentPoints(fragmentMeta, totalPoints); + } + } + // TODO add write lock + // 创建新分片 + FragmentMeta newFragment = new FragmentMeta(startTimeseries, endTimeseries, startTime, endTime, targetStorageUnit); + DefaultMetaManager.getInstance().addFragment(newFragment); + + for (FragmentMeta fragmentMeta : fragmentGroup) { + String storageUnitId = fragmentMeta.getMasterStorageUnitId(); + if (!storageUnitId.equals(targetStorageUnit.getId())) { + // 删除原分片元数据信息 + DefaultMetaManager.getInstance().removeFragment(fragmentMeta); + } + } + // TODO release write lock + + for (FragmentMeta fragmentMeta : fragmentGroup) { + String storageUnitId = fragmentMeta.getMasterStorageUnitId(); + if (!storageUnitId.equals(targetStorageUnit.getId())) { + // 删除原分片节点数据 + List paths = new ArrayList<>(); + paths.add(fragmentMeta.getMasterStorageUnitId() + "*"); + List timeRanges = new ArrayList<>(); + timeRanges.add(new TimeRange(fragmentMeta.getTimeInterval().getStartTime(), true, + fragmentMeta.getTimeInterval().getEndTime(), false)); + Delete delete = new Delete(new FragmentSource(fragmentMeta), timeRanges, paths, null); + physicalEngine.execute(delete); + } + } + } +} \ No newline at end of file diff --git a/core/src/main/java/cn/edu/tsinghua/iginx/compaction/CompactionManager.java b/core/src/main/java/cn/edu/tsinghua/iginx/compaction/CompactionManager.java new file mode 100644 index 000000000..b7217d800 --- /dev/null +++ b/core/src/main/java/cn/edu/tsinghua/iginx/compaction/CompactionManager.java @@ -0,0 +1,36 @@ +package cn.edu.tsinghua.iginx.compaction; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; + +public class CompactionManager { + + private static final Logger logger = LoggerFactory.getLogger(CompactionManager.class); + + private static final List compactionList = new ArrayList<>(); + + static { + compactionList.add(new FragmentDeletionCompaction()); + compactionList.add(new LowWriteFragmentCompaction()); + compactionList.add(new LowAccessFragmentCompaction()); + } + + private static final CompactionManager instance = new CompactionManager(); + + public static CompactionManager getInstance() { + return instance; + } + + public void clearFragment() throws Exception { + logger.info("start to compact fragments"); + for (Compaction compaction : compactionList) { + if (compaction.needCompaction()) { + compaction.compact(); + } + } + logger.info("end compact fragments"); + } +} diff --git a/core/src/main/java/cn/edu/tsinghua/iginx/compaction/FragmentDeletionCompaction.java b/core/src/main/java/cn/edu/tsinghua/iginx/compaction/FragmentDeletionCompaction.java new file mode 100644 index 000000000..6a1844cce --- /dev/null +++ b/core/src/main/java/cn/edu/tsinghua/iginx/compaction/FragmentDeletionCompaction.java @@ -0,0 +1,61 @@ +package cn.edu.tsinghua.iginx.compaction; + +import cn.edu.tsinghua.iginx.engine.physical.exception.PhysicalException; +import cn.edu.tsinghua.iginx.engine.shared.TimeRange; +import cn.edu.tsinghua.iginx.engine.shared.operator.Delete; +import cn.edu.tsinghua.iginx.engine.shared.source.FragmentSource; +import cn.edu.tsinghua.iginx.metadata.entity.FragmentMeta; +import cn.edu.tsinghua.iginx.utils.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.*; + +public class FragmentDeletionCompaction extends Compaction { + + private static final Logger logger = LoggerFactory.getLogger(FragmentDeletionCompaction.class); + private List toDeletionFragments; + + @Override + public boolean needCompaction() throws Exception { + //集中信息(初版主要是统计分区热度) + Pair, Map> fragmentHeatPair = metaManager + .loadFragmentHeat(); + Map fragmentHeatWriteMap = fragmentHeatPair.getK(); + Map fragmentHeatReadMap = fragmentHeatPair.getV(); + if (fragmentHeatWriteMap == null) { + fragmentHeatWriteMap = new HashMap<>(); + } + if (fragmentHeatReadMap == null) { + fragmentHeatReadMap = new HashMap<>(); + } + + long totalHeats = 0; + for (Map.Entry fragmentHeatReadEntry : fragmentHeatReadMap.entrySet()) { + totalHeats += fragmentHeatReadEntry.getValue(); + } + double limitReadHeats = totalHeats * 1.0 / fragmentHeatReadMap.size(); + + // 判断是否要删除可定制化副本生成的冗余分片 + // TODO + + return !toDeletionFragments.isEmpty(); + } + + @Override + public void compact() throws PhysicalException { + for (FragmentMeta fragmentMeta : toDeletionFragments) { + // 删除可定制化副本分片元数据 + // TODO + + // 删除节点数据 + List paths = new ArrayList<>(); + paths.add(fragmentMeta.getMasterStorageUnitId() + "*"); + List timeRanges = new ArrayList<>(); + timeRanges.add(new TimeRange(fragmentMeta.getTimeInterval().getStartTime(), true, + fragmentMeta.getTimeInterval().getEndTime(), false)); + Delete delete = new Delete(new FragmentSource(fragmentMeta), timeRanges, paths, null); + physicalEngine.execute(delete); + } + } +} diff --git a/core/src/main/java/cn/edu/tsinghua/iginx/compaction/LowAccessFragmentCompaction.java b/core/src/main/java/cn/edu/tsinghua/iginx/compaction/LowAccessFragmentCompaction.java new file mode 100644 index 000000000..fdbc480de --- /dev/null +++ b/core/src/main/java/cn/edu/tsinghua/iginx/compaction/LowAccessFragmentCompaction.java @@ -0,0 +1,100 @@ +package cn.edu.tsinghua.iginx.compaction; + +import cn.edu.tsinghua.iginx.conf.ConfigDescriptor; +import cn.edu.tsinghua.iginx.metadata.entity.FragmentMeta; +import cn.edu.tsinghua.iginx.metadata.entity.StorageUnitMeta; +import cn.edu.tsinghua.iginx.utils.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class LowAccessFragmentCompaction extends Compaction { + + private static final Logger logger = LoggerFactory.getLogger(LowAccessFragmentCompaction.class); + private static final long fragmentCompactionWriteThreshold = ConfigDescriptor.getInstance().getConfig().getFragmentCompactionWriteThreshold(); + private static final long fragmentCompactionReadThreshold = ConfigDescriptor.getInstance().getConfig().getFragmentCompactionReadThreshold(); + + private List> toCompactFragmentGroups; + + @Override + public boolean needCompaction() throws Exception { + //集中信息(初版主要是统计分区热度) + Pair, Map> fragmentHeatPair = metaManager + .loadFragmentHeat(); + Map fragmentHeatWriteMap = fragmentHeatPair.getK(); + Map fragmentHeatReadMap = fragmentHeatPair.getV(); + if (fragmentHeatWriteMap == null) { + fragmentHeatWriteMap = new HashMap<>(); + } + if (fragmentHeatReadMap == null) { + fragmentHeatReadMap = new HashMap<>(); + } + + List fragmentMetaSet = metaManager.getFragments(); + + List candidateFragments = new ArrayList<>(); + // 判断是否要合并不再被写入的的历史分片 + for (FragmentMeta fragmentMeta : fragmentMetaSet) { + long writeLoad = fragmentHeatWriteMap.getOrDefault(fragmentMeta, 0L); + long readLoad = fragmentHeatReadMap.getOrDefault(fragmentMeta, 0L); + if (fragmentMeta.getTimeInterval().getEndTime() != Long.MAX_VALUE && writeLoad < fragmentCompactionWriteThreshold && readLoad <= fragmentCompactionReadThreshold) { + candidateFragments.add(fragmentMeta); + } + } + + toCompactFragmentGroups = packFragmentsByGroup(candidateFragments); + + return !toCompactFragmentGroups.isEmpty(); + } + + @Override + public void compact() throws Exception { + logger.info("start to compact low access fragments"); + Map fragmentMetaPointsMap = metaManager.loadFragmentPoints(); + + // 优先存储到点数最少的节点上(剩余磁盘空间较大) + Map storageEnginePointsMap = new HashMap<>(); + for (Map.Entry fragmentMetaPointsEntry : fragmentMetaPointsMap.entrySet()) { + FragmentMeta fragmentMeta = fragmentMetaPointsEntry.getKey(); + long storageEngineId = fragmentMeta.getMasterStorageUnit().getStorageEngineId(); + long points = fragmentMetaPointsEntry.getValue(); + long allPoints = storageEnginePointsMap.getOrDefault(storageEngineId, 0L); + allPoints += points; + storageEnginePointsMap.put(storageEngineId, allPoints); + } + long minPoints = Long.MAX_VALUE; + long minStorageEngineId = 0; + for (Map.Entry storageEnginePointsEntry : storageEnginePointsMap.entrySet()) { + if (minPoints > storageEnginePointsEntry.getValue()) { + minStorageEngineId = storageEnginePointsEntry.getKey(); + minPoints = storageEnginePointsEntry.getValue(); + } + } + + for (List fragmentGroup : toCompactFragmentGroups) { + // 分别计算每个du的数据量,取其中数据量最多的du作为目标合并du + StorageUnitMeta maxStorageUnitMeta = fragmentGroup.get(0).getMasterStorageUnit(); + long maxStorageUnitPoint = 0; + long totalPoints = 0; + Map storageUnitPointsMap = new HashMap<>(); + for (FragmentMeta fragmentMeta : fragmentGroup) { + // 优先按照节点当前存储的点数最小做选择 + if (fragmentMeta.getMasterStorageUnit().getStorageEngineId() == minStorageEngineId) { + long pointsNum = storageUnitPointsMap.getOrDefault(fragmentMeta.getMasterStorageUnitId(), 0L); + pointsNum += fragmentMetaPointsMap.getOrDefault(fragmentMeta, 0L); + if (pointsNum > maxStorageUnitPoint) { + maxStorageUnitMeta = fragmentMeta.getMasterStorageUnit(); + } + storageUnitPointsMap.put(fragmentMeta.getMasterStorageUnitId(), pointsNum); + } + totalPoints += fragmentMetaPointsMap.getOrDefault(fragmentMeta, 0L); + } + + compactFragmentGroupToTargetStorageUnit(fragmentGroup, maxStorageUnitMeta, totalPoints); + } + } +} diff --git a/core/src/main/java/cn/edu/tsinghua/iginx/compaction/LowWriteFragmentCompaction.java b/core/src/main/java/cn/edu/tsinghua/iginx/compaction/LowWriteFragmentCompaction.java new file mode 100644 index 000000000..cd3a39074 --- /dev/null +++ b/core/src/main/java/cn/edu/tsinghua/iginx/compaction/LowWriteFragmentCompaction.java @@ -0,0 +1,81 @@ +package cn.edu.tsinghua.iginx.compaction; + +import cn.edu.tsinghua.iginx.conf.ConfigDescriptor; +import cn.edu.tsinghua.iginx.metadata.entity.FragmentMeta; +import cn.edu.tsinghua.iginx.metadata.entity.StorageUnitMeta; +import cn.edu.tsinghua.iginx.utils.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.*; + +public class LowWriteFragmentCompaction extends Compaction { + + private static final Logger logger = LoggerFactory.getLogger(LowWriteFragmentCompaction.class); + private static final long fragmentCompactionWriteThreshold = ConfigDescriptor.getInstance().getConfig().getFragmentCompactionWriteThreshold(); + private static final double fragmentCompactionReadRatioThreshold = ConfigDescriptor.getInstance().getConfig().getFragmentCompactionReadRatioThreshold(); + private static final long fragmentCompactionReadThreshold = ConfigDescriptor.getInstance().getConfig().getFragmentCompactionReadThreshold(); + + private List> toCompactFragmentGroups; + + @Override + public boolean needCompaction() throws Exception { + //集中信息(初版主要是统计分区热度) + Pair, Map> fragmentHeatPair = metaManager + .loadFragmentHeat(); + Map fragmentHeatWriteMap = fragmentHeatPair.getK(); + Map fragmentHeatReadMap = fragmentHeatPair.getV(); + if (fragmentHeatWriteMap == null) { + fragmentHeatWriteMap = new HashMap<>(); + } + if (fragmentHeatReadMap == null) { + fragmentHeatReadMap = new HashMap<>(); + } + + List fragmentMetaSet = metaManager.getFragments(); + + long totalHeats = 0; + for (Map.Entry fragmentHeatReadEntry : fragmentHeatReadMap.entrySet()) { + totalHeats += fragmentHeatReadEntry.getValue(); + } + double averageReadHeats = totalHeats * 1.0 / fragmentHeatReadMap.size(); + + List candidateFragments = new ArrayList<>(); + // 判断是否要合并不再被写入的的历史分片 + for (FragmentMeta fragmentMeta : fragmentMetaSet) { + long writeLoad = fragmentHeatWriteMap.getOrDefault(fragmentMeta, 0L); + long readLoad = fragmentHeatReadMap.getOrDefault(fragmentMeta, 0L); + if (fragmentMeta.getTimeInterval().getEndTime() != Long.MAX_VALUE && writeLoad < fragmentCompactionWriteThreshold && readLoad < averageReadHeats * fragmentCompactionReadRatioThreshold && readLoad > fragmentCompactionReadThreshold) { + candidateFragments.add(fragmentMeta); + } + } + + toCompactFragmentGroups = packFragmentsByGroup(candidateFragments); + + return !toCompactFragmentGroups.isEmpty(); + } + + @Override + public void compact() throws Exception { + logger.info("start to compact low write fragments"); + Map fragmentMetaPointsMap = metaManager.loadFragmentPoints(); + for (List fragmentGroup : toCompactFragmentGroups) { + // 分别计算每个du的数据量,取其中数据量最多的du作为目标合并du + StorageUnitMeta maxStorageUnitMeta = fragmentGroup.get(0).getMasterStorageUnit(); + long maxStorageUnitPoint = 0; + Map storageUnitPointsMap = new HashMap<>(); + long totalPoints = 0; + for (FragmentMeta fragmentMeta : fragmentGroup) { + long pointsNum = storageUnitPointsMap.getOrDefault(fragmentMeta.getMasterStorageUnitId(), 0L); + pointsNum += fragmentMetaPointsMap.getOrDefault(fragmentMeta, 0L); + if (pointsNum > maxStorageUnitPoint) { + maxStorageUnitMeta = fragmentMeta.getMasterStorageUnit(); + } + storageUnitPointsMap.put(fragmentMeta.getMasterStorageUnitId(), pointsNum); + totalPoints += fragmentMetaPointsMap.getOrDefault(fragmentMeta, 0L); + } + + compactFragmentGroupToTargetStorageUnit(fragmentGroup, maxStorageUnitMeta, totalPoints); + } + } +} diff --git a/core/src/main/java/cn/edu/tsinghua/iginx/conf/Config.java b/core/src/main/java/cn/edu/tsinghua/iginx/conf/Config.java index ac06fd0ce..ba483a6eb 100644 --- a/core/src/main/java/cn/edu/tsinghua/iginx/conf/Config.java +++ b/core/src/main/java/cn/edu/tsinghua/iginx/conf/Config.java @@ -55,6 +55,16 @@ public class Config { private String policyClassName = "cn.edu.tsinghua.iginx.policy.naive.NativePolicy"; + private boolean enableMonitor = true; + + private int loadBalanceCheckInterval = 3; + + private long fragmentCompactionWriteThreshold = 1000; + + private long fragmentCompactionReadThreshold = 1000; + + private double fragmentCompactionReadRatioThreshold = 0.1; + private long reshardFragmentTimeMargin = 60; private String migrationPolicyClassName = "cn.edu.tsinghua.iginx.migration.GreedyMigrationPolicy"; @@ -297,6 +307,46 @@ public void setPolicyClassName(String policyClassName) { this.policyClassName = policyClassName; } + public boolean isEnableMonitor() { + return enableMonitor; + } + + public void setEnableMonitor(boolean enableMonitor) { + this.enableMonitor = enableMonitor; + } + + public int getLoadBalanceCheckInterval() { + return loadBalanceCheckInterval; + } + + public void setLoadBalanceCheckInterval(int loadBalanceCheckInterval) { + this.loadBalanceCheckInterval = loadBalanceCheckInterval; + } + + public long getFragmentCompactionWriteThreshold() { + return fragmentCompactionWriteThreshold; + } + + public void setFragmentCompactionWriteThreshold(long fragmentCompactionWriteThreshold) { + this.fragmentCompactionWriteThreshold = fragmentCompactionWriteThreshold; + } + + public long getFragmentCompactionReadThreshold() { + return fragmentCompactionReadThreshold; + } + + public void setFragmentCompactionReadThreshold(long fragmentCompactionReadThreshold) { + this.fragmentCompactionReadThreshold = fragmentCompactionReadThreshold; + } + + public double getFragmentCompactionReadRatioThreshold() { + return fragmentCompactionReadRatioThreshold; + } + + public void setFragmentCompactionReadRatioThreshold(double fragmentCompactionReadRatioThreshold) { + this.fragmentCompactionReadRatioThreshold = fragmentCompactionReadRatioThreshold; + } + public long getMigrationBatchSize() { return migrationBatchSize; } diff --git a/core/src/main/java/cn/edu/tsinghua/iginx/conf/ConfigDescriptor.java b/core/src/main/java/cn/edu/tsinghua/iginx/conf/ConfigDescriptor.java index 1beebe533..b3322ab96 100644 --- a/core/src/main/java/cn/edu/tsinghua/iginx/conf/ConfigDescriptor.java +++ b/core/src/main/java/cn/edu/tsinghua/iginx/conf/ConfigDescriptor.java @@ -59,9 +59,9 @@ private void loadPropsFromFile() { config.setUsername(properties.getProperty("username", "root")); config.setPassword(properties.getProperty("password", "root")); config.setZookeeperConnectionString(properties.getProperty("zookeeperConnectionString", - "127.0.0.1:2181")); + "127.0.0.1:2181")); config.setStorageEngineList(properties.getProperty("storageEngineList", - "127.0.0.1#6667#iotdb11#username=root#password=root#sessionPoolSize=20#dataDir=/path/to/your/data/")); + "127.0.0.1#6667#iotdb11#username=root#password=root#sessionPoolSize=20#dataDir=/path/to/your/data/")); config.setMaxAsyncRetryTimes(Integer.parseInt(properties.getProperty("maxAsyncRetryTimes", "3"))); config.setSyncExecuteThreadPool(Integer.parseInt(properties.getProperty("syncExecuteThreadPool", "60"))); config.setAsyncExecuteThreadPool(Integer.parseInt(properties.getProperty("asyncExecuteThreadPool", "20"))); @@ -71,6 +71,11 @@ private void loadPropsFromFile() { //,opentsdb=cn.edu.tsinghua.iginx.opentsdb.OpenTSDBStorage,timescaledb=cn.edu.tsinghua.iginx.timescaledb.TimescaleDBStorage,postgresql=cn.edu.tsinghua.iginx.postgresql.PostgreSQLStorage config.setPolicyClassName(properties.getProperty("policyClassName", "cn.edu.tsinghua.iginx.policy.naive.NativePolicy")); + config.setEnableMonitor(Boolean.parseBoolean(properties.getProperty("enableMonitor", "true"))); + config.setLoadBalanceCheckInterval(Integer.parseInt(properties.getProperty("loadBalanceCheckInterval", "10"))); + config.setFragmentCompactionWriteThreshold(Long.parseLong(properties.getProperty("fragmentCompactionWriteThreshold", "1000"))); + config.setFragmentCompactionReadThreshold(Long.parseLong(properties.getProperty("fragmentCompactionReadThreshold", "1000"))); + config.setFragmentCompactionReadRatioThreshold(Long.parseLong(properties.getProperty("fragmentCompactionReadRatioThreshold", "0.1"))); config.setMigrationBatchSize(Integer.parseInt(properties.getProperty("migrationBatchSize", "100"))); config.setReshardFragmentTimeMargin(Long.parseLong(properties.getProperty("reshardFragmentTimeMargin", "60"))); config.setMaxReshardFragmentsNum(Integer.parseInt(properties.getProperty("maxReshardFragmentsNum", "3"))); @@ -215,9 +220,9 @@ private void loadUDFListFromFile() { while ((line = bufferedReader.readLine()) != null) { line = line.trim(); if (line.toLowerCase().startsWith(Constants.UDAF) || - line.toLowerCase().startsWith(Constants.UDTF) || - line.toLowerCase().startsWith(Constants.UDSF) || - line.toLowerCase().startsWith(Constants.TRANSFORM)) { + line.toLowerCase().startsWith(Constants.UDTF) || + line.toLowerCase().startsWith(Constants.UDSF) || + line.toLowerCase().startsWith(Constants.TRANSFORM)) { config.getUdfList().add(line); } } diff --git a/core/src/main/java/cn/edu/tsinghua/iginx/engine/physical/PhysicalEngineImpl.java b/core/src/main/java/cn/edu/tsinghua/iginx/engine/physical/PhysicalEngineImpl.java index 257bc73e1..4491fdf90 100644 --- a/core/src/main/java/cn/edu/tsinghua/iginx/engine/physical/PhysicalEngineImpl.java +++ b/core/src/main/java/cn/edu/tsinghua/iginx/engine/physical/PhysicalEngineImpl.java @@ -156,9 +156,6 @@ public RowStream execute(Operator root) throws PhysicalException { } insertDataByBatch(timestampList, valuesList, bitmapList, bitmapBufferList, toMigrateFragment, selectResultPaths, selectResultTypes, targetStorageUnitMeta.getId()); - - // 设置分片现在所属的du - toMigrateFragment.setMasterStorageUnit(targetStorageUnitMeta); return selectResult.getRowStream(); } else { GlobalPhysicalTask task = new GlobalPhysicalTask(root); diff --git a/core/src/main/java/cn/edu/tsinghua/iginx/engine/shared/operator/Migration.java b/core/src/main/java/cn/edu/tsinghua/iginx/engine/shared/operator/Migration.java index bcf6a7333..515faec8e 100644 --- a/core/src/main/java/cn/edu/tsinghua/iginx/engine/shared/operator/Migration.java +++ b/core/src/main/java/cn/edu/tsinghua/iginx/engine/shared/operator/Migration.java @@ -4,50 +4,37 @@ import cn.edu.tsinghua.iginx.engine.shared.source.GlobalSource; import cn.edu.tsinghua.iginx.metadata.entity.FragmentMeta; import cn.edu.tsinghua.iginx.metadata.entity.StorageUnitMeta; + import java.util.List; public class Migration extends AbstractUnaryOperator { - private final long sourceStorageEngineId; - private final long targetStorageEngineId; - private final FragmentMeta fragmentMeta; - private final List paths; - private final StorageUnitMeta targetStorageUnitMeta; - - public Migration(GlobalSource source, long sourceStorageEngineId, long targetStorageEngineId, - FragmentMeta fragmentMeta, - List paths, StorageUnitMeta targetStorageUnitMeta) { - super(OperatorType.Migration, source); - this.sourceStorageEngineId = sourceStorageEngineId; - this.targetStorageEngineId = targetStorageEngineId; - this.fragmentMeta = fragmentMeta; - this.paths = paths; - this.targetStorageUnitMeta = targetStorageUnitMeta; - } - - public long getSourceStorageEngineId() { - return sourceStorageEngineId; - } - - public long getTargetStorageEngineId() { - return targetStorageEngineId; - } - - public FragmentMeta getFragmentMeta() { - return fragmentMeta; - } - - public StorageUnitMeta getTargetStorageUnitMeta() { - return targetStorageUnitMeta; - } - - public List getPaths() { - return paths; - } - - @Override - public Operator copy() { - return new Migration((GlobalSource) getSource().copy(), sourceStorageEngineId, - targetStorageEngineId, fragmentMeta, paths, targetStorageUnitMeta); - } + private final FragmentMeta fragmentMeta; + private final List paths; + private final StorageUnitMeta targetStorageUnitMeta; + + public Migration(GlobalSource source, FragmentMeta fragmentMeta, + List paths, StorageUnitMeta targetStorageUnitMeta) { + super(OperatorType.Migration, source); + this.fragmentMeta = fragmentMeta; + this.paths = paths; + this.targetStorageUnitMeta = targetStorageUnitMeta; + } + + public FragmentMeta getFragmentMeta() { + return fragmentMeta; + } + + public StorageUnitMeta getTargetStorageUnitMeta() { + return targetStorageUnitMeta; + } + + public List getPaths() { + return paths; + } + + @Override + public Operator copy() { + return new Migration((GlobalSource) getSource().copy(), fragmentMeta, paths, targetStorageUnitMeta); + } } diff --git a/core/src/main/java/cn/edu/tsinghua/iginx/metadata/DefaultMetaManager.java b/core/src/main/java/cn/edu/tsinghua/iginx/metadata/DefaultMetaManager.java index f73feca3a..052b65c59 100644 --- a/core/src/main/java/cn/edu/tsinghua/iginx/metadata/DefaultMetaManager.java +++ b/core/src/main/java/cn/edu/tsinghua/iginx/metadata/DefaultMetaManager.java @@ -30,6 +30,9 @@ import cn.edu.tsinghua.iginx.metadata.storage.IMetaStorage; import cn.edu.tsinghua.iginx.metadata.storage.etcd.ETCDMetaStorage; import cn.edu.tsinghua.iginx.metadata.storage.zk.ZooKeeperMetaStorage; +import cn.edu.tsinghua.iginx.metadata.utils.ReshardStatus; +import cn.edu.tsinghua.iginx.monitor.HotSpotMonitor; +import cn.edu.tsinghua.iginx.monitor.RequestsMonitor; import cn.edu.tsinghua.iginx.policy.simple.TimeSeriesCalDO; import cn.edu.tsinghua.iginx.sql.statement.InsertStatement; import cn.edu.tsinghua.iginx.thrift.AuthType; @@ -37,14 +40,19 @@ import cn.edu.tsinghua.iginx.utils.Pair; import cn.edu.tsinghua.iginx.utils.SnowFlakeUtils; import cn.edu.tsinghua.iginx.utils.StringUtils; + import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.*; import java.util.stream.Collectors; +import static cn.edu.tsinghua.iginx.metadata.utils.ReshardStatus.*; +import static cn.edu.tsinghua.iginx.metadata.utils.ReshardStatus.NON_RESHARDING; + public class DefaultMetaManager implements IMetaManager { private static final Logger logger = LoggerFactory.getLogger(DefaultMetaManager.class); @@ -60,6 +68,12 @@ public class DefaultMetaManager implements IMetaManager { private AtomicLong maxActiveEndTime = new AtomicLong(-1L); private AtomicInteger maxActiveEndTimeStatisticsCounter = new AtomicInteger(0); + // 重分片状态 + private ReshardStatus reshardStatus = NON_RESHARDING; + + // 在重分片过程中,是否为提出者 + private boolean isProposer = false; + private DefaultMetaManager() { cache = DefaultMetaCache.getInstance(); @@ -97,6 +111,8 @@ private DefaultMetaManager() { initUser(); initTransform(); initMaxActiveEndTimeStatistics(); + initReshardStatus(); + initReshardCounter(); } catch (MetaStorageException e) { logger.error("init meta manager error: ", e); System.exit(-1); @@ -122,10 +138,69 @@ private void initMaxActiveEndTimeStatistics() throws MetaStorageException { updateMaxActiveEndTime(endTime); int updatedCounter = maxActiveEndTimeStatisticsCounter.incrementAndGet(); logger.info("iginx node {} increment max active end time statistics counter {}", this.id, - updatedCounter); + updatedCounter); + }); + } + + private void initReshardStatus() throws MetaStorageException { + storage.registerReshardStatusHook(status -> { + try { + reshardStatus = status; + if (reshardStatus.equals(EXECUTING)) { + storage.lockMaxActiveEndTimeStatistics(); + storage.addOrUpdateMaxActiveEndTimeStatistics(maxActiveEndTime.get()); + storage.releaseMaxActiveEndTimeStatistics(); + + storage.lockReshardCounter(); + storage.incrementReshardCounter(); + storage.releaseReshardCounter(); + } + if (reshardStatus.equals(NON_RESHARDING)) { + if (isProposer) { + logger.info("iginx node {}(proposer) finish to reshard", id); + } else { + logger.info("iginx node {} finish to reshard", id); + } + + isProposer = false; + maxActiveEndTimeStatisticsCounter.set(0); + } + } catch (MetaStorageException e) { + logger.error("encounter error when switching reshard status: ", e); + } }); + storage.lockReshardStatus(); + storage.removeReshardStatus(); + storage.releaseReshardStatus(); } + private void initReshardCounter() throws MetaStorageException { + storage.registerReshardCounterChangeHook(counter -> { + try { + if (counter <= 0) { + return; + } + if (isProposer && counter == getIginxList().size() - 1) { + storage.lockReshardCounter(); + storage.resetReshardCounter(); + storage.releaseReshardCounter(); + + if (reshardStatus == EXECUTING) { + storage.lockReshardStatus(); + storage.updateReshardStatus(NON_RESHARDING); + storage.releaseReshardStatus(); + } + } + } catch (MetaStorageException e) { + logger.error("encounter error when updating reshard counter: ", e); + } + }); + storage.lockReshardCounter(); + storage.removeReshardCounter(); + storage.releaseReshardCounter(); + } + + private void initIginx() throws MetaStorageException { storage.registerIginxChangeHook((id, iginx) -> { if (iginx == null) { @@ -138,7 +213,7 @@ private void initIginx() throws MetaStorageException { cache.addIginx(iginx); } IginxMeta iginx = new IginxMeta(0L, ConfigDescriptor.getInstance().getConfig().getIp(), - ConfigDescriptor.getInstance().getConfig().getPort(), null); + ConfigDescriptor.getInstance().getConfig().getPort(), null); id = storage.registerIginx(iginx); SnowFlakeUtils.init(id); } @@ -262,7 +337,7 @@ private void initPolicy() { storage.registerVersionChangeHook((version, num) -> { double sum = cache.getSumFromTimeSeries(); Map timeseriesData = cache.getMaxValueFromTimeSeries().stream(). - collect(Collectors.toMap(TimeSeriesCalDO::getTimeSeries, TimeSeriesCalDO::getValue)); + collect(Collectors.toMap(TimeSeriesCalDO::getTimeSeries, TimeSeriesCalDO::getValue)); double countSum = timeseriesData.values().stream().mapToDouble(Double::doubleValue).sum(); if (countSum > 1e-9) { timeseriesData.forEach((k, v) -> timeseriesData.put(k, v / countSum * sum)); @@ -304,7 +379,7 @@ private void initTransform() throws MetaStorageException { cache.addOrUpdateTransformTask(transformTask); } })); - for (TransformTaskMeta task: storage.loadTransformTask()) { + for (TransformTaskMeta task : storage.loadTransformTask()) { cache.addOrUpdateTransformTask(task); } } @@ -427,7 +502,7 @@ public Map> getFragmentMapByTimeSeriesInterv fragmentsMap = storage.getFragmentMapByTimeSeriesIntervalAndTimeInterval(tsInterval, beforeTimeInterval); updateStorageUnitReference(fragmentsMap); Map> recentFragmentsMap = cache.getFragmentMapByTimeSeriesInterval(tsInterval); - for (TimeSeriesRange ts: recentFragmentsMap.keySet()) { + for (TimeSeriesRange ts : recentFragmentsMap.keySet()) { List fragments = recentFragmentsMap.get(ts); if (fragmentsMap.containsKey(ts)) { fragmentsMap.get(ts).addAll(fragments); @@ -474,7 +549,7 @@ public Map> getFragmentMapByTimeSeriesInterv fragmentsMap = storage.getFragmentMapByTimeSeriesIntervalAndTimeInterval(tsInterval, beforeTimeInterval); updateStorageUnitReference(fragmentsMap); Map> recentFragmentsMap = cache.getFragmentMapByTimeSeriesIntervalAndTimeInterval(tsInterval, timeInterval); - for (TimeSeriesRange ts: recentFragmentsMap.keySet()) { + for (TimeSeriesRange ts : recentFragmentsMap.keySet()) { List fragments = recentFragmentsMap.get(ts); if (fragmentsMap.containsKey(ts)) { fragmentsMap.get(ts).addAll(fragments); @@ -493,7 +568,7 @@ public Map> getFragmentMapByTimeSeriesInterv } private void mergeToFragmentMap(Map> fragmentsMap, List fragmentList) { - for (FragmentMeta fragment: fragmentList) { + for (FragmentMeta fragment : fragmentList) { TimeSeriesRange tsInterval = fragment.getTsInterval(); if (!fragmentsMap.containsKey(tsInterval)) { fragmentsMap.put(tsInterval, new ArrayList<>()); @@ -611,7 +686,7 @@ public boolean createFragmentsAndStorageUnits(List storageUnits @Override public FragmentMeta splitFragmentAndStorageUnit(StorageUnitMeta toAddStorageUnit, - FragmentMeta toAddFragment, FragmentMeta fragment) { + FragmentMeta toAddFragment, FragmentMeta fragment) { try { storage.lockFragment(); storage.lockStorageUnit(); @@ -621,7 +696,7 @@ public FragmentMeta splitFragmentAndStorageUnit(StorageUnitMeta toAddStorageUnit toAddStorageUnit.setCreatedBy(id); String actualName = storage.addStorageUnit(); StorageUnitMeta actualMasterStorageUnit = toAddStorageUnit - .renameStorageUnitMeta(actualName, actualName); + .renameStorageUnitMeta(actualName, actualName); cache.updateStorageUnit(actualMasterStorageUnit); for (StorageUnitHook hook : storageUnitHooks) { hook.onChange(null, actualMasterStorageUnit); @@ -631,7 +706,7 @@ public FragmentMeta splitFragmentAndStorageUnit(StorageUnitMeta toAddStorageUnit slaveStorageUnit.setCreatedBy(id); String slaveActualName = storage.addStorageUnit(); StorageUnitMeta actualSlaveStorageUnit = slaveStorageUnit - .renameStorageUnitMeta(slaveActualName, actualName); + .renameStorageUnitMeta(slaveActualName, actualName); actualMasterStorageUnit.addReplica(actualSlaveStorageUnit); for (StorageUnitHook hook : storageUnitHooks) { hook.onChange(null, actualSlaveStorageUnit); @@ -643,7 +718,7 @@ public FragmentMeta splitFragmentAndStorageUnit(StorageUnitMeta toAddStorageUnit // 结束旧分片 cache.deleteFragmentByTsInterval(fragment.getTsInterval(), fragment); fragment = fragment - .endFragmentMeta(toAddFragment.getTimeInterval().getStartTime()); + .endFragmentMeta(toAddFragment.getTimeInterval().getStartTime()); cache.addFragment(fragment); fragment.setUpdatedBy(id); storage.updateFragment(fragment); @@ -672,6 +747,23 @@ public FragmentMeta splitFragmentAndStorageUnit(StorageUnitMeta toAddStorageUnit return fragment; } + @Override + public void removeFragment(FragmentMeta fragmentMeta) { + try { + storage.lockFragment(); + cache.deleteFragmentByTsInterval(fragmentMeta.getTsInterval(), fragmentMeta); + storage.removeFragment(fragmentMeta); + } catch (MetaStorageException e) { + logger.error("remove fragment error: ", e); + } finally { + try { + storage.releaseFragment(); + } catch (MetaStorageException e) { + logger.error("release fragment lock error: ", e); + } + } + } + @Override public void addFragment(FragmentMeta fragmentMeta) { try { @@ -694,8 +786,8 @@ public void endFragmentByTimeSeriesInterval(FragmentMeta fragmentMeta, String en try { storage.lockFragment(); TimeSeriesRange sourceTsInterval = new TimeSeriesInterval( - fragmentMeta.getTsInterval().getStartTimeSeries(), - fragmentMeta.getTsInterval().getEndTimeSeries()); + fragmentMeta.getTsInterval().getStartTimeSeries(), + fragmentMeta.getTsInterval().getEndTimeSeries()); cache.deleteFragmentByTsInterval(fragmentMeta.getTsInterval(), fragmentMeta); fragmentMeta.getTsInterval().setEndTimeSeries(endTimeSeries); cache.addFragment(fragmentMeta); @@ -735,16 +827,16 @@ public boolean hasFragment() { private void checkInitialFragmentCompletion(List fragments) { Map> fragmentsByStartTime = new HashMap<>(); - for (FragmentMeta fragment: fragments) { + for (FragmentMeta fragment : fragments) { List fragmentList = fragmentsByStartTime.computeIfAbsent(fragment.getTimeInterval().getStartTime(), e -> new ArrayList<>()); fragmentList.add(fragment); } // 检查空间边界是否完备 - for (long startTime: fragmentsByStartTime.keySet()) { + for (long startTime : fragmentsByStartTime.keySet()) { List fragmentList = fragmentsByStartTime.get(startTime); long endTime = -1; Map borders = new HashMap<>(); - for (FragmentMeta fragment: fragmentList) { + for (FragmentMeta fragment : fragmentList) { if (endTime == -1) { endTime = fragment.getTimeInterval().getEndTime(); } @@ -757,7 +849,7 @@ private void checkInitialFragmentCompletion(List fragments) { borders.put(startTs, borders.getOrDefault(startTs, 0) - 1); borders.put(endTs, borders.getOrDefault(endTs, 0) + 1); } - for (String border: borders.keySet()) { + for (String border : borders.keySet()) { if (borders.get(border) != 0) { logger.error("initial fragments should be completion"); return; @@ -766,13 +858,13 @@ private void checkInitialFragmentCompletion(List fragments) { } // 检查时间边界是否完备 Map timeBorders = new HashMap<>(); - for (long startTime: fragmentsByStartTime.keySet()) { + for (long startTime : fragmentsByStartTime.keySet()) { long endTime = fragmentsByStartTime.get(startTime).get(0).getTimeInterval().getEndTime(); timeBorders.put(startTime, timeBorders.getOrDefault(startTime, 0) - 1); timeBorders.put(endTime, timeBorders.getOrDefault(endTime, 0) + 1); } boolean seeZeroTime = false, seeMaxTime = false; - for (long time: timeBorders.keySet()) { + for (long time : timeBorders.keySet()) { if (time == 0) { seeZeroTime = true; if (timeBorders.get(time) != -1) { @@ -798,10 +890,11 @@ private void checkInitialFragmentCompletion(List fragments) { logger.error("initial fragments should be completion"); } } + private void checkFragmentCompletion(List fragments) { long startTime = -1; Map borders = new HashMap<>(); - for (FragmentMeta fragment: fragments) { + for (FragmentMeta fragment : fragments) { if (fragment.getTimeInterval().getEndTime() != Long.MAX_VALUE) { logger.error("end time for new fragment should be Long.MAX_VALUE"); return; @@ -818,7 +911,7 @@ private void checkFragmentCompletion(List fragments) { borders.put(startTs, borders.getOrDefault(startTs, 0) - 1); borders.put(endTs, borders.getOrDefault(endTs, 0) + 1); } - for (String border: borders.keySet()) { + for (String border : borders.keySet()) { if (borders.get(border) != 0) { logger.error("new fragments created at the same time should be completion"); return; @@ -923,10 +1016,10 @@ public boolean createInitialFragmentsAndStorageUnits(List stora @Override public StorageUnitMeta generateNewStorageUnitMetaByFragment(FragmentMeta fragmentMeta, - long targetStorageId) throws MetaStorageException { + long targetStorageId) throws MetaStorageException { String actualName = storage.addStorageUnit(); StorageUnitMeta storageUnitMeta = new StorageUnitMeta(actualName, targetStorageId, actualName, - true, false); + true, false); storageUnitMeta.setCreatedBy(getIginxId()); cache.updateStorageUnit(storageUnitMeta); @@ -1130,15 +1223,15 @@ public UserMeta getUser(String username) { } protected void updateStorageUnitReference(Map> fragmentsMap) { - for (List fragments: fragmentsMap.values()) { - for (FragmentMeta fragment: fragments) { + for (List fragments : fragmentsMap.values()) { + for (FragmentMeta fragment : fragments) { fragment.setMasterStorageUnit(cache.getStorageUnit(fragment.getMasterStorageUnitId())); } } } protected void updateStorageUnitReference(List fragments) { - for (FragmentMeta fragment: fragments) { + for (FragmentMeta fragment : fragments) { fragment.setMasterStorageUnit(cache.getStorageUnit(fragment.getMasterStorageUnitId())); } } @@ -1234,10 +1327,177 @@ public List getTransformTasks() { return cache.getTransformTasks(); } + @Override + public void updateFragmentRequests(Map writeRequestsMap, + Map readRequestsMap) { + try { + storage.lockFragmentRequestsCounter(); + storage.updateFragmentRequests(writeRequestsMap, readRequestsMap); + storage.incrementFragmentRequestsCounter(); + storage.releaseFragmentRequestsCounter(); + } catch (Exception e) { + logger.error("encounter error when update fragment requests: ", e); + } + } + + @Override + public void updateFragmentHeat(Map writeHotspotMap, + Map readHotspotMap) { + try { + storage.lockFragmentHeatCounter(); + storage.updateFragmentHeat(writeHotspotMap, readHotspotMap); + storage.incrementFragmentHeatCounter(); + storage.releaseFragmentHeatCounter(); + } catch (Exception e) { + logger.error("encounter error when update fragment heat: ", e); + } + } + + @Override + public void updateTimeseriesHeat(Map timeseriesHeatMap) { + try { + storage.lockTimeseriesHeatCounter(); + storage.updateTimeseriesLoad(timeseriesHeatMap); + storage.incrementTimeseriesHeatCounter(); + storage.releaseTimeseriesHeatCounter(); + } catch (Exception e) { + logger.error("encounter error when update timeseries heat: ", e); + } + } + + @Override + public void clearMonitors() { + try { + Thread.sleep(1000); + if (getIginxList().get(0).getId() == getIginxId()) { + storage.lockFragmentRequestsCounter(); + storage.lockFragmentHeatCounter(); + storage.lockTimeseriesHeatCounter(); + + storage.resetFragmentRequestsCounter(); + storage.resetFragmentHeatCounter(); + storage.resetTimeseriesHeatCounter(); + storage.removeFragmentRequests(); + storage.removeFragmentHeat(); + storage.removeTimeseriesHeat(); + + storage.releaseFragmentRequestsCounter(); + storage.releaseFragmentHeatCounter(); + storage.releaseTimeseriesHeatCounter(); + } + HotSpotMonitor.getInstance().clear(); + RequestsMonitor.getInstance().clear(); + } catch (Exception e) { + logger.error("encounter error when clear monitors: ", e); + } + } + + @Override + public Pair, Map> loadFragmentHeat() { + try { + return storage.loadFragmentHeat(cache); + } catch (Exception e) { + logger.error("encounter error when remove fragment heat: ", e); + return new Pair<>(new HashMap<>(), new HashMap<>()); + } + } + + @Override + public void updateFragmentPoints(FragmentMeta fragmentMeta, long points) { + try { + storage.updateFragmentPoints(fragmentMeta, points); + } catch (Exception e) { + logger.error("encounter error when add fragment points: ", e); + } + } + + @Override + public Map loadFragmentPoints() { + try { + return storage.loadFragmentPoints(cache); + } catch (Exception e) { + logger.error("encounter error when load fragment points: ", e); + return new HashMap<>(); + } + } + + @Override + public Map loadTimeseriesHeat() { + try { + return storage.loadTimeseriesHeat(); + } catch (Exception e) { + logger.error("encounter error when load fragment points: ", e); + return new HashMap<>(); + } + } + + @Override + public void executeReshardJudging() { + try { + if (!reshardStatus.equals(NON_RESHARDING)) { + return; + } + storage.lockReshardStatus(); + reshardStatus = JUDGING; + isProposer = true; + logger.info("iginx node {} propose to judge reshard", id); + storage.releaseReshardStatus(); + } catch (MetaStorageException e) { + logger.error("encounter error when proposing to reshard: ", e); + } + } + + @Override + public boolean executeReshard() { + try { + logger.error("reshardStatus = {}", reshardStatus); + if (!reshardStatus.equals(JUDGING)) { + return false; + } + storage.lockReshardStatus(); + try { + // 提议进入重分片流程,返回值为 true 代表提议成功,本节点成为 proposer;为 false 代表提议失败,说明已有其他节点提议成功 + if (storage.proposeToReshard()) { + reshardStatus = EXECUTING; + isProposer = true; + // 生成最终节点状态和整体迁移计划 + // 根据整体迁移计划进行迁移 + logger.info("iginx node {} propose to reshard", id); + // 在重分片判断阶段,proposer 节点不需要推送本地的存储后端统计信息 + return true; + } else { + return false; + } + } finally { + storage.releaseReshardStatus(); + } + } catch (MetaStorageException e) { + logger.error("encounter error when proposing to reshard: ", e); + } + return false; + } + + @Override + public void doneReshard() { + try { + storage.lockReshardStatus(); + reshardStatus = NON_RESHARDING; + storage.updateReshardStatus(NON_RESHARDING); + storage.releaseReshardStatus(); + } catch (MetaStorageException e) { + logger.error("encounter error when proposing to reshard: ", e); + } + } + + @Override + public boolean isResharding() { + return reshardStatus != NON_RESHARDING; + } + @Override public void updateMaxActiveEndTime(long endTime) { maxActiveEndTime.getAndUpdate(e -> Math.max(e, endTime - + ConfigDescriptor.getInstance().getConfig().getReshardFragmentTimeMargin() * 1000)); + + ConfigDescriptor.getInstance().getConfig().getReshardFragmentTimeMargin() * 1000)); } @Override diff --git a/core/src/main/java/cn/edu/tsinghua/iginx/metadata/IMetaManager.java b/core/src/main/java/cn/edu/tsinghua/iginx/metadata/IMetaManager.java index 2d0e4543b..83509d326 100644 --- a/core/src/main/java/cn/edu/tsinghua/iginx/metadata/IMetaManager.java +++ b/core/src/main/java/cn/edu/tsinghua/iginx/metadata/IMetaManager.java @@ -241,6 +241,34 @@ StorageUnitMeta generateNewStorageUnitMetaByFragment(FragmentMeta fragmentMeta, List getTransformTasks(); + void updateFragmentRequests(Map writeRequestsMap, + Map readRequestsMap) throws Exception; + + void updateFragmentHeat(Map writeHotspotMap, + Map readHotspotMap) throws Exception; + + void updateTimeseriesHeat(Map timeseriesHeatMap) throws Exception; + + Pair, Map> loadFragmentHeat() throws Exception; + + void updateFragmentPoints(FragmentMeta fragmentMeta, long points); + + Map loadFragmentPoints() throws Exception; + + Map loadTimeseriesHeat() throws Exception; + + void clearMonitors(); + + boolean isResharding(); + + void executeReshardJudging(); + + boolean executeReshard(); + + void doneReshard(); + + void removeFragment(FragmentMeta fragmentMeta); + void addFragment(FragmentMeta fragmentMeta); void endFragmentByTimeSeriesInterval(FragmentMeta fragmentMeta, String endTimeSeries); @@ -252,4 +280,5 @@ StorageUnitMeta generateNewStorageUnitMetaByFragment(FragmentMeta fragmentMeta, long getMaxActiveEndTime(); void submitMaxActiveEndTime(); + } diff --git a/core/src/main/java/cn/edu/tsinghua/iginx/metadata/cache/DefaultMetaCache.java b/core/src/main/java/cn/edu/tsinghua/iginx/metadata/cache/DefaultMetaCache.java index 56ad394b1..4ca55d350 100644 --- a/core/src/main/java/cn/edu/tsinghua/iginx/metadata/cache/DefaultMetaCache.java +++ b/core/src/main/java/cn/edu/tsinghua/iginx/metadata/cache/DefaultMetaCache.java @@ -95,7 +95,7 @@ public class DefaultMetaCache implements IMetaCache { private DefaultMetaCache() { if (enableFragmentCacheControl) { long sizeOfFragment = FragmentMeta.sizeOf(); - fragmentCacheMaxSize = (int) ((long)(config.getFragmentCacheThreshold() * 1024) / sizeOfFragment); + fragmentCacheMaxSize = (int) ((long) (config.getFragmentCacheThreshold() * 1024) / sizeOfFragment); fragmentCacheSize = 0; } else { fragmentCacheMaxSize = -1; @@ -210,7 +210,7 @@ public void initFragment(Map> fragmentListMa storageUnitLock.readLock().unlock(); fragmentLock.writeLock().lock(); sortedFragmentMetaLists.addAll(fragmentListMap.entrySet().stream().sorted(Map.Entry.comparingByKey()) - .map(e -> new Pair<>(e.getKey(), e.getValue())).collect(Collectors.toList())); + .map(e -> new Pair<>(e.getKey(), e.getValue())).collect(Collectors.toList())); fragmentListMap.forEach(fragmentMetaListMap::put); if (enableFragmentCacheControl) { // 统计分片总数 @@ -224,7 +224,7 @@ public void initFragment(Map> fragmentListMa private void kickOffHistoryFragment() { long nextMinTimestamp = 0L; - for (List fragmentList: fragmentMetaListMap.values()) { + for (List fragmentList : fragmentMetaListMap.values()) { FragmentMeta fragment = fragmentList.get(0); if (fragment.getTimeInterval().getStartTime() == minTimestamp) { fragmentList.remove(0); @@ -285,7 +285,6 @@ private void updateSortedFragmentsList(TimeSeriesRange tsInterval, List getFragmentMapByExactTimeSeriesInterval(TimeSeriesRange tsInterval) { + List res = fragmentMetaListMap.getOrDefault(tsInterval, new ArrayList<>()); + // 对象不匹配的情况需要手动匹配(?) + if (res.size() == 0) { + for (Map.Entry> fragmentMetaListEntry : fragmentMetaListMap + .entrySet()) { + if (fragmentMetaListEntry.getKey().toString().equals(tsInterval.toString())) { + return fragmentMetaListEntry.getValue(); + } + } + } + return res; + } + public Map> getFragmentMapByTimeSeriesInterval(TimeSeriesRange tsInterval) { Map> resultMap = new HashMap<>(); fragmentLock.readLock().lock(); @@ -355,7 +368,7 @@ public Map> getFragmentMapByTimeSeriesInterv public List getDummyFragmentsByTimeSeriesInterval(TimeSeriesRange tsInterval) { fragmentLock.readLock().lock(); List results = new ArrayList<>(); - for (FragmentMeta fragmentMeta: dummyFragments) { + for (FragmentMeta fragmentMeta : dummyFragments) { if (fragmentMeta.getTsInterval().isIntersect(tsInterval)) { results.add(fragmentMeta); } @@ -369,7 +382,7 @@ public Map getLatestFragmentMap() { Map latestFragmentMap = new HashMap<>(); fragmentLock.readLock().lock(); sortedFragmentMetaLists.stream().map(e -> e.v.get(e.v.size() - 1)).filter(e -> e.getTimeInterval().getEndTime() == Long.MAX_VALUE) - .forEach(e -> latestFragmentMap.put(e.getTsInterval(), e)); + .forEach(e -> latestFragmentMap.put(e.getTsInterval(), e)); fragmentLock.readLock().unlock(); return latestFragmentMap; } @@ -379,7 +392,7 @@ public Map getLatestFragmentMapByTimeSeriesInterv Map latestFragmentMap = new HashMap<>(); fragmentLock.readLock().lock(); searchFragmentSeriesList(sortedFragmentMetaLists, tsInterval).stream().map(e -> e.v.get(e.v.size() - 1)).filter(e -> e.getTimeInterval().getEndTime() == Long.MAX_VALUE) - .forEach(e -> latestFragmentMap.put(e.getTsInterval(), e)); + .forEach(e -> latestFragmentMap.put(e.getTsInterval(), e)); fragmentLock.readLock().unlock(); return latestFragmentMap; } @@ -402,7 +415,7 @@ public Map> getFragmentMapByTimeSeriesInterv public List getDummyFragmentsByTimeSeriesIntervalAndTimeInterval(TimeSeriesRange tsInterval, TimeInterval timeInterval) { fragmentLock.readLock().lock(); List results = new ArrayList<>(); - for (FragmentMeta fragmentMeta: dummyFragments) { + for (FragmentMeta fragmentMeta : dummyFragments) { if (fragmentMeta.getTsInterval().isIntersect(tsInterval) && fragmentMeta.getTimeInterval().isIntersect(timeInterval)) { results.add(fragmentMeta); } @@ -433,7 +446,7 @@ public FragmentMeta getLatestFragmentByTimeSeriesName(String tsName) { FragmentMeta result; fragmentLock.readLock().lock(); result = searchFragmentSeriesList(sortedFragmentMetaLists, tsName).stream().map(e -> e.v).flatMap(List::stream) - .filter(e -> e.getTimeInterval().getEndTime() == Long.MAX_VALUE).findFirst().orElse(null); + .filter(e -> e.getTimeInterval().getEndTime() == Long.MAX_VALUE).findFirst().orElse(null); fragmentLock.readLock().unlock(); return result; } @@ -443,7 +456,7 @@ public List getFragmentListByTimeSeriesNameAndTimeInterval(String List resultList; fragmentLock.readLock().lock(); List fragmentMetas = searchFragmentSeriesList(sortedFragmentMetaLists, tsName).stream().map(e -> e.v).flatMap(List::stream) - .sorted(Comparator.comparingLong(o -> o.getTimeInterval().getStartTime())).collect(Collectors.toList()); + .sorted(Comparator.comparingLong(o -> o.getTimeInterval().getStartTime())).collect(Collectors.toList()); resultList = searchFragmentList(fragmentMetas, timeInterval); fragmentLock.readLock().unlock(); return resultList; @@ -454,7 +467,7 @@ public List getFragmentListByStorageUnitId(String storageUnitId) { List resultList; fragmentLock.readLock().lock(); List fragmentMetas = sortedFragmentMetaLists.stream().map(e -> e.v).flatMap(List::stream) - .sorted(Comparator.comparingLong(o -> o.getTimeInterval().getStartTime())).collect(Collectors.toList()); + .sorted(Comparator.comparingLong(o -> o.getTimeInterval().getStartTime())).collect(Collectors.toList()); resultList = searchFragmentList(fragmentMetas, storageUnitId); fragmentLock.readLock().unlock(); return resultList; @@ -757,7 +770,7 @@ private long transDatatypeToByte(DataType dataType) { public List getMaxValueFromTimeSeries() { insertRecordLock.readLock().lock(); List ret = timeSeriesCalDOConcurrentHashMap.values().stream() - .filter(e -> random.nextDouble() < config.getCachedTimeseriesProb()).collect(Collectors.toList()); + .filter(e -> random.nextDouble() < config.getCachedTimeseriesProb()).collect(Collectors.toList()); insertRecordLock.readLock().unlock(); return ret; } diff --git a/core/src/main/java/cn/edu/tsinghua/iginx/metadata/cache/IMetaCache.java b/core/src/main/java/cn/edu/tsinghua/iginx/metadata/cache/IMetaCache.java index 41a339257..0c67a4710 100644 --- a/core/src/main/java/cn/edu/tsinghua/iginx/metadata/cache/IMetaCache.java +++ b/core/src/main/java/cn/edu/tsinghua/iginx/metadata/cache/IMetaCache.java @@ -41,6 +41,8 @@ public interface IMetaCache { void deleteFragmentByTsInterval(TimeSeriesRange tsInterval, FragmentMeta fragmentMeta); + List getFragmentMapByExactTimeSeriesInterval(TimeSeriesRange tsInterval); + Map> getFragmentMapByTimeSeriesInterval(TimeSeriesRange tsInterval); List getDummyFragmentsByTimeSeriesInterval(TimeSeriesRange tsInterval); diff --git a/core/src/main/java/cn/edu/tsinghua/iginx/metadata/hook/EnableMonitorChangeHook.java b/core/src/main/java/cn/edu/tsinghua/iginx/metadata/hook/EnableMonitorChangeHook.java new file mode 100644 index 000000000..ebbb4a87e --- /dev/null +++ b/core/src/main/java/cn/edu/tsinghua/iginx/metadata/hook/EnableMonitorChangeHook.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 cn.edu.tsinghua.iginx.metadata.hook; + +public interface EnableMonitorChangeHook { + + void onChange(boolean status); + +} diff --git a/core/src/main/java/cn/edu/tsinghua/iginx/metadata/hook/ReshardCounterChangeHook.java b/core/src/main/java/cn/edu/tsinghua/iginx/metadata/hook/ReshardCounterChangeHook.java new file mode 100644 index 000000000..17beed78b --- /dev/null +++ b/core/src/main/java/cn/edu/tsinghua/iginx/metadata/hook/ReshardCounterChangeHook.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 cn.edu.tsinghua.iginx.metadata.hook; + +public interface ReshardCounterChangeHook { + + void onChange(int counter); + +} diff --git a/core/src/main/java/cn/edu/tsinghua/iginx/metadata/hook/ReshardStatusChangeHook.java b/core/src/main/java/cn/edu/tsinghua/iginx/metadata/hook/ReshardStatusChangeHook.java new file mode 100644 index 000000000..dd4358973 --- /dev/null +++ b/core/src/main/java/cn/edu/tsinghua/iginx/metadata/hook/ReshardStatusChangeHook.java @@ -0,0 +1,27 @@ +/* + * 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 cn.edu.tsinghua.iginx.metadata.hook; + +import cn.edu.tsinghua.iginx.metadata.utils.ReshardStatus; + +public interface ReshardStatusChangeHook { + + void onChange(ReshardStatus status); + +} diff --git a/core/src/main/java/cn/edu/tsinghua/iginx/metadata/storage/IMetaStorage.java b/core/src/main/java/cn/edu/tsinghua/iginx/metadata/storage/IMetaStorage.java index e5ba2e330..5e0f35799 100644 --- a/core/src/main/java/cn/edu/tsinghua/iginx/metadata/storage/IMetaStorage.java +++ b/core/src/main/java/cn/edu/tsinghua/iginx/metadata/storage/IMetaStorage.java @@ -19,8 +19,11 @@ package cn.edu.tsinghua.iginx.metadata.storage; import cn.edu.tsinghua.iginx.exceptions.MetaStorageException; +import cn.edu.tsinghua.iginx.metadata.cache.IMetaCache; import cn.edu.tsinghua.iginx.metadata.entity.*; import cn.edu.tsinghua.iginx.metadata.hook.*; +import cn.edu.tsinghua.iginx.metadata.utils.ReshardStatus; +import cn.edu.tsinghua.iginx.utils.Pair; import java.util.List; import java.util.Map; @@ -69,6 +72,8 @@ public interface IMetaStorage { void updateFragmentByTsInterval(TimeSeriesRange tsInterval, FragmentMeta fragmentMeta) throws MetaStorageException; + void removeFragment(FragmentMeta fragmentMeta) throws MetaStorageException; + void addFragment(FragmentMeta fragmentMeta) throws MetaStorageException; void releaseFragment() throws MetaStorageException; @@ -99,8 +104,6 @@ public interface IMetaStorage { int updateVersion(); - void updateTimeseriesLoad(Map timeseriesLoadMap) throws Exception; - void registerTransformChangeHook(TransformChangeHook hook); List loadTransformTask() throws MetaStorageException; @@ -111,6 +114,88 @@ public interface IMetaStorage { void dropTransformTask(String name) throws MetaStorageException; + void updateTimeseriesLoad(Map timeseriesLoadMap) throws Exception; + + Map loadTimeseriesHeat() throws MetaStorageException, Exception; + + void removeTimeseriesHeat() throws MetaStorageException; + + void lockTimeseriesHeatCounter() throws MetaStorageException; + + void incrementTimeseriesHeatCounter() throws MetaStorageException; + + void resetTimeseriesHeatCounter() throws MetaStorageException; + + void releaseTimeseriesHeatCounter() throws MetaStorageException; + + int getTimeseriesHeatCounter() throws MetaStorageException; + + void updateFragmentRequests(Map writeRequestsMap, + Map readRequestsMap) throws Exception; + + void removeFragmentRequests() throws MetaStorageException; + + void lockFragmentRequestsCounter() throws MetaStorageException; + + void incrementMonitorClearCounter() throws MetaStorageException; + + int getMonitorClearCounter() throws MetaStorageException; + + void incrementFragmentRequestsCounter() throws MetaStorageException; + + void resetFragmentRequestsCounter() throws MetaStorageException; + + void releaseFragmentRequestsCounter() throws MetaStorageException; + + int getFragmentRequestsCounter() throws MetaStorageException; + + Map loadFragmentPoints(IMetaCache cache) throws Exception; + + void deleteFragmentPoints(TimeSeriesInterval tsInterval, TimeInterval timeInterval) throws Exception; + + void updateFragmentPoints(FragmentMeta fragmentMeta, long points) throws Exception; + + void updateFragmentHeat(Map writeHotspotMap, + Map readHotspotMap) throws Exception; + + Pair, Map> loadFragmentHeat(IMetaCache cache) throws Exception; + + void removeFragmentHeat() throws MetaStorageException; + + void lockFragmentHeatCounter() throws MetaStorageException; + + void incrementFragmentHeatCounter() throws MetaStorageException; + + void resetFragmentHeatCounter() throws MetaStorageException; + + void releaseFragmentHeatCounter() throws MetaStorageException; + + int getFragmentHeatCounter() throws MetaStorageException; + + boolean proposeToReshard() throws MetaStorageException; + + void lockReshardStatus() throws MetaStorageException; + + void updateReshardStatus(ReshardStatus status) throws MetaStorageException; + + void releaseReshardStatus() throws MetaStorageException; + + void removeReshardStatus() throws MetaStorageException; + + void registerReshardStatusHook(ReshardStatusChangeHook hook); + + void lockReshardCounter() throws MetaStorageException; + + void incrementReshardCounter() throws MetaStorageException; + + void resetReshardCounter() throws MetaStorageException; + + void releaseReshardCounter() throws MetaStorageException; + + void removeReshardCounter() throws MetaStorageException; + + void registerReshardCounterChangeHook(ReshardCounterChangeHook hook); + void lockMaxActiveEndTimeStatistics() throws MetaStorageException; void addOrUpdateMaxActiveEndTimeStatistics(long endTime) throws MetaStorageException; diff --git a/core/src/main/java/cn/edu/tsinghua/iginx/metadata/storage/etcd/ETCDMetaStorage.java b/core/src/main/java/cn/edu/tsinghua/iginx/metadata/storage/etcd/ETCDMetaStorage.java index f0114c48b..6265edd67 100644 --- a/core/src/main/java/cn/edu/tsinghua/iginx/metadata/storage/etcd/ETCDMetaStorage.java +++ b/core/src/main/java/cn/edu/tsinghua/iginx/metadata/storage/etcd/ETCDMetaStorage.java @@ -20,10 +20,13 @@ import cn.edu.tsinghua.iginx.conf.ConfigDescriptor; import cn.edu.tsinghua.iginx.exceptions.MetaStorageException; +import cn.edu.tsinghua.iginx.metadata.cache.IMetaCache; import cn.edu.tsinghua.iginx.metadata.entity.*; import cn.edu.tsinghua.iginx.metadata.hook.*; import cn.edu.tsinghua.iginx.metadata.storage.IMetaStorage; +import cn.edu.tsinghua.iginx.metadata.utils.ReshardStatus; import cn.edu.tsinghua.iginx.utils.JsonUtils; +import cn.edu.tsinghua.iginx.utils.Pair; import io.etcd.jetcd.*; import io.etcd.jetcd.kv.GetResponse; import io.etcd.jetcd.lease.LeaseKeepAliveResponse; @@ -62,6 +65,18 @@ public class ETCDMetaStorage implements IMetaStorage { private static final String TRANSFORM_LOCK = "/lock/transform/"; + private static final String RESHARD_STATUS_LOCK_NODE = "/lock/status/reshard"; + + private static final String RESHARD_COUNTER_LOCK_NODE = "/lock/counter/reshard"; + + private static final String ACTIVE_END_TIME_COUNTER_LOCK_NODE = "/lock/counter/end/time/active/max"; + + private static final String LATENCY_COUNTER_LOCK_NODE = "/lock/counter/latency"; + + private static final String FRAGMENT_HEAT_COUNTER_LOCK_NODE = "/lock/counter/fragment/heat"; + + private static final String TIMESERIES_HEAT_COUNTER_LOCK_NODE = "/lock/counter/timeseries/heat"; + private static final String SCHEMA_MAPPING_PREFIX = "/schema/"; private static final String IGINX_PREFIX = "/iginx/"; @@ -74,6 +89,36 @@ public class ETCDMetaStorage implements IMetaStorage { private static final String USER_PREFIX = "/user/"; + private static final String STATISTICS_FRAGMENT_POINTS_PREFIX = "/statistics/fragment/points"; + + private static final String STATISTICS_FRAGMENT_REQUESTS_PREFIX_WRITE = "/statistics/fragment/requests/write"; + + private static final String STATISTICS_FRAGMENT_REQUESTS_PREFIX_READ = "/statistics/fragment/requests/read"; + + private static final String STATISTICS_MONITOR_CLEAR_COUNTER_PREFIX = "/statistics/monitor/clear/counter"; + + private static final String STATISTICS_FRAGMENT_REQUESTS_COUNTER_PREFIX = "/statistics/fragment/requests/counter"; + + private static final String STATISTICS_FRAGMENT_HEAT_PREFIX_WRITE = "/statistics/fragment/heat/write"; + + private static final String STATISTICS_FRAGMENT_HEAT_PREFIX_READ = "/statistics/fragment/heat/read"; + + private static final String STATISTICS_FRAGMENT_HEAT_COUNTER_PREFIX = "/statistics/fragment/heat/counter"; + + private static final String STATISTICS_TIMESERIES_HEAT_PREFIX = "/statistics/timeseries/heat"; + + private static final String STATISTICS_TIMESERIES_HEAT_COUNTER_PREFIX = "/statistics/timeseries/heat/counter"; + + private static final String MAX_ACTIVE_END_TIME_STATISTICS_NODE = "/statistics/end/time/active/max/node"; + + private static final String MAX_ACTIVE_END_TIME_STATISTICS_NODE_PREFIX = "/statistics/end/time/active/max"; + + private static final String RESHARD_STATUS_NODE_PREFIX = "/status/reshard"; + + private static final String RESHARD_COUNTER_NODE_PREFIX = "/counter/reshard"; + + private static final String TIMESERIES_NODE_PREFIX = "/timeseries"; + private static final String TRANSFORM_PREFIX = "/transform/"; private static final long MAX_LOCK_TIME = 30; // 最长锁住 30 秒 @@ -87,6 +132,12 @@ public class ETCDMetaStorage implements IMetaStorage { private final Lock fragmentLeaseLock = new ReentrantLock(); private final Lock userLeaseLock = new ReentrantLock(); private final Lock transformLeaseLock = new ReentrantLock(); + private final Lock fragmentRequestsCounterLeaseLock = new ReentrantLock(); + private final Lock fragmentHeatCounterLeaseLock = new ReentrantLock(); + private final Lock timeseriesHeatCounterLeaseLock = new ReentrantLock(); + private final Lock reshardStatusLeaseLock = new ReentrantLock(); + private final Lock reshardCounterLeaseLock = new ReentrantLock(); + private final Lock maxActiveEndTimeStatisticsLeaseLock = new ReentrantLock(); private Client client; @@ -109,6 +160,15 @@ public class ETCDMetaStorage implements IMetaStorage { private Watch.Watcher transformWatcher; private TransformChangeHook transformChangeHook = null; private long transformLease = -1L; + private Watch.Watcher reshardStatusWatcher; + private ReshardStatusChangeHook reshardStatusChangeHook = null; + private long reshardStatusLease = -1L; + private Watch.Watcher reshardCounterWatcher; + private ReshardCounterChangeHook reshardCounterChangeHook = null; + private long reshardCounterLease = -1L; + private Watch.Watcher maxActiveEndTimeStatisticsWatcher; + private MaxActiveEndTimeStatisticsChangeHook maxActiveEndTimeStatisticsChangeHook = null; + private long maxActiveEndTimeStatisticsLease = -1L; public ETCDMetaStorage() { client = Client.builder() @@ -378,6 +438,120 @@ public void onCompleted() { } }); + + // 注册 reshardStatus 的监听 + this.reshardStatusWatcher = client.getWatchClient().watch(ByteSequence.from(RESHARD_STATUS_NODE_PREFIX.getBytes()), + WatchOption.newBuilder().withPrefix(ByteSequence.from(RESHARD_STATUS_NODE_PREFIX.getBytes())).withPrevKV(true).build(), + new Watch.Listener() { + @Override + public void onNext(WatchResponse watchResponse) { + if (ETCDMetaStorage.this.reshardStatusChangeHook == null) { + return; + } + for (WatchEvent event : watchResponse.getEvents()) { + ReshardStatus status; + switch (event.getEventType()) { + case PUT: + status = JsonUtils.fromJson(event.getKeyValue().getValue().getBytes(), ReshardStatus.class); + reshardStatusChangeHook.onChange(status); + break; + case DELETE: + status = JsonUtils.fromJson(event.getPrevKV().getValue().getBytes(), ReshardStatus.class); + reshardStatusChangeHook.onChange(status); + break; + default: + logger.error("unexpected watchEvent: " + event.getEventType()); + break; + } + } + } + + @Override + public void onError(Throwable throwable) { + + } + + @Override + public void onCompleted() { + + } + }); + + // 注册 reshardCounter 的监听 + this.reshardCounterWatcher = client.getWatchClient().watch(ByteSequence.from(RESHARD_COUNTER_NODE_PREFIX.getBytes()), + WatchOption.newBuilder().withPrefix(ByteSequence.from(RESHARD_COUNTER_NODE_PREFIX.getBytes())).withPrevKV(true).build(), + new Watch.Listener() { + @Override + public void onNext(WatchResponse watchResponse) { + if (ETCDMetaStorage.this.reshardCounterChangeHook == null) { + return; + } + for (WatchEvent event : watchResponse.getEvents()) { + int counter; + switch (event.getEventType()) { + case PUT: + counter = JsonUtils.fromJson(event.getKeyValue().getValue().getBytes(), Integer.class); + reshardCounterChangeHook.onChange(counter); + break; + case DELETE: + counter = JsonUtils.fromJson(event.getPrevKV().getValue().getBytes(), Integer.class); + reshardCounterChangeHook.onChange(counter); + break; + default: + logger.error("unexpected watchEvent: " + event.getEventType()); + break; + } + } + } + + @Override + public void onError(Throwable throwable) { + + } + + @Override + public void onCompleted() { + + } + }); + + // 注册 maxActiveEndTimeStatistics 的监听 + this.maxActiveEndTimeStatisticsWatcher = client.getWatchClient().watch(ByteSequence.from(MAX_ACTIVE_END_TIME_STATISTICS_NODE_PREFIX.getBytes()), + WatchOption.newBuilder().withPrefix(ByteSequence.from(MAX_ACTIVE_END_TIME_STATISTICS_NODE_PREFIX.getBytes())).withPrevKV(true).build(), + new Watch.Listener() { + @Override + public void onNext(WatchResponse watchResponse) { + if (ETCDMetaStorage.this.maxActiveEndTimeStatisticsChangeHook == null) { + return; + } + for (WatchEvent event : watchResponse.getEvents()) { + long endTime; + switch (event.getEventType()) { + case PUT: + endTime = JsonUtils.fromJson(event.getKeyValue().getValue().getBytes(), Long.class); + maxActiveEndTimeStatisticsChangeHook.onChange(endTime); + break; + case DELETE: + endTime = JsonUtils.fromJson(event.getPrevKV().getValue().getBytes(), Long.class); + maxActiveEndTimeStatisticsChangeHook.onChange(endTime); + break; + default: + logger.error("unexpected watchEvent: " + event.getEventType()); + break; + } + } + } + + @Override + public void onError(Throwable throwable) { + + } + + @Override + public void onCompleted() { + + } + }); } public static ETCDMetaStorage getInstance() { @@ -759,6 +933,25 @@ public void updateFragment(FragmentMeta fragmentMeta) throws MetaStorageExceptio @Override public void updateFragmentByTsInterval(TimeSeriesRange tsInterval, FragmentMeta fragmentMeta) throws MetaStorageException { + try { + client.getKVClient().delete(ByteSequence.from((FRAGMENT_PREFIX + tsInterval.toString() + "/" + fragmentMeta.getTimeInterval().toString()).getBytes())); + List timeIntervalNames = new ArrayList<>(); + GetResponse response = this.client.getKVClient() + .get(ByteSequence.from(SCHEMA_MAPPING_PREFIX.getBytes()), + GetOption.newBuilder().withPrefix(ByteSequence.from(SCHEMA_MAPPING_PREFIX.getBytes())).build()) + .get(); + response.getKvs().forEach(e -> { + String schema = e.getKey().toString(StandardCharsets.UTF_8).substring(SCHEMA_MAPPING_PREFIX.length()); + Map schemaMapping = JsonUtils.transform(e.getValue().toString(StandardCharsets.UTF_8)); + schemaMappings.put(schema, schemaMapping); + }); + } catch (InterruptedException | ExecutionException e) { + throw new MetaStorageException("update storage unit error: ", e); + } + } + + @Override + public void removeFragment(FragmentMeta fragmentMeta) throws MetaStorageException { } @@ -924,6 +1117,201 @@ public void updateTimeseriesLoad(Map timeseriesLoadMap) throws Exc } + @Override + public Map loadTimeseriesHeat() throws MetaStorageException, Exception { + return null; + } + + @Override + public void removeTimeseriesHeat() throws MetaStorageException { + + } + + @Override + public void lockTimeseriesHeatCounter() throws MetaStorageException { + + } + + @Override + public void incrementTimeseriesHeatCounter() throws MetaStorageException { + + } + + @Override + public void resetTimeseriesHeatCounter() throws MetaStorageException { + + } + + @Override + public void releaseTimeseriesHeatCounter() throws MetaStorageException { + + } + + @Override + public int getTimeseriesHeatCounter() throws MetaStorageException { + return 0; + } + + @Override + public void updateFragmentRequests(Map writeRequestsMap, Map readRequestsMap) throws Exception { + + } + + @Override + public void removeFragmentRequests() throws MetaStorageException { + + } + + @Override + public void lockFragmentRequestsCounter() throws MetaStorageException { + + } + + @Override + public void incrementMonitorClearCounter() throws MetaStorageException { + + } + + @Override + public int getMonitorClearCounter() throws MetaStorageException { + return 0; + } + + @Override + public void incrementFragmentRequestsCounter() throws MetaStorageException { + + } + + @Override + public void resetFragmentRequestsCounter() throws MetaStorageException { + + } + + @Override + public void releaseFragmentRequestsCounter() throws MetaStorageException { + + } + + @Override + public int getFragmentRequestsCounter() throws MetaStorageException { + return 0; + } + + @Override + public Map loadFragmentPoints(IMetaCache cache) throws Exception { + return null; + } + + @Override + public void deleteFragmentPoints(TimeSeriesInterval tsInterval, TimeInterval timeInterval) throws Exception { + + } + + @Override + public void updateFragmentPoints(FragmentMeta fragmentMeta, long points) throws Exception { + + } + + @Override + public void updateFragmentHeat(Map writeHotspotMap, Map readHotspotMap) throws Exception { + + } + + @Override + public Pair, Map> loadFragmentHeat(IMetaCache cache) throws Exception { + return null; + } + + @Override + public void removeFragmentHeat() throws MetaStorageException { + + } + + @Override + public void lockFragmentHeatCounter() throws MetaStorageException { + + } + + @Override + public void incrementFragmentHeatCounter() throws MetaStorageException { + + } + + @Override + public void resetFragmentHeatCounter() throws MetaStorageException { + + } + + @Override + public void releaseFragmentHeatCounter() throws MetaStorageException { + + } + + @Override + public int getFragmentHeatCounter() throws MetaStorageException { + return 0; + } + + @Override + public boolean proposeToReshard() throws MetaStorageException { + return false; + } + + @Override + public void lockReshardStatus() throws MetaStorageException { + + } + + @Override + public void updateReshardStatus(ReshardStatus status) throws MetaStorageException { + + } + + @Override + public void releaseReshardStatus() throws MetaStorageException { + + } + + @Override + public void removeReshardStatus() throws MetaStorageException { + + } + + @Override + public void registerReshardStatusHook(ReshardStatusChangeHook hook) { + + } + + @Override + public void lockReshardCounter() throws MetaStorageException { + + } + + @Override + public void incrementReshardCounter() throws MetaStorageException { + + } + + @Override + public void resetReshardCounter() throws MetaStorageException { + + } + + @Override + public void releaseReshardCounter() throws MetaStorageException { + + } + + @Override + public void removeReshardCounter() throws MetaStorageException { + + } + + @Override + public void registerReshardCounterChangeHook(ReshardCounterChangeHook hook) { + + } + private void lockTransform() throws MetaStorageException { try { transformLeaseLock.lock(); diff --git a/core/src/main/java/cn/edu/tsinghua/iginx/metadata/storage/zk/ZooKeeperMetaStorage.java b/core/src/main/java/cn/edu/tsinghua/iginx/metadata/storage/zk/ZooKeeperMetaStorage.java index 2431b44d4..d55c5cff1 100644 --- a/core/src/main/java/cn/edu/tsinghua/iginx/metadata/storage/zk/ZooKeeperMetaStorage.java +++ b/core/src/main/java/cn/edu/tsinghua/iginx/metadata/storage/zk/ZooKeeperMetaStorage.java @@ -20,11 +20,16 @@ import cn.edu.tsinghua.iginx.conf.ConfigDescriptor; import cn.edu.tsinghua.iginx.exceptions.MetaStorageException; +import cn.edu.tsinghua.iginx.metadata.cache.IMetaCache; import cn.edu.tsinghua.iginx.metadata.entity.*; import cn.edu.tsinghua.iginx.metadata.hook.*; import cn.edu.tsinghua.iginx.metadata.storage.IMetaStorage; +import cn.edu.tsinghua.iginx.metadata.utils.ReshardStatus; +import cn.edu.tsinghua.iginx.utils.Pair; + import cn.edu.tsinghua.iginx.utils.JsonUtils; import java.util.Map.Entry; + import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; import org.apache.curator.framework.recipes.cache.TreeCache; @@ -43,6 +48,8 @@ import java.util.concurrent.locks.ReentrantLock; import java.util.stream.Collectors; +import static cn.edu.tsinghua.iginx.metadata.utils.ReshardStatus.*; + public class ZooKeeperMetaStorage implements IMetaStorage { private static final Logger logger = LoggerFactory.getLogger(ZooKeeperMetaStorage.class); @@ -77,8 +84,18 @@ public class ZooKeeperMetaStorage implements IMetaStorage { private static final String USER_LOCK_NODE = "/lock/user"; + private static final String RESHARD_STATUS_LOCK_NODE = "/lock/status/reshard"; + + private static final String RESHARD_COUNTER_LOCK_NODE = "/lock/counter/reshard"; + private static final String ACTIVE_END_TIME_COUNTER_LOCK_NODE = "/lock/counter/end/time/active/max"; + private static final String LATENCY_COUNTER_LOCK_NODE = "/lock/counter/latency"; + + private static final String FRAGMENT_HEAT_COUNTER_LOCK_NODE = "/lock/counter/fragment/heat"; + + private static final String TIMESERIES_HEAT_COUNTER_LOCK_NODE = "/lock/counter/timeseries/heat"; + private static final String POLICY_NODE_PREFIX = "/policy"; private static final String POLICY_LEADER = "/policy/leader"; @@ -111,6 +128,10 @@ public class ZooKeeperMetaStorage implements IMetaStorage { private static final String MAX_ACTIVE_END_TIME_STATISTICS_NODE_PREFIX = "/statistics/end/time/active/max"; + private static final String RESHARD_STATUS_NODE_PREFIX = "/status/reshard"; + + private static final String RESHARD_COUNTER_NODE_PREFIX = "/counter/reshard"; + private static final String TIMESERIES_NODE_PREFIX = "/timeseries"; private static final String TRANSFORM_NODE_PREFIX = "/transform"; @@ -126,14 +147,28 @@ public class ZooKeeperMetaStorage implements IMetaStorage { private final InterProcessMutex storageUnitMutex; private final Lock fragmentMutexLock = new ReentrantLock(); private final InterProcessMutex fragmentMutex; + private final Lock fragmentRequestsCounterMutexLock = new ReentrantLock(); + private final InterProcessMutex fragmentRequestsCounterMutex; + private final Lock fragmentHeatCounterMutexLock = new ReentrantLock(); + private final InterProcessMutex fragmentHeatCounterMutex; + private final Lock timeseriesHeatCounterMutexLock = new ReentrantLock(); + private final InterProcessMutex timeseriesHeatCounterMutex; + private final Lock reshardStatusMutexLock = new ReentrantLock(); + private final InterProcessMutex reshardStatusMutex; + private final Lock reshardCounterMutexLock = new ReentrantLock(); + private final InterProcessMutex reshardCounterMutex; private final Lock maxActiveEndTimeStatisticsMutexLock = new ReentrantLock(); private final InterProcessMutex maxActiveEndTimeStatisticsMutex; + protected TreeCache schemaMappingsCache; protected TreeCache iginxCache; protected TreeCache storageEngineCache; protected TreeCache storageUnitCache; protected TreeCache fragmentCache; + protected TreeCache reshardStatusCache; + protected TreeCache reshardCounterCache; protected TreeCache maxActiveEndTimeStatisticsCache; + private SchemaMappingChangeHook schemaMappingChangeHook = null; private IginxChangeHook iginxChangeHook = null; private StorageChangeHook storageChangeHook = null; @@ -143,6 +178,8 @@ public class ZooKeeperMetaStorage implements IMetaStorage { private TimeSeriesChangeHook timeSeriesChangeHook = null; private VersionChangeHook versionChangeHook = null; private TransformChangeHook transformChangeHook = null; + private ReshardStatusChangeHook reshardStatusChangeHook = null; + private ReshardCounterChangeHook reshardCounterChangeHook = null; private MaxActiveEndTimeStatisticsChangeHook maxActiveEndTimeStatisticsChangeHook = null; protected TreeCache userCache; @@ -157,16 +194,21 @@ public class ZooKeeperMetaStorage implements IMetaStorage { public ZooKeeperMetaStorage() { client = CuratorFrameworkFactory.builder() - .connectString(ConfigDescriptor.getInstance().getConfig().getZookeeperConnectionString()) - .connectionTimeoutMs(15000) - .retryPolicy(new RetryForever(1000)) - .build(); + .connectString(ConfigDescriptor.getInstance().getConfig().getZookeeperConnectionString()) + .connectionTimeoutMs(15000) + .retryPolicy(new RetryForever(1000)) + .build(); client.start(); fragmentMutex = new InterProcessMutex(client, FRAGMENT_LOCK_NODE); storageUnitMutex = new InterProcessMutex(client, STORAGE_UNIT_LOCK_NODE); + reshardStatusMutex = new InterProcessMutex(client, RESHARD_STATUS_LOCK_NODE); + reshardCounterMutex = new InterProcessMutex(client, RESHARD_COUNTER_LOCK_NODE); maxActiveEndTimeStatisticsMutex = new InterProcessMutex(client, - ACTIVE_END_TIME_COUNTER_LOCK_NODE); + ACTIVE_END_TIME_COUNTER_LOCK_NODE); + fragmentRequestsCounterMutex = new InterProcessMutex(client, LATENCY_COUNTER_LOCK_NODE); + fragmentHeatCounterMutex = new InterProcessMutex(client, FRAGMENT_HEAT_COUNTER_LOCK_NODE); + timeseriesHeatCounterMutex = new InterProcessMutex(client, TIMESERIES_HEAT_COUNTER_LOCK_NODE); } public static ZooKeeperMetaStorage getInstance() { @@ -189,11 +231,11 @@ public Map> loadSchemaMapping() throws MetaStorageE if (client.checkExists().forPath(SCHEMA_MAPPING_PREFIX) == null) { // 当前还没有数据,创建父节点,然后不需要解析数据 client.create() - .withMode(CreateMode.PERSISTENT) - .forPath(SCHEMA_MAPPING_PREFIX); + .withMode(CreateMode.PERSISTENT) + .forPath(SCHEMA_MAPPING_PREFIX); } else { List schemas = this.client.getChildren() - .forPath(SCHEMA_MAPPING_PREFIX); + .forPath(SCHEMA_MAPPING_PREFIX); for (String schema : schemas) { Map schemaMapping = JsonUtils.transform(new String(this.client.getData() .forPath(SCHEMA_MAPPING_PREFIX + "/" + schema))); @@ -284,14 +326,14 @@ public Map loadIginx() throws MetaStorageException { if (client.checkExists().forPath(IGINX_NODE_PREFIX) == null) { // 当前还没有数据,创建父节点,然后不需要解析数据 client.create() - .withMode(CreateMode.PERSISTENT) - .forPath(IGINX_NODE_PREFIX); + .withMode(CreateMode.PERSISTENT) + .forPath(IGINX_NODE_PREFIX); } else { List children = client.getChildren() - .forPath(IGINX_NODE_PREFIX); + .forPath(IGINX_NODE_PREFIX); for (String childName : children) { byte[] data = client.getData() - .forPath(IGINX_NODE_PREFIX + "/" + childName); + .forPath(IGINX_NODE_PREFIX + "/" + childName); IginxMeta iginxMeta = JsonUtils.fromJson(data, IginxMeta.class); if (iginxMeta == null) { logger.error("resolve data from " + IGINX_NODE_PREFIX + "/" + childName + " error"); @@ -319,14 +361,14 @@ public long registerIginx(IginxMeta iginx) throws MetaStorageException { try { mutex.acquire(); String nodeName = this.client.create() - .creatingParentsIfNeeded() - .withMode(CreateMode.EPHEMERAL_SEQUENTIAL) - .forPath(IGINX_NODE, "".getBytes(StandardCharsets.UTF_8)); + .creatingParentsIfNeeded() + .withMode(CreateMode.EPHEMERAL_SEQUENTIAL) + .forPath(IGINX_NODE, "".getBytes(StandardCharsets.UTF_8)); long id = Long.parseLong(nodeName.substring(IGINX_NODE.length())); IginxMeta iginxMeta = new IginxMeta(id, iginx.getIp(), - iginx.getPort(), iginx.getExtraParams()); + iginx.getPort(), iginx.getExtraParams()); this.client.setData() - .forPath(nodeName, JsonUtils.toJson(iginxMeta)); + .forPath(nodeName, JsonUtils.toJson(iginxMeta)); return id; } catch (Exception e) { throw new MetaStorageException("get error when load iginx", e); @@ -396,13 +438,13 @@ public Map loadStorageEngine(List st mutex.acquire(); if (this.client.checkExists().forPath(STORAGE_ENGINE_NODE_PREFIX) == null) { // 节点不存在,说明还没有别的 iginx 节点写入过元信息 this.client.create().creatingParentsIfNeeded() - .withMode(CreateMode.PERSISTENT) - .forPath(STORAGE_ENGINE_NODE_PREFIX); + .withMode(CreateMode.PERSISTENT) + .forPath(STORAGE_ENGINE_NODE_PREFIX); for (StorageEngineMeta storageEngineMeta : storageEngines) { String nodeName = this.client.create() - .creatingParentsIfNeeded() - .withMode(CreateMode.PERSISTENT_SEQUENTIAL) - .forPath(STORAGE_ENGINE_NODE, "".getBytes(StandardCharsets.UTF_8)); + .creatingParentsIfNeeded() + .withMode(CreateMode.PERSISTENT_SEQUENTIAL) + .forPath(STORAGE_ENGINE_NODE, "".getBytes(StandardCharsets.UTF_8)); long id = Long.parseLong(nodeName.substring(STORAGE_ENGINE_NODE.length())); storageEngineMeta.setId(id); this.client.setData().forPath(nodeName, JsonUtils.toJson(storageEngineMeta)); @@ -423,6 +465,10 @@ public Map loadStorageEngine(List st } storageEngineMetaMap.putIfAbsent(storageEngineMeta.getId(), storageEngineMeta); } + + registerMaxActiveEndTimeStatisticsListener(); + registerReshardStatusListener(); + registerReshardCounterListener(); return storageEngineMetaMap; } catch (Exception e) { throw new MetaStorageException("get error when load schema mapping", e); @@ -441,14 +487,13 @@ public long addStorageEngine(StorageEngineMeta storageEngine) throws MetaStorage try { mutex.acquire(); String nodeName = this.client.create() - .creatingParentsIfNeeded() - .withMode(CreateMode.PERSISTENT_SEQUENTIAL) - .forPath(STORAGE_ENGINE_NODE, "".getBytes(StandardCharsets.UTF_8)); + .creatingParentsIfNeeded() + .withMode(CreateMode.PERSISTENT_SEQUENTIAL) + .forPath(STORAGE_ENGINE_NODE, "".getBytes(StandardCharsets.UTF_8)); long id = Long.parseLong(nodeName.substring(STORAGE_ENGINE_NODE.length())); storageEngine.setId(id); String tmp = new String(JsonUtils.toJson(storageEngine)); - this.client.setData() - .forPath(nodeName, tmp.getBytes()); + this.client.setData().forPath(nodeName, tmp.getBytes()); return id; } catch (Exception e) { throw new MetaStorageException("get error when add storage engine", e); @@ -530,7 +575,7 @@ public Map loadStorageUnit() throws MetaStorageExceptio for (String storageUnitId : storageUnitIds) { logger.info("load storage unit: " + storageUnitId); byte[] data = this.client.getData() - .forPath(STORAGE_UNIT_NODE_PREFIX + "/" + storageUnitId); + .forPath(STORAGE_UNIT_NODE_PREFIX + "/" + storageUnitId); StorageUnitMeta storageUnitMeta = JsonUtils.fromJson(data, StorageUnitMeta.class); if (!storageUnitMeta.isMaster()) { // 需要加入到主节点的子节点列表中 StorageUnitMeta masterStorageUnitMeta = storageUnitMetaMap.get(storageUnitMeta.getMasterId()); @@ -565,9 +610,9 @@ public void lockStorageUnit() throws MetaStorageException { public String addStorageUnit() throws MetaStorageException { // 只在有锁的情况下调用,内部不需要加锁 try { String nodeName = this.client.create() - .creatingParentsIfNeeded() - .withMode(CreateMode.PERSISTENT_SEQUENTIAL) - .forPath(STORAGE_UNIT_NODE, "".getBytes(StandardCharsets.UTF_8)); + .creatingParentsIfNeeded() + .withMode(CreateMode.PERSISTENT_SEQUENTIAL) + .forPath(STORAGE_UNIT_NODE, "".getBytes(StandardCharsets.UTF_8)); return nodeName.substring(STORAGE_UNIT_NODE_PREFIX.length() + 1); } catch (Exception e) { throw new MetaStorageException("add storage unit error: ", e); @@ -578,7 +623,7 @@ public String addStorageUnit() throws MetaStorageException { // 只在有锁的 public void updateStorageUnit(StorageUnitMeta storageUnitMeta) throws MetaStorageException { // 只在有锁的情况下调用,内部不需要加锁 try { this.client.setData() - .forPath(STORAGE_UNIT_NODE_PREFIX + "/" + storageUnitMeta.getId(), JsonUtils.toJson(storageUnitMeta)); + .forPath(STORAGE_UNIT_NODE_PREFIX + "/" + storageUnitMeta.getId(), JsonUtils.toJson(storageUnitMeta)); } catch (Exception e) { throw new MetaStorageException("add storage unit error: ", e); } @@ -766,17 +811,30 @@ public void lockFragment() throws MetaStorageException { public void updateFragment(FragmentMeta fragmentMeta) throws MetaStorageException { // 只在有锁的情况下调用,内部不需要加锁 try { this.client.setData() - .forPath(FRAGMENT_NODE_PREFIX + "/" + fragmentMeta.getTsInterval().toString() + "/" + fragmentMeta.getTimeInterval().toString(), JsonUtils.toJson(fragmentMeta)); + .forPath(FRAGMENT_NODE_PREFIX + "/" + fragmentMeta.getTsInterval().toString() + "/" + fragmentMeta.getTimeInterval().toString(), JsonUtils.toJson(fragmentMeta)); } catch (Exception e) { throw new MetaStorageException("get error when update fragment", e); } } + @Override + public void removeFragment(FragmentMeta fragmentMeta) throws MetaStorageException { // 只在有锁的情况下调用,内部不需要加锁 + try { + this.client.delete().forPath(FRAGMENT_NODE_PREFIX + "/" + fragmentMeta.getTsInterval().toString() + "/" + fragmentMeta.getTimeInterval().toString()); + // 删除不需要的统计数据 + this.client.delete().forPath(STATISTICS_FRAGMENT_REQUESTS_PREFIX_WRITE + "/" + fragmentMeta.getTsInterval().toString() + "/" + fragmentMeta.getTimeInterval().toString()); + this.client.delete().forPath(STATISTICS_FRAGMENT_REQUESTS_PREFIX_READ + "/" + fragmentMeta.getTsInterval().toString() + "/" + fragmentMeta.getTimeInterval().toString()); + this.client.delete().forPath(STATISTICS_FRAGMENT_POINTS_PREFIX + "/" + fragmentMeta.getTsInterval().toString() + "/" + fragmentMeta.getTimeInterval().toString()); + } catch (Exception e) { + throw new MetaStorageException("get error when add fragment", e); + } + } + @Override public void addFragment(FragmentMeta fragmentMeta) throws MetaStorageException { // 只在有锁的情况下调用,内部不需要加锁 try { this.client.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT) - .forPath(FRAGMENT_NODE_PREFIX + "/" + fragmentMeta.getTsInterval().toString() + "/" + fragmentMeta.getTimeInterval().toString(), JsonUtils.toJson(fragmentMeta)); + .forPath(FRAGMENT_NODE_PREFIX + "/" + fragmentMeta.getTsInterval().toString() + "/" + fragmentMeta.getTimeInterval().toString(), JsonUtils.toJson(fragmentMeta)); } catch (Exception e) { throw new MetaStorageException("get error when add fragment", e); } @@ -787,17 +845,17 @@ public void updateFragmentByTsInterval(TimeSeriesRange tsInterval, FragmentMeta throws MetaStorageException { try { this.client.delete() - .forPath(FRAGMENT_NODE_PREFIX + "/" + tsInterval.toString() + "/" + fragmentMeta - .getTimeInterval().toString()); + .forPath(FRAGMENT_NODE_PREFIX + "/" + tsInterval.toString() + "/" + fragmentMeta + .getTimeInterval().toString()); List timeIntervalNames = this.client.getChildren() - .forPath(FRAGMENT_NODE_PREFIX + "/" + tsInterval.toString()); + .forPath(FRAGMENT_NODE_PREFIX + "/" + tsInterval.toString()); if (timeIntervalNames.isEmpty()) { this.client.delete() - .forPath(FRAGMENT_NODE_PREFIX + "/" + tsInterval.toString()); + .forPath(FRAGMENT_NODE_PREFIX + "/" + tsInterval.toString()); } this.client.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT) - .forPath(FRAGMENT_NODE_PREFIX + "/" + fragmentMeta.getTsInterval().toString() + "/" - + fragmentMeta.getTimeInterval().toString(), JsonUtils.toJson(fragmentMeta)); + .forPath(FRAGMENT_NODE_PREFIX + "/" + fragmentMeta.getTsInterval().toString() + "/" + + fragmentMeta.getTimeInterval().toString(), JsonUtils.toJson(fragmentMeta)); } catch (Exception e) { throw new MetaStorageException("get error when update fragment", e); } @@ -814,21 +872,6 @@ public void releaseFragment() throws MetaStorageException { } } - @Override - public void updateTimeseriesLoad(Map timeseriesLoadMap) throws Exception { - for (Entry timeseriesLoadEntry : timeseriesLoadMap.entrySet()) { - String path = STATISTICS_TIMESERIES_HEAT_PREFIX + "/" + timeseriesLoadEntry.getKey(); - if (this.client.checkExists().forPath(path) == null) { - this.client.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT) - .forPath(path, JsonUtils.toJson(timeseriesLoadEntry.getValue())); - } - byte[] data = this.client.getData().forPath(path); - long heat = JsonUtils.fromJson(data, Long.class); - this.client.setData() - .forPath(path, JsonUtils.toJson(heat + timeseriesLoadEntry.getValue())); - } - } - @Override public void registerFragmentChangeHook(FragmentChangeHook hook) { this.fragmentChangeHook = hook; @@ -875,13 +918,13 @@ public List loadUser(UserMeta userMeta) throws MetaStorageException { mutex.acquire(); if (this.client.checkExists().forPath(USER_NODE_PREFIX) == null) { // 节点不存在,说明系统中第一个用户还没有被创建 this.client.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT) - .forPath(USER_NODE_PREFIX + "/" + userMeta.getUsername(), JsonUtils.toJson(userMeta)); + .forPath(USER_NODE_PREFIX + "/" + userMeta.getUsername(), JsonUtils.toJson(userMeta)); } List users = new ArrayList<>(); List usernames = this.client.getChildren().forPath(USER_NODE_PREFIX); for (String username : usernames) { byte[] data = this.client.getData() - .forPath(USER_NODE_PREFIX + "/" + username); + .forPath(USER_NODE_PREFIX + "/" + username); UserMeta user = JsonUtils.fromJson(data, UserMeta.class); if (user == null) { logger.error("resolve data from " + USER_NODE_PREFIX + "/" + username + " error"); @@ -913,7 +956,7 @@ public void addUser(UserMeta userMeta) throws MetaStorageException { try { mutex.acquire(); this.client.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT) - .forPath(USER_NODE_PREFIX + "/" + userMeta.getUsername(), JsonUtils.toJson(userMeta)); + .forPath(USER_NODE_PREFIX + "/" + userMeta.getUsername(), JsonUtils.toJson(userMeta)); } catch (Exception e) { throw new MetaStorageException("get error when add user", e); } finally { @@ -931,7 +974,7 @@ public void updateUser(UserMeta userMeta) throws MetaStorageException { try { mutex.acquire(); this.client.setData() - .forPath(USER_NODE_PREFIX + "/" + userMeta.getUsername(), JsonUtils.toJson(userMeta)); + .forPath(USER_NODE_PREFIX + "/" + userMeta.getUsername(), JsonUtils.toJson(userMeta)); } catch (Exception e) { throw new MetaStorageException("get error when update user", e); } finally { @@ -949,7 +992,7 @@ public void removeUser(String username) throws MetaStorageException { try { mutex.acquire(); this.client.delete() - .forPath(USER_NODE_PREFIX + "/" + username); + .forPath(USER_NODE_PREFIX + "/" + username); } catch (Exception e) { throw new MetaStorageException("get error when remove user", e); } finally { @@ -978,9 +1021,9 @@ public boolean election() { } try { this.client.create() - .creatingParentsIfNeeded() - .withMode(CreateMode.EPHEMERAL) - .forPath(POLICY_LEADER); + .creatingParentsIfNeeded() + .withMode(CreateMode.EPHEMERAL) + .forPath(POLICY_LEADER); logger.info("成功"); isMaster = true; } catch (KeeperException.NodeExistsException e) { @@ -1018,11 +1061,11 @@ public Map getTimeseriesData() { Map ret = new HashMap<>(); try { Set idSet = loadIginx().keySet().stream(). - map(Long::intValue).collect(Collectors.toSet()); + map(Long::intValue).collect(Collectors.toSet()); List children = client.getChildren().forPath(TIMESERIES_NODE_PREFIX); for (String child : children) { byte[] data = this.client.getData() - .forPath(TIMESERIES_NODE_PREFIX + "/" + child); + .forPath(TIMESERIES_NODE_PREFIX + "/" + child); if (!idSet.contains(Integer.parseInt(child))) { continue; } @@ -1084,37 +1127,37 @@ public void registerPolicy(long iginxId, int num) throws Exception { mutex.acquire(); if (client.checkExists().forPath(POLICY_NODE_PREFIX) == null) { this.client.create() - .creatingParentsIfNeeded() - .withMode(CreateMode.PERSISTENT) - .forPath(POLICY_NODE_PREFIX); + .creatingParentsIfNeeded() + .withMode(CreateMode.PERSISTENT) + .forPath(POLICY_NODE_PREFIX); } if (client.checkExists().forPath(TIMESERIES_NODE_PREFIX) == null) { this.client.create() - .creatingParentsIfNeeded() - .withMode(CreateMode.PERSISTENT) - .forPath(TIMESERIES_NODE_PREFIX); + .creatingParentsIfNeeded() + .withMode(CreateMode.PERSISTENT) + .forPath(TIMESERIES_NODE_PREFIX); } if (client.checkExists().forPath(POLICY_VERSION) == null) { this.client.create() - .creatingParentsIfNeeded() - .withMode(CreateMode.PERSISTENT) - .forPath(POLICY_VERSION); + .creatingParentsIfNeeded() + .withMode(CreateMode.PERSISTENT) + .forPath(POLICY_VERSION); this.client.setData().forPath(POLICY_VERSION, ("0" + "\t" + num).getBytes()); } this.client.create() - .creatingParentsIfNeeded() - .withMode(CreateMode.PERSISTENT) - .forPath(POLICY_NODE_PREFIX + "/" + iginxId); + .creatingParentsIfNeeded() + .withMode(CreateMode.PERSISTENT) + .forPath(POLICY_NODE_PREFIX + "/" + iginxId); this.client.setData().forPath(POLICY_NODE_PREFIX + "/" + iginxId, "0".getBytes()); this.client.create() - .creatingParentsIfNeeded() - .withMode(CreateMode.PERSISTENT) - .forPath(TIMESERIES_NODE_PREFIX + "/" + iginxId); + .creatingParentsIfNeeded() + .withMode(CreateMode.PERSISTENT) + .forPath(TIMESERIES_NODE_PREFIX + "/" + iginxId); this.policyCache = new TreeCache(this.client, POLICY_LEADER); TreeCacheListener listener = (curatorFramework, event) -> { @@ -1210,14 +1253,14 @@ public List loadTransformTask() throws MetaStorageException { if (this.client.checkExists().forPath(TRANSFORM_NODE_PREFIX) == null) { // 当前还没有数据,创建父节点,然后不需要解析数据 client.create() - .creatingParentsIfNeeded() - .withMode(CreateMode.PERSISTENT) - .forPath(TRANSFORM_NODE_PREFIX); + .creatingParentsIfNeeded() + .withMode(CreateMode.PERSISTENT) + .forPath(TRANSFORM_NODE_PREFIX); } else { List classNames = this.client.getChildren().forPath(TRANSFORM_NODE_PREFIX); - for (String className: classNames) { + for (String className : classNames) { byte[] data = this.client.getData() - .forPath(TRANSFORM_NODE_PREFIX + "/" + className); + .forPath(TRANSFORM_NODE_PREFIX + "/" + className); TransformTaskMeta task = JsonUtils.fromJson(data, TransformTaskMeta.class); if (task == null) { logger.error("resolve data from " + TRANSFORM_NODE_PREFIX + "/" + className + " error"); @@ -1250,7 +1293,7 @@ private void registerTransformListener() throws Exception { case NODE_ADDED: case NODE_UPDATED: if (event.getData() == null || event.getData().getPath() == null || - event.getData().getPath().equals(TRANSFORM_NODE_PREFIX)) { + event.getData().getPath().equals(TRANSFORM_NODE_PREFIX)) { return; // 前缀事件,非含数据的节点的变化,不需要处理 } taskMeta = JsonUtils.fromJson(event.getData().getData(), TransformTaskMeta.class); @@ -1280,7 +1323,7 @@ public void addTransformTask(TransformTaskMeta transformTask) throws MetaStorage try { mutex.acquire(); this.client.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT) - .forPath(TRANSFORM_NODE_PREFIX + "/" + transformTask.getName(), JsonUtils.toJson(transformTask)); + .forPath(TRANSFORM_NODE_PREFIX + "/" + transformTask.getName(), JsonUtils.toJson(transformTask)); } catch (Exception e) { throw new MetaStorageException("get error when add transform task", e); } finally { @@ -1298,7 +1341,7 @@ public void updateTransformTask(TransformTaskMeta transformTask) throws MetaStor try { mutex.acquire(); this.client.setData() - .forPath(TRANSFORM_NODE_PREFIX + "/" + transformTask.getName(), JsonUtils.toJson(transformTask)); + .forPath(TRANSFORM_NODE_PREFIX + "/" + transformTask.getName(), JsonUtils.toJson(transformTask)); } catch (Exception e) { throw new MetaStorageException("get error when update transform task", e); } finally { @@ -1316,7 +1359,7 @@ public void dropTransformTask(String name) throws MetaStorageException { try { mutex.acquire(); this.client.delete() - .forPath(TRANSFORM_NODE_PREFIX + "/" + name); + .forPath(TRANSFORM_NODE_PREFIX + "/" + name); } catch (Exception e) { throw new MetaStorageException("get error when drop transform task", e); } finally { @@ -1328,9 +1371,698 @@ public void dropTransformTask(String name) throws MetaStorageException { } } + @Override + public void updateTimeseriesLoad(Map timeseriesLoadMap) throws Exception { + for (Entry timeseriesLoadEntry : timeseriesLoadMap.entrySet()) { + String path = STATISTICS_TIMESERIES_HEAT_PREFIX + "/" + timeseriesLoadEntry.getKey(); + if (this.client.checkExists().forPath(path) == null) { + this.client.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT) + .forPath(path, JsonUtils.toJson(timeseriesLoadEntry.getValue())); + } + byte[] data = this.client.getData().forPath(path); + long heat = JsonUtils.fromJson(data, Long.class); + this.client.setData() + .forPath(path, JsonUtils.toJson(heat + timeseriesLoadEntry.getValue())); + } + } + + @Override + public Map loadTimeseriesHeat() throws Exception { + Map timeseriesHeatMap = new HashMap<>(); + List children = client.getChildren().forPath(STATISTICS_TIMESERIES_HEAT_PREFIX); + for (String child : children) { + byte[] data = this.client.getData() + .forPath(STATISTICS_TIMESERIES_HEAT_PREFIX + "/" + child); + long heat = JsonUtils.fromJson(data, Long.class); + timeseriesHeatMap.put(child, heat); + } + return timeseriesHeatMap; + } + + @Override + public void removeTimeseriesHeat() throws MetaStorageException { + try { + if (this.client.checkExists().forPath(STATISTICS_TIMESERIES_HEAT_PREFIX) != null) { + this.client.delete().deletingChildrenIfNeeded() + .forPath(STATISTICS_TIMESERIES_HEAT_PREFIX); + } + } catch (Exception e) { + throw new MetaStorageException("encounter error when removing timeseries heat: ", e); + } + } + + @Override + public void lockTimeseriesHeatCounter() throws MetaStorageException { + try { + timeseriesHeatCounterMutexLock.lock(); + timeseriesHeatCounterMutex.acquire(); + } catch (Exception e) { + timeseriesHeatCounterMutexLock.unlock(); + throw new MetaStorageException( + "encounter error when acquiring timeseries heat counter mutex: ", + e); + } + } + + @Override + public void incrementTimeseriesHeatCounter() throws MetaStorageException { + try { + if (this.client.checkExists().forPath(STATISTICS_TIMESERIES_HEAT_COUNTER_PREFIX) == null) { + this.client.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT) + .forPath(STATISTICS_TIMESERIES_HEAT_COUNTER_PREFIX, JsonUtils.toJson(1)); + } else { + int counter = JsonUtils.fromJson( + this.client.getData().forPath(STATISTICS_TIMESERIES_HEAT_COUNTER_PREFIX), + Integer.class); + this.client.setData() + .forPath(STATISTICS_TIMESERIES_HEAT_COUNTER_PREFIX, JsonUtils.toJson(counter + 1)); + } + } catch (Exception e) { + throw new MetaStorageException("encounter error when updating timeseries heat counter: ", e); + } + } + + @Override + public void resetTimeseriesHeatCounter() throws MetaStorageException { + try { + if (this.client.checkExists().forPath(STATISTICS_TIMESERIES_HEAT_COUNTER_PREFIX) == null) { + this.client.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT) + .forPath(STATISTICS_TIMESERIES_HEAT_COUNTER_PREFIX, JsonUtils.toJson(0)); + } else { + this.client.setData() + .forPath(STATISTICS_TIMESERIES_HEAT_COUNTER_PREFIX, JsonUtils.toJson(0)); + } + } catch (Exception e) { + throw new MetaStorageException("encounter error when resetting timeseries heat counter: ", e); + } + } + + @Override + public void releaseTimeseriesHeatCounter() throws MetaStorageException { + try { + timeseriesHeatCounterMutex.release(); + } catch (Exception e) { + throw new MetaStorageException("encounter error when releasing timeseries heat mutex: ", e); + } finally { + timeseriesHeatCounterMutexLock.unlock(); + } + } + + @Override + public int getTimeseriesHeatCounter() throws MetaStorageException { + try { + if (this.client.checkExists().forPath(STATISTICS_TIMESERIES_HEAT_COUNTER_PREFIX) == null) { + return 0; + } else { + return JsonUtils.fromJson( + this.client.getData().forPath(STATISTICS_TIMESERIES_HEAT_COUNTER_PREFIX), + Integer.class); + } + } catch (Exception e) { + throw new MetaStorageException("encounter error when get timeseries heat counter: ", e); + } + } + + @Override + public void updateFragmentRequests(Map writeRequestsMap, + Map readRequestsMap) throws Exception { + for (Entry writeRequestsEntry : writeRequestsMap.entrySet()) { + if (writeRequestsEntry.getValue() > 0) { + String requestsPath = + STATISTICS_FRAGMENT_REQUESTS_PREFIX_WRITE + "/" + writeRequestsEntry.getKey() + .getTsInterval() + .toString() + "/" + writeRequestsEntry.getKey().getTimeInterval().toString(); + String pointsPath = + STATISTICS_FRAGMENT_POINTS_PREFIX + "/" + writeRequestsEntry.getKey() + .getTsInterval() + .toString() + "/" + writeRequestsEntry.getKey().getTimeInterval().toString(); + if (this.client.checkExists().forPath(requestsPath) == null) { + this.client.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT) + .forPath(requestsPath, JsonUtils.toJson(writeRequestsEntry.getValue())); + } + byte[] data = this.client.getData().forPath(requestsPath); + long requests = JsonUtils.fromJson(data, Long.class); + this.client.setData() + .forPath(requestsPath, JsonUtils.toJson(requests + writeRequestsEntry.getValue())); + + if (this.client.checkExists().forPath(pointsPath) == null) { + this.client.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT) + .forPath(pointsPath, JsonUtils.toJson(writeRequestsEntry.getValue())); + } + data = this.client.getData().forPath(pointsPath); + long points = JsonUtils.fromJson(data, Long.class); + this.client.setData() + .forPath(pointsPath, JsonUtils.toJson(points + writeRequestsEntry.getValue())); + } + } + for (Entry readRequestsEntry : readRequestsMap.entrySet()) { + String path = + STATISTICS_FRAGMENT_REQUESTS_PREFIX_READ + "/" + readRequestsEntry.getKey() + .getTsInterval() + .toString() + "/" + readRequestsEntry.getKey().getTimeInterval().toString(); + if (this.client.checkExists().forPath(path) == null) { + this.client.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT) + .forPath(path, JsonUtils.toJson(readRequestsEntry.getValue())); + } + byte[] data = this.client.getData().forPath(path); + long requests = JsonUtils.fromJson(data, Long.class); + this.client.setData() + .forPath(path, JsonUtils.toJson(requests + readRequestsEntry.getValue())); + } + } + + @Override + public Map loadFragmentPoints( + IMetaCache cache) throws Exception { + Map writePointsMap = new HashMap<>(); + if (this.client.checkExists().forPath(STATISTICS_FRAGMENT_POINTS_PREFIX) != null) { + List children = client.getChildren().forPath(STATISTICS_FRAGMENT_POINTS_PREFIX); + for (String child : children) { + TimeSeriesRange timeSeriesRange = TimeSeriesInterval.fromString(child); + List fragmentMetas = cache + .getFragmentMapByExactTimeSeriesInterval(timeSeriesRange); + + List timeIntervals = client.getChildren() + .forPath(STATISTICS_FRAGMENT_POINTS_PREFIX + "/" + child); + + for (String timeInterval : timeIntervals) { + long startTime = Long.parseLong(timeInterval); + for (FragmentMeta fragmentMeta : fragmentMetas) { + if (fragmentMeta.getTimeInterval().getStartTime() == startTime) { + byte[] data = this.client.getData() + .forPath(STATISTICS_FRAGMENT_POINTS_PREFIX + "/" + child + "/" + timeInterval); + long points = JsonUtils.fromJson(data, Long.class); + writePointsMap.put(fragmentMeta, points); + } + } + } + } + } + return writePointsMap; + } + + @Override + public void deleteFragmentPoints(TimeSeriesInterval tsInterval, TimeInterval timeInterval) + throws Exception { + String path = + STATISTICS_FRAGMENT_POINTS_PREFIX + "/" + tsInterval.toString() + "/" + timeInterval + .toString(); + if (this.client.checkExists().forPath(path) != null) { + this.client.delete().forPath(path); + } + } + + @Override + public void updateFragmentPoints(FragmentMeta fragmentMeta, long points) throws Exception { + String path = + STATISTICS_FRAGMENT_POINTS_PREFIX + "/" + fragmentMeta.getTsInterval() + .toString() + "/" + fragmentMeta.getTimeInterval().toString(); + if (this.client.checkExists().forPath(path) == null) { + this.client.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT) + .forPath(path, JsonUtils.toJson(points)); + } else { + this.client.setData().forPath(path, JsonUtils.toJson(points)); + } + } + + @Override + public void removeFragmentRequests() throws MetaStorageException { + try { + if (this.client.checkExists().forPath(STATISTICS_FRAGMENT_REQUESTS_PREFIX_WRITE) != null) { + this.client.delete().deletingChildrenIfNeeded() + .forPath(STATISTICS_FRAGMENT_REQUESTS_PREFIX_WRITE); + } + if (this.client.checkExists().forPath(STATISTICS_FRAGMENT_REQUESTS_PREFIX_READ) != null) { + this.client.delete().deletingChildrenIfNeeded() + .forPath(STATISTICS_FRAGMENT_REQUESTS_PREFIX_READ); + } + } catch (Exception e) { + throw new MetaStorageException("encounter error when removing fragment requests: ", e); + } + } + + @Override + public void lockFragmentRequestsCounter() throws MetaStorageException { + try { + fragmentRequestsCounterMutexLock.lock(); + fragmentRequestsCounterMutex.acquire(); + } catch (Exception e) { + fragmentRequestsCounterMutexLock.unlock(); + throw new MetaStorageException( + "encounter error when acquiring fragment requests counter mutex: ", + e); + } + } + + @Override + public void incrementMonitorClearCounter() throws MetaStorageException { + try { + if (this.client.checkExists().forPath(STATISTICS_MONITOR_CLEAR_COUNTER_PREFIX) == null) { + this.client.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT) + .forPath(STATISTICS_MONITOR_CLEAR_COUNTER_PREFIX, JsonUtils.toJson(1)); + } else { + int counter = JsonUtils.fromJson( + this.client.getData().forPath(STATISTICS_MONITOR_CLEAR_COUNTER_PREFIX), + Integer.class); + this.client.setData() + .forPath(STATISTICS_MONITOR_CLEAR_COUNTER_PREFIX, JsonUtils.toJson(counter + 1)); + } + } catch (Exception e) { + throw new MetaStorageException("encounter error when updating monitor clear counter: ", + e); + } + } + + @Override + public int getMonitorClearCounter() throws MetaStorageException { + try { + if (this.client.checkExists().forPath(STATISTICS_MONITOR_CLEAR_COUNTER_PREFIX) == null) { + return 0; + } else { + return JsonUtils.fromJson( + this.client.getData().forPath(STATISTICS_MONITOR_CLEAR_COUNTER_PREFIX), + Integer.class); + } + } catch (Exception e) { + throw new MetaStorageException("encounter error when get monitor clear counter: ", e); + } + } + + @Override + public void incrementFragmentRequestsCounter() throws MetaStorageException { + try { + if (this.client.checkExists().forPath(STATISTICS_FRAGMENT_REQUESTS_COUNTER_PREFIX) == null) { + this.client.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT) + .forPath(STATISTICS_FRAGMENT_REQUESTS_COUNTER_PREFIX, JsonUtils.toJson(1)); + } else { + int counter = JsonUtils.fromJson( + this.client.getData().forPath(STATISTICS_FRAGMENT_REQUESTS_COUNTER_PREFIX), + Integer.class); + this.client.setData() + .forPath(STATISTICS_FRAGMENT_REQUESTS_COUNTER_PREFIX, JsonUtils.toJson(counter + 1)); + } + } catch (Exception e) { + throw new MetaStorageException("encounter error when updating fragment requests counter: ", + e); + } + } + + @Override + public void resetFragmentRequestsCounter() throws MetaStorageException { + try { + if (this.client.checkExists().forPath(STATISTICS_FRAGMENT_REQUESTS_COUNTER_PREFIX) == null) { + this.client.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT) + .forPath(STATISTICS_FRAGMENT_REQUESTS_COUNTER_PREFIX, JsonUtils.toJson(0)); + } else { + this.client.setData() + .forPath(STATISTICS_FRAGMENT_REQUESTS_COUNTER_PREFIX, JsonUtils.toJson(0)); + } + } catch (Exception e) { + throw new MetaStorageException("encounter error when resetting fragment requests counter: ", + e); + } + } + + @Override + public void releaseFragmentRequestsCounter() throws MetaStorageException { + try { + fragmentRequestsCounterMutex.release(); + } catch (Exception e) { + throw new MetaStorageException( + "encounter error when releasing fragment requests counter mutex: ", e); + } finally { + fragmentRequestsCounterMutexLock.unlock(); + } + } + + @Override + public int getFragmentRequestsCounter() throws MetaStorageException { + try { + if (this.client.checkExists().forPath(STATISTICS_FRAGMENT_REQUESTS_COUNTER_PREFIX) == null) { + return 0; + } else { + return JsonUtils.fromJson( + this.client.getData().forPath(STATISTICS_FRAGMENT_REQUESTS_COUNTER_PREFIX), + Integer.class); + } + } catch (Exception e) { + throw new MetaStorageException("encounter error when get fragment requests counter: ", e); + } + } + + @Override + public void updateFragmentHeat(Map writeHotspotMap, + Map readHotspotMap) throws Exception { + for (Entry writeHotspotEntry : writeHotspotMap.entrySet()) { + String path = + STATISTICS_FRAGMENT_HEAT_PREFIX_WRITE + "/" + writeHotspotEntry.getKey().getTsInterval() + .toString() + "/" + writeHotspotEntry.getKey().getTimeInterval().toString(); + if (this.client.checkExists().forPath(path) == null) { + this.client.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT) + .forPath(path, JsonUtils.toJson(writeHotspotEntry.getValue())); + } else { + byte[] data = this.client.getData().forPath(path); + long heat = JsonUtils.fromJson(data, Long.class); + this.client.setData() + .forPath(path, JsonUtils.toJson(heat + writeHotspotEntry.getValue())); + } + } + for (Entry readHotspotEntry : readHotspotMap.entrySet()) { + String path = + STATISTICS_FRAGMENT_HEAT_PREFIX_READ + "/" + readHotspotEntry.getKey().getTsInterval() + .toString() + "/" + readHotspotEntry.getKey().getTimeInterval().toString(); + if (this.client.checkExists().forPath(path) == null) { + this.client.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT) + .forPath(path, JsonUtils.toJson(readHotspotEntry.getValue())); + } else { + byte[] data = this.client.getData().forPath(path); + long heat = JsonUtils.fromJson(data, Long.class); + this.client.setData() + .forPath(path, JsonUtils.toJson(heat + readHotspotEntry.getValue())); + } + } + } + + @Override + public Pair, Map> loadFragmentHeat( + IMetaCache cache) + throws Exception { + Map writeHotspotMap = new HashMap<>(); + if (this.client.checkExists().forPath(STATISTICS_FRAGMENT_HEAT_PREFIX_WRITE) != null) { + List children = client.getChildren().forPath(STATISTICS_FRAGMENT_HEAT_PREFIX_WRITE); + for (String child : children) { + TimeSeriesRange timeSeriesRange = TimeSeriesInterval.fromString(child); + Map> fragmentMapOfTimeSeriesInterval = cache + .getFragmentMapByTimeSeriesInterval(timeSeriesRange); + List fragmentMetas = fragmentMapOfTimeSeriesInterval.get(timeSeriesRange); + + if (fragmentMetas != null) { + List timeIntervals = client.getChildren() + .forPath(STATISTICS_FRAGMENT_HEAT_PREFIX_WRITE + "/" + child); + for (String timeInterval : timeIntervals) { + long startTime = Long.parseLong(timeInterval); + for (FragmentMeta fragmentMeta : fragmentMetas) { + if (fragmentMeta.getTimeInterval().getStartTime() == startTime) { + byte[] data = this.client.getData() + .forPath( + STATISTICS_FRAGMENT_HEAT_PREFIX_WRITE + "/" + child + "/" + timeInterval); + long heat = JsonUtils.fromJson(data, Long.class); + writeHotspotMap.put(fragmentMeta, heat); + } + } + } + } + } + } + + Map readHotspotMap = new HashMap<>(); + if (this.client.checkExists().forPath(STATISTICS_FRAGMENT_HEAT_PREFIX_READ) != null) { + List children = client.getChildren().forPath(STATISTICS_FRAGMENT_HEAT_PREFIX_READ); + for (String child : children) { + TimeSeriesRange timeSeriesRange = TimeSeriesInterval.fromString(child); + Map> fragmentMapOfTimeSeriesInterval = cache + .getFragmentMapByTimeSeriesInterval(timeSeriesRange); + List fragmentMetas = fragmentMapOfTimeSeriesInterval.get(timeSeriesRange); + + if (fragmentMetas != null) { + List timeIntervals = client.getChildren() + .forPath(STATISTICS_FRAGMENT_HEAT_PREFIX_READ + "/" + child); + for (String timeInterval : timeIntervals) { + long startTime = Long.parseLong(timeInterval); + for (FragmentMeta fragmentMeta : fragmentMetas) { + if (fragmentMeta.getTimeInterval().getStartTime() == startTime) { + byte[] data = this.client.getData() + .forPath( + STATISTICS_FRAGMENT_HEAT_PREFIX_READ + "/" + child + "/" + timeInterval); + long heat = JsonUtils.fromJson(data, Long.class); + readHotspotMap.put(fragmentMeta, heat); + } + } + } + } + } + } + return new Pair<>(writeHotspotMap, readHotspotMap); + } + + @Override + public void removeFragmentHeat() throws MetaStorageException { + try { + if (this.client.checkExists().forPath(STATISTICS_FRAGMENT_HEAT_PREFIX_WRITE) != null) { + this.client.delete().deletingChildrenIfNeeded() + .forPath(STATISTICS_FRAGMENT_HEAT_PREFIX_WRITE); + } + if (this.client.checkExists().forPath(STATISTICS_FRAGMENT_HEAT_PREFIX_READ) != null) { + this.client.delete().deletingChildrenIfNeeded() + .forPath(STATISTICS_FRAGMENT_HEAT_PREFIX_READ); + } + } catch (Exception e) { + throw new MetaStorageException("encounter error when removing fragment heat: ", e); + } + } + + @Override + public void lockFragmentHeatCounter() throws MetaStorageException { + try { + fragmentHeatCounterMutexLock.lock(); + fragmentHeatCounterMutex.acquire(); + } catch (Exception e) { + fragmentHeatCounterMutexLock.unlock(); + throw new MetaStorageException("encounter error when acquiring fragment heat counter mutex: ", + e); + } + } + + @Override + public void incrementFragmentHeatCounter() throws MetaStorageException { + try { + if (this.client.checkExists().forPath(STATISTICS_FRAGMENT_HEAT_COUNTER_PREFIX) == null) { + logger.error("create FragmentHeatCounter"); + this.client.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT) + .forPath(STATISTICS_FRAGMENT_HEAT_COUNTER_PREFIX, JsonUtils.toJson(1)); + } else { + logger.error("inc FragmentHeatCounter"); + int counter = JsonUtils.fromJson( + this.client.getData().forPath(STATISTICS_FRAGMENT_HEAT_COUNTER_PREFIX), Integer.class); + logger.error("counter=" + counter); + this.client.setData() + .forPath(STATISTICS_FRAGMENT_HEAT_COUNTER_PREFIX, JsonUtils.toJson(counter + 1)); + } + } catch (Exception e) { + throw new MetaStorageException("encounter error when updating fragment heat counter: ", e); + } + } + + @Override + public void resetFragmentHeatCounter() throws MetaStorageException { + try { + if (this.client.checkExists().forPath(STATISTICS_FRAGMENT_HEAT_COUNTER_PREFIX) == null) { + this.client.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT) + .forPath(STATISTICS_FRAGMENT_HEAT_COUNTER_PREFIX, JsonUtils.toJson(0)); + } else { + this.client.setData() + .forPath(STATISTICS_FRAGMENT_HEAT_COUNTER_PREFIX, JsonUtils.toJson(0)); + } + } catch (Exception e) { + throw new MetaStorageException("encounter error when resetting fragment heat counter: ", e); + } + } + + @Override + public void releaseFragmentHeatCounter() throws MetaStorageException { + try { + fragmentHeatCounterMutex.release(); + } catch (Exception e) { + throw new MetaStorageException("encounter error when releasing latency counter mutex: ", e); + } finally { + fragmentHeatCounterMutexLock.unlock(); + } + } + + @Override + public int getFragmentHeatCounter() throws MetaStorageException { + try { + if (this.client.checkExists().forPath(STATISTICS_FRAGMENT_HEAT_COUNTER_PREFIX) == null) { + return 0; + } else { + return JsonUtils.fromJson( + this.client.getData().forPath(STATISTICS_FRAGMENT_HEAT_COUNTER_PREFIX), Integer.class); + } + } catch (Exception e) { + throw new MetaStorageException("encounter error when get fragment heat counter: ", e); + } + } + + @Override + public boolean proposeToReshard() throws MetaStorageException { + try { + ReshardStatus status; + if (this.client.checkExists().forPath(RESHARD_STATUS_NODE_PREFIX) == null) { + this.client.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT) + .forPath(RESHARD_STATUS_NODE_PREFIX, JsonUtils.toJson(EXECUTING)); + return true; + } else { + status = JsonUtils.fromJson( + this.client.getData().forPath(RESHARD_STATUS_NODE_PREFIX), ReshardStatus.class); + if (status.equals(NON_RESHARDING) || status.equals(JUDGING)) { + this.client.setData() + .forPath(RESHARD_STATUS_NODE_PREFIX, JsonUtils.toJson(EXECUTING)); + return true; + } + return false; + } + } catch (Exception e) { + throw new MetaStorageException("encounter error when proposing to reshard: ", e); + } + } + + @Override + public void lockReshardStatus() throws MetaStorageException { + try { + reshardStatusMutexLock.lock(); + reshardStatusMutex.acquire(); + } catch (Exception e) { + reshardStatusMutexLock.unlock(); + throw new MetaStorageException("encounter error when acquiring reshard status mutex: ", e); + } + } + + @Override + public void updateReshardStatus(ReshardStatus status) throws MetaStorageException { + try { + if (this.client.checkExists().forPath(RESHARD_STATUS_NODE_PREFIX) == null) { + this.client.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT) + .forPath(RESHARD_STATUS_NODE_PREFIX, JsonUtils.toJson(status)); + } else { + this.client.setData() + .forPath(RESHARD_STATUS_NODE_PREFIX, JsonUtils.toJson(status)); + } + } catch (Exception e) { + throw new MetaStorageException("encounter error when updating reshard status: ", e); + } + } + + @Override + public void releaseReshardStatus() throws MetaStorageException { + try { + reshardStatusMutex.release(); + } catch (Exception e) { + throw new MetaStorageException("encounter error when releasing reshard status mutex: ", e); + } finally { + reshardStatusMutexLock.unlock(); + } + } + + @Override + public void removeReshardStatus() throws MetaStorageException { + try { + if (this.client.checkExists().forPath(RESHARD_STATUS_NODE_PREFIX) != null) { + this.client.delete().forPath(RESHARD_STATUS_NODE_PREFIX); + } + } catch (Exception e) { + throw new MetaStorageException("encounter error when removing reshard status: ", e); + } + } + + @Override + public void registerReshardStatusHook(ReshardStatusChangeHook hook) { + this.reshardStatusChangeHook = hook; + } + + private void registerReshardStatusListener() throws Exception { + this.reshardStatusCache = new TreeCache(this.client, RESHARD_STATUS_NODE_PREFIX); + TreeCacheListener listener = (curatorFramework, event) -> { + byte[] data; + ReshardStatus status; + switch (event.getType()) { + case NODE_ADDED: + case NODE_UPDATED: + data = event.getData().getData(); + status = JsonUtils.fromJson(data, ReshardStatus.class); + logger.error("status = {}", status); + reshardStatusChangeHook.onChange(status); + break; + default: + break; + } + }; + this.reshardStatusCache.getListenable().addListener(listener); + this.reshardStatusCache.start(); + } + + @Override + public void lockReshardCounter() throws MetaStorageException { + try { + reshardCounterMutexLock.lock(); + reshardCounterMutex.acquire(); + } catch (Exception e) { + reshardCounterMutexLock.unlock(); + throw new MetaStorageException("encounter error when acquiring reshard counter mutex: ", e); + } + } + + @Override + public void incrementReshardCounter() throws MetaStorageException { + try { + if (this.client.checkExists().forPath(RESHARD_COUNTER_NODE_PREFIX) == null) { + this.client.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT) + .forPath(RESHARD_COUNTER_NODE_PREFIX, JsonUtils.toJson(1)); + } else { + int counter = JsonUtils.fromJson( + this.client.getData().forPath(RESHARD_COUNTER_NODE_PREFIX), Integer.class); + this.client.setData() + .forPath(RESHARD_COUNTER_NODE_PREFIX, JsonUtils.toJson(counter + 1)); + } + } catch (Exception e) { + throw new MetaStorageException("encounter error when updating reshard counter: ", e); + } + } + + @Override + public void resetReshardCounter() throws MetaStorageException { + try { + if (this.client.checkExists().forPath(RESHARD_COUNTER_NODE_PREFIX) == null) { + this.client.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT) + .forPath(RESHARD_COUNTER_NODE_PREFIX, JsonUtils.toJson(0)); + } else { + this.client.setData() + .forPath(RESHARD_COUNTER_NODE_PREFIX, JsonUtils.toJson(0)); + } + } catch (Exception e) { + throw new MetaStorageException("encounter error when resetting reshard counter: ", e); + } + } + + @Override + public void releaseReshardCounter() throws MetaStorageException { + try { + reshardCounterMutex.release(); + } catch (Exception e) { + throw new MetaStorageException("encounter error when releasing reshard counter mutex: ", e); + } finally { + reshardCounterMutexLock.unlock(); + } + } + + @Override + public void removeReshardCounter() throws MetaStorageException { + try { + if (this.client.checkExists().forPath(RESHARD_COUNTER_NODE_PREFIX) != null) { + this.client.delete().forPath(RESHARD_COUNTER_NODE_PREFIX); + } + } catch (Exception e) { + throw new MetaStorageException("encounter error when removing reshard counter: ", e); + } + } + + @Override + public void registerReshardCounterChangeHook(ReshardCounterChangeHook hook) { + this.reshardCounterChangeHook = hook; + } + private void registerMaxActiveEndTimeStatisticsListener() throws Exception { this.maxActiveEndTimeStatisticsCache = new TreeCache(this.client, - MAX_ACTIVE_END_TIME_STATISTICS_NODE_PREFIX); + MAX_ACTIVE_END_TIME_STATISTICS_NODE_PREFIX); TreeCacheListener listener = (curatorFramework, event) -> { if (maxActiveEndTimeStatisticsChangeHook == null) { return; @@ -1365,25 +2097,25 @@ public void lockMaxActiveEndTimeStatistics() throws MetaStorageException { } catch (Exception e) { maxActiveEndTimeStatisticsMutexLock.unlock(); throw new MetaStorageException( - "encounter error when acquiring mac active end time statistics mutex: ", e); + "encounter error when acquiring mac active end time statistics mutex: ", e); } } @Override public void addOrUpdateMaxActiveEndTimeStatistics(long endTime) - throws MetaStorageException { + throws MetaStorageException { try { if (this.client.checkExists() - .forPath(MAX_ACTIVE_END_TIME_STATISTICS_NODE) == null) { + .forPath(MAX_ACTIVE_END_TIME_STATISTICS_NODE) == null) { this.client.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT) - .forPath(MAX_ACTIVE_END_TIME_STATISTICS_NODE, JsonUtils.toJson(endTime)); + .forPath(MAX_ACTIVE_END_TIME_STATISTICS_NODE, JsonUtils.toJson(endTime)); } else { this.client.setData() - .forPath(MAX_ACTIVE_END_TIME_STATISTICS_NODE, JsonUtils.toJson(endTime)); + .forPath(MAX_ACTIVE_END_TIME_STATISTICS_NODE, JsonUtils.toJson(endTime)); } } catch (Exception e) { throw new MetaStorageException( - "encounter error when adding or updating max active end time statistics: ", e); + "encounter error when adding or updating max active end time statistics: ", e); } } @@ -1391,15 +2123,15 @@ public void addOrUpdateMaxActiveEndTimeStatistics(long endTime) public long getMaxActiveEndTimeStatistics() throws MetaStorageException { try { if (this.client.checkExists() - .forPath(MAX_ACTIVE_END_TIME_STATISTICS_NODE) != null) { + .forPath(MAX_ACTIVE_END_TIME_STATISTICS_NODE) != null) { return JsonUtils.fromJson( - this.client.getData() - .forPath(MAX_ACTIVE_END_TIME_STATISTICS_NODE), - Long.class); + this.client.getData() + .forPath(MAX_ACTIVE_END_TIME_STATISTICS_NODE), + Long.class); } } catch (Exception e) { throw new MetaStorageException( - "encounter error when adding or updating max active end time statistics: ", e); + "encounter error when adding or updating max active end time statistics: ", e); } return -1; } @@ -1410,7 +2142,7 @@ public void releaseMaxActiveEndTimeStatistics() throws MetaStorageException { maxActiveEndTimeStatisticsMutex.release(); } catch (Exception e) { throw new MetaStorageException( - "encounter error when releasing max active end time statistics mutex: ", e); + "encounter error when releasing max active end time statistics mutex: ", e); } finally { maxActiveEndTimeStatisticsMutexLock.unlock(); } @@ -1418,10 +2150,30 @@ public void releaseMaxActiveEndTimeStatistics() throws MetaStorageException { @Override public void registerMaxActiveEndTimeStatisticsChangeHook( - MaxActiveEndTimeStatisticsChangeHook hook) throws MetaStorageException { + MaxActiveEndTimeStatisticsChangeHook hook) throws MetaStorageException { this.maxActiveEndTimeStatisticsChangeHook = hook; } + private void registerReshardCounterListener() throws Exception { + this.reshardCounterCache = new TreeCache(this.client, RESHARD_COUNTER_NODE_PREFIX); + TreeCacheListener listener = (curatorFramework, event) -> { + byte[] data; + int counter; + switch (event.getType()) { + case NODE_ADDED: + case NODE_UPDATED: + data = event.getData().getData(); + counter = JsonUtils.fromJson(data, Integer.class); + reshardCounterChangeHook.onChange(counter); + break; + default: + break; + } + }; + this.reshardCounterCache.getListenable().addListener(listener); + this.reshardCounterCache.start(); + } + public static boolean isNumeric(String str) { String bigStr; try { diff --git a/core/src/main/java/cn/edu/tsinghua/iginx/metadata/utils/ReshardStatus.java b/core/src/main/java/cn/edu/tsinghua/iginx/metadata/utils/ReshardStatus.java new file mode 100644 index 000000000..09576c1e2 --- /dev/null +++ b/core/src/main/java/cn/edu/tsinghua/iginx/metadata/utils/ReshardStatus.java @@ -0,0 +1,26 @@ +/* + * 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 cn.edu.tsinghua.iginx.metadata.utils; + +public enum ReshardStatus { + RECOVER, // 恢复阶段 + NON_RESHARDING, // 非重分片阶段 + JUDGING, // 重分片判断阶段 + EXECUTING; // 重分片执行阶段 +} \ No newline at end of file diff --git a/core/src/main/java/cn/edu/tsinghua/iginx/migration/MigrationPolicy.java b/core/src/main/java/cn/edu/tsinghua/iginx/migration/MigrationPolicy.java index fe156e04f..4521c1978 100644 --- a/core/src/main/java/cn/edu/tsinghua/iginx/migration/MigrationPolicy.java +++ b/core/src/main/java/cn/edu/tsinghua/iginx/migration/MigrationPolicy.java @@ -15,13 +15,17 @@ import cn.edu.tsinghua.iginx.engine.shared.source.GlobalSource; import cn.edu.tsinghua.iginx.exceptions.MetaStorageException; import cn.edu.tsinghua.iginx.metadata.DefaultMetaManager; -import cn.edu.tsinghua.iginx.metadata.entity.*; +import cn.edu.tsinghua.iginx.metadata.entity.FragmentMeta; +import cn.edu.tsinghua.iginx.metadata.entity.StorageUnitMeta; +import cn.edu.tsinghua.iginx.metadata.entity.TimeInterval; +import cn.edu.tsinghua.iginx.metadata.entity.TimeSeriesInterval; import cn.edu.tsinghua.iginx.migration.recover.MigrationExecuteTask; import cn.edu.tsinghua.iginx.migration.recover.MigrationExecuteType; import cn.edu.tsinghua.iginx.migration.recover.MigrationLogger; import cn.edu.tsinghua.iginx.policy.IPolicy; import cn.edu.tsinghua.iginx.policy.PolicyManager; import cn.edu.tsinghua.iginx.utils.Pair; + import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -33,458 +37,460 @@ import java.util.SortedSet; import java.util.TreeSet; import java.util.concurrent.ExecutorService; + import org.slf4j.Logger; public abstract class MigrationPolicy { - protected ExecutorService executor; - - protected static final Config config = ConfigDescriptor.getInstance().getConfig(); - - private Logger logger; - - private final IPolicy policy = PolicyManager.getInstance() - .getPolicy(ConfigDescriptor.getInstance().getConfig().getPolicyClassName()); - private final int maxReshardFragmentsNum = config.getMaxReshardFragmentsNum(); - private static final double maxTimeseriesLoadBalanceThreshold = ConfigDescriptor.getInstance() - .getConfig().getMaxTimeseriesLoadBalanceThreshold(); - - private MigrationLogger migrationLogger; - - private final static PhysicalEngine physicalEngine = PhysicalEngineImpl.getInstance(); - - public MigrationPolicy(Logger logger) { - this.logger = logger; - } - - public void setMigrationLogger(MigrationLogger migrationLogger) { - this.migrationLogger = migrationLogger; - } - - public abstract void migrate(List migrationTasks, - Map> nodeFragmentMap, - Map fragmentWriteLoadMap, Map fragmentReadLoadMap); - - /** - * 可定制化副本 - */ - public void reshardByCustomizableReplica(FragmentMeta fragmentMeta, - Map timeseriesLoadMap, Set overLoadTimeseries, long totalLoad, - long points, Map storageHeat) throws MetaStorageException { - try { - migrationLogger.logMigrationExecuteTaskStart( - new MigrationExecuteTask(fragmentMeta, fragmentMeta.getMasterStorageUnitId(), 0L, 0L, - MigrationExecuteType.RESHARD_TIME_SERIES)); - - List timeseries = new ArrayList<>(timeseriesLoadMap.keySet()); - String currStartTimeseries = fragmentMeta.getTsInterval().getStartTimeSeries(); - long currLoad = 0L; - String endTimeseries = fragmentMeta.getTsInterval().getEndTimeSeries(); - long startTime = fragmentMeta.getTimeInterval().getStartTime(); - long endTime = fragmentMeta.getTimeInterval().getEndTime(); - StorageUnitMeta storageUnitMeta = fragmentMeta.getMasterStorageUnit(); - List fakedFragmentMetas = new ArrayList<>(); - List fakedFragmentMetaLoads = new ArrayList<>(); - // 按超负载序列进行分片 - for (int i = 0; i < timeseries.size(); i++) { - if (overLoadTimeseries.contains(timeseries.get(i))) { - fakedFragmentMetas.add(new FragmentMeta(currStartTimeseries, timeseries.get(i), startTime, - endTime, storageUnitMeta)); - fakedFragmentMetaLoads.add(currLoad); - currLoad = 0; - if (i != (timeseries.size() - 1)) { - fakedFragmentMetas - .add(new FragmentMeta(timeseries.get(i), timeseries.get(i + 1), startTime, + protected ExecutorService executor; + + protected static final Config config = ConfigDescriptor.getInstance().getConfig(); + + private Logger logger; + + private final IPolicy policy = PolicyManager.getInstance() + .getPolicy(ConfigDescriptor.getInstance().getConfig().getPolicyClassName()); + private final int maxReshardFragmentsNum = config.getMaxReshardFragmentsNum(); + private static final double maxTimeseriesLoadBalanceThreshold = ConfigDescriptor.getInstance() + .getConfig().getMaxTimeseriesLoadBalanceThreshold(); + + private MigrationLogger migrationLogger; + + private final static PhysicalEngine physicalEngine = PhysicalEngineImpl.getInstance(); + + public MigrationPolicy(Logger logger) { + this.logger = logger; + } + + public void setMigrationLogger(MigrationLogger migrationLogger) { + this.migrationLogger = migrationLogger; + } + + public abstract void migrate(List migrationTasks, + Map> nodeFragmentMap, + Map fragmentWriteLoadMap, Map fragmentReadLoadMap); + + /** + * 可定制化副本 + */ + public void reshardByCustomizableReplica(FragmentMeta fragmentMeta, + Map timeseriesLoadMap, Set overLoadTimeseries, long totalLoad, + long points, Map storageHeat) throws MetaStorageException { + try { + migrationLogger.logMigrationExecuteTaskStart( + new MigrationExecuteTask(fragmentMeta, fragmentMeta.getMasterStorageUnitId(), 0L, 0L, + MigrationExecuteType.RESHARD_TIME_SERIES)); + + List timeseries = new ArrayList<>(timeseriesLoadMap.keySet()); + String currStartTimeseries = fragmentMeta.getTsInterval().getStartTimeSeries(); + long currLoad = 0L; + String endTimeseries = fragmentMeta.getTsInterval().getEndTimeSeries(); + long startTime = fragmentMeta.getTimeInterval().getStartTime(); + long endTime = fragmentMeta.getTimeInterval().getEndTime(); + StorageUnitMeta storageUnitMeta = fragmentMeta.getMasterStorageUnit(); + List fakedFragmentMetas = new ArrayList<>(); + List fakedFragmentMetaLoads = new ArrayList<>(); + // 按超负载序列进行分片 + for (int i = 0; i < timeseries.size(); i++) { + if (overLoadTimeseries.contains(timeseries.get(i))) { + fakedFragmentMetas.add(new FragmentMeta(currStartTimeseries, timeseries.get(i), startTime, + endTime, storageUnitMeta)); + fakedFragmentMetaLoads.add(currLoad); + currLoad = 0; + if (i != (timeseries.size() - 1)) { + fakedFragmentMetas + .add(new FragmentMeta(timeseries.get(i), timeseries.get(i + 1), startTime, + endTime, storageUnitMeta)); + fakedFragmentMetaLoads.add(timeseriesLoadMap.get(timeseries.get(i))); + currStartTimeseries = timeseries.get(i + 1); + } else { + currStartTimeseries = timeseries.get(i); + currLoad = timeseriesLoadMap.get(timeseries.get(i)); + } + } + currLoad += timeseriesLoadMap.get(timeseries.get(i)); + } + fakedFragmentMetas.add(new FragmentMeta(currStartTimeseries, endTimeseries, startTime, endTime, storageUnitMeta)); - fakedFragmentMetaLoads.add(timeseriesLoadMap.get(timeseries.get(i))); - currStartTimeseries = timeseries.get(i + 1); - } else { - currStartTimeseries = timeseries.get(i); - currLoad = timeseriesLoadMap.get(timeseries.get(i)); - } - } - currLoad += timeseriesLoadMap.get(timeseries.get(i)); - } - fakedFragmentMetas.add(new FragmentMeta(currStartTimeseries, endTimeseries, startTime, - endTime, storageUnitMeta)); - fakedFragmentMetaLoads.add(currLoad); - - // 模拟进行时间序列分片 - while (fakedFragmentMetas.size() > maxReshardFragmentsNum) { - double currAverageLoad = totalLoad * 1.0 / fakedFragmentMetaLoads.size(); - boolean canMergeFragments = false; - for (int i = 0; i < fakedFragmentMetaLoads.size(); i++) { - FragmentMeta currFragmentMeta = fakedFragmentMetas.get(i); - // 合并时间序列分片 - if (fakedFragmentMetaLoads.get(i) <= currAverageLoad * (1 - + maxTimeseriesLoadBalanceThreshold) && currFragmentMeta.getTsInterval() - .getStartTimeSeries().equals(currFragmentMeta.getTsInterval().getEndTimeSeries())) { - - // 与他最近的负载最低的时间分区进行合并 - if (i == (fakedFragmentMetaLoads.size() - 1) - || fakedFragmentMetaLoads.get(i + 1) > fakedFragmentMetaLoads.get(i - 1)) { - FragmentMeta toMergeFragmentMeta = fakedFragmentMetas.get(i - 1); - toMergeFragmentMeta.getTsInterval().setEndTimeSeries( - fakedFragmentMetas.get(i).getTsInterval().getEndTimeSeries()); - fakedFragmentMetas.remove(i); - fakedFragmentMetaLoads - .set(i - 1, fakedFragmentMetaLoads.get(i - 1) + fakedFragmentMetaLoads.get(i)); - fakedFragmentMetaLoads.remove(i); - } else if (fakedFragmentMetaLoads.get(i + 1) <= fakedFragmentMetaLoads.get(i - 1)) { - FragmentMeta toMergeFragmentMeta = fakedFragmentMetas.get(i); - toMergeFragmentMeta.getTsInterval().setEndTimeSeries( - fakedFragmentMetas.get(i + 1).getTsInterval().getEndTimeSeries()); - fakedFragmentMetas.remove(i + 1); - fakedFragmentMetaLoads - .set(i, fakedFragmentMetaLoads.get(i) + fakedFragmentMetaLoads.get(i + 1)); - fakedFragmentMetaLoads.remove(i + 1); + fakedFragmentMetaLoads.add(currLoad); + + // 模拟进行时间序列分片 + while (fakedFragmentMetas.size() > maxReshardFragmentsNum) { + double currAverageLoad = totalLoad * 1.0 / fakedFragmentMetaLoads.size(); + boolean canMergeFragments = false; + for (int i = 0; i < fakedFragmentMetaLoads.size(); i++) { + FragmentMeta currFragmentMeta = fakedFragmentMetas.get(i); + // 合并时间序列分片 + if (fakedFragmentMetaLoads.get(i) <= currAverageLoad * (1 + + maxTimeseriesLoadBalanceThreshold) && currFragmentMeta.getTsInterval() + .getStartTimeSeries().equals(currFragmentMeta.getTsInterval().getEndTimeSeries())) { + + // 与他最近的负载最低的时间分区进行合并 + if (i == (fakedFragmentMetaLoads.size() - 1) + || fakedFragmentMetaLoads.get(i + 1) > fakedFragmentMetaLoads.get(i - 1)) { + FragmentMeta toMergeFragmentMeta = fakedFragmentMetas.get(i - 1); + toMergeFragmentMeta.getTsInterval().setEndTimeSeries( + fakedFragmentMetas.get(i).getTsInterval().getEndTimeSeries()); + fakedFragmentMetas.remove(i); + fakedFragmentMetaLoads + .set(i - 1, fakedFragmentMetaLoads.get(i - 1) + fakedFragmentMetaLoads.get(i)); + fakedFragmentMetaLoads.remove(i); + } else if (fakedFragmentMetaLoads.get(i + 1) <= fakedFragmentMetaLoads.get(i - 1)) { + FragmentMeta toMergeFragmentMeta = fakedFragmentMetas.get(i); + toMergeFragmentMeta.getTsInterval().setEndTimeSeries( + fakedFragmentMetas.get(i + 1).getTsInterval().getEndTimeSeries()); + fakedFragmentMetas.remove(i + 1); + fakedFragmentMetaLoads + .set(i, fakedFragmentMetaLoads.get(i) + fakedFragmentMetaLoads.get(i + 1)); + fakedFragmentMetaLoads.remove(i + 1); + } + + // 需要合并 + canMergeFragments = true; + } + } + // 合并最小分片 + if (canMergeFragments) { + long maxTwoFragmentLoads = 0L; + int startIndex = 0; + for (int i = 0; i < fakedFragmentMetaLoads.size(); i++) { + if (i < fakedFragmentMetaLoads.size() - 1) { + long currTwoFragmentLoad = + fakedFragmentMetaLoads.get(i) + fakedFragmentMetaLoads.get(i + 1); + if (currTwoFragmentLoad > maxTwoFragmentLoads) { + maxTwoFragmentLoads = currTwoFragmentLoad; + startIndex = i; + } + } + } + FragmentMeta toMergeFragmentMeta = fakedFragmentMetas.get(startIndex); + toMergeFragmentMeta.getTsInterval().setEndTimeSeries( + fakedFragmentMetas.get(startIndex + 1).getTsInterval().getEndTimeSeries()); + fakedFragmentMetas.remove(startIndex + 1); + fakedFragmentMetaLoads.set(startIndex, + fakedFragmentMetaLoads.get(startIndex) + fakedFragmentMetaLoads.get(startIndex + 1)); + fakedFragmentMetaLoads.remove(startIndex + 1); + } } - // 需要合并 - canMergeFragments = true; - } - } - // 合并最小分片 - if (canMergeFragments) { - long maxTwoFragmentLoads = 0L; - int startIndex = 0; - for (int i = 0; i < fakedFragmentMetaLoads.size(); i++) { - if (i < fakedFragmentMetaLoads.size() - 1) { - long currTwoFragmentLoad = - fakedFragmentMetaLoads.get(i) + fakedFragmentMetaLoads.get(i + 1); - if (currTwoFragmentLoad > maxTwoFragmentLoads) { - maxTwoFragmentLoads = currTwoFragmentLoad; - startIndex = i; - } + // 给每个节点负载做排序以方便后续迁移 + // 去掉本身节点 + storageHeat.remove(fragmentMeta.getMasterStorageUnit().getStorageEngineId()); + List> storageHeatEntryList = new ArrayList<>(storageHeat.entrySet()); + storageHeatEntryList.sort(Entry.comparingByValue()); + + // 开始实际切分片 + double currAverageLoad = totalLoad * 1.0 / fakedFragmentMetaLoads.size(); + TimeSeriesInterval sourceTsInterval = new TimeSeriesInterval( + fragmentMeta.getTsInterval().getStartTimeSeries(), + fragmentMeta.getTsInterval().getEndTimeSeries()); + for (int i = 0; i < fakedFragmentMetas.size(); i++) { + FragmentMeta targetFragmentMeta = fakedFragmentMetas.get(i); + if (i == 0) { + DefaultMetaManager.getInstance().endFragmentByTimeSeriesInterval(fragmentMeta, + targetFragmentMeta.getTsInterval().getEndTimeSeries()); + DefaultMetaManager.getInstance() + .updateFragmentByTsInterval(sourceTsInterval, fragmentMeta); + } else { + FragmentMeta newFragment = new FragmentMeta( + targetFragmentMeta.getTsInterval().getStartTimeSeries(), + targetFragmentMeta.getTsInterval().getEndTimeSeries(), + fragmentMeta.getTimeInterval().getStartTime(), + fragmentMeta.getTimeInterval().getEndTime(), fragmentMeta.getMasterStorageUnit()); + DefaultMetaManager.getInstance().addFragment(newFragment); + } + // 开始拷贝副本,有一个先决条件,时间分区必须为闭区间,即只有查询请求,在之后不会被再写入数据,也不会被拆分读写 + if (fakedFragmentMetaLoads.get(i) >= currAverageLoad * (1 + + maxTimeseriesLoadBalanceThreshold)) { + int replicas = (int) (fakedFragmentMetaLoads.get(i) / currAverageLoad); + for (int num = 1; num < replicas; num++) { + long targetStorageId = storageHeatEntryList.get(num % storageHeatEntryList.size()) + .getKey(); + StorageUnitMeta newStorageUnitMeta = DefaultMetaManager.getInstance() + .generateNewStorageUnitMetaByFragment(fragmentMeta, targetStorageId); + FragmentMeta newFragment = new FragmentMeta( + targetFragmentMeta.getTsInterval().getStartTimeSeries(), + targetFragmentMeta.getTsInterval().getEndTimeSeries(), + fragmentMeta.getTimeInterval().getStartTime(), + fragmentMeta.getTimeInterval().getEndTime(), newStorageUnitMeta); + DefaultMetaManager.getInstance().addFragment(newFragment); + } + } } - } - FragmentMeta toMergeFragmentMeta = fakedFragmentMetas.get(startIndex); - toMergeFragmentMeta.getTsInterval().setEndTimeSeries( - fakedFragmentMetas.get(startIndex + 1).getTsInterval().getEndTimeSeries()); - fakedFragmentMetas.remove(startIndex + 1); - fakedFragmentMetaLoads.set(startIndex, - fakedFragmentMetaLoads.get(startIndex) + fakedFragmentMetaLoads.get(startIndex + 1)); - fakedFragmentMetaLoads.remove(startIndex + 1); - } - } - - // 给每个节点负载做排序以方便后续迁移 - // 去掉本身节点 - storageHeat.remove(fragmentMeta.getMasterStorageUnit().getStorageEngineId()); - List> storageHeatEntryList = new ArrayList<>(storageHeat.entrySet()); - storageHeatEntryList.sort(Entry.comparingByValue()); - - // 开始实际切分片 - double currAverageLoad = totalLoad * 1.0 / fakedFragmentMetaLoads.size(); - TimeSeriesInterval sourceTsInterval = new TimeSeriesInterval( - fragmentMeta.getTsInterval().getStartTimeSeries(), - fragmentMeta.getTsInterval().getEndTimeSeries()); - for (int i = 0; i < fakedFragmentMetas.size(); i++) { - FragmentMeta targetFragmentMeta = fakedFragmentMetas.get(i); - if (i == 0) { - DefaultMetaManager.getInstance().endFragmentByTimeSeriesInterval(fragmentMeta, - targetFragmentMeta.getTsInterval().getEndTimeSeries()); - DefaultMetaManager.getInstance() - .updateFragmentByTsInterval(sourceTsInterval, fragmentMeta); - } else { - FragmentMeta newFragment = new FragmentMeta( - targetFragmentMeta.getTsInterval().getStartTimeSeries(), - targetFragmentMeta.getTsInterval().getEndTimeSeries(), - fragmentMeta.getTimeInterval().getStartTime(), - fragmentMeta.getTimeInterval().getEndTime(), fragmentMeta.getMasterStorageUnit()); - DefaultMetaManager.getInstance().addFragment(newFragment); - } - // 开始拷贝副本,有一个先决条件,时间分区必须为闭区间,即只有查询请求,在之后不会被再写入数据,也不会被拆分读写 - if (fakedFragmentMetaLoads.get(i) >= currAverageLoad * (1 - + maxTimeseriesLoadBalanceThreshold)) { - int replicas = (int) (fakedFragmentMetaLoads.get(i) / currAverageLoad); - for (int num = 1; num < replicas; num++) { - long targetStorageId = storageHeatEntryList.get(num % storageHeatEntryList.size()) - .getKey(); - StorageUnitMeta newStorageUnitMeta = DefaultMetaManager.getInstance() - .generateNewStorageUnitMetaByFragment(fragmentMeta, targetStorageId); - FragmentMeta newFragment = new FragmentMeta( - targetFragmentMeta.getTsInterval().getStartTimeSeries(), - targetFragmentMeta.getTsInterval().getEndTimeSeries(), - fragmentMeta.getTimeInterval().getStartTime(), - fragmentMeta.getTimeInterval().getEndTime(), newStorageUnitMeta); - DefaultMetaManager.getInstance().addFragment(newFragment); - } - } - } - } finally { - migrationLogger.logMigrationExecuteTaskEnd(); + } finally { + migrationLogger.logMigrationExecuteTaskEnd(); + } } - } - - /** - * 在时间序列层面将分片在同一个du下分为两块(未知时间序列, 写入场景) - */ - public void reshardWriteByTimeseries(FragmentMeta fragmentMeta, long points) - throws PhysicalException { - // 分区不存在直接返回 + + /** + * 在时间序列层面将分片在同一个du下分为两块(未知时间序列, 写入场景) + */ + public void reshardWriteByTimeseries(FragmentMeta fragmentMeta, long points) + throws PhysicalException { + // 分区不存在直接返回 // if (!DefaultMetaManager.getInstance() // .checkFragmentExistenceByTimeInterval(fragmentMeta.getTsInterval())) { // return; // } - try { - logger.info("start to reshard timeseries by write"); - migrationLogger.logMigrationExecuteTaskStart( - new MigrationExecuteTask(fragmentMeta, fragmentMeta.getMasterStorageUnitId(), 0L, 0L, - MigrationExecuteType.RESHARD_TIME_SERIES)); - - Set pathRegexSet = new HashSet<>(); - pathRegexSet.add(fragmentMeta.getMasterStorageUnitId()); - ShowTimeSeries showTimeSeries = new ShowTimeSeries(new GlobalSource(), - pathRegexSet, null, Integer.MAX_VALUE, 0); - RowStream rowStream = physicalEngine.execute(showTimeSeries); - SortedSet pathSet = new TreeSet<>(); - while (rowStream.hasNext()) { - Row row = rowStream.next(); - String timeSeries = new String((byte[]) row.getValue(0)); - if (fragmentMeta.getTsInterval().isContain(timeSeries)) { - pathSet.add(timeSeries); + try { + logger.info("start to reshard timeseries by write"); + migrationLogger.logMigrationExecuteTaskStart( + new MigrationExecuteTask(fragmentMeta, fragmentMeta.getMasterStorageUnitId(), 0L, 0L, + MigrationExecuteType.RESHARD_TIME_SERIES)); + + Set pathRegexSet = new HashSet<>(); + pathRegexSet.add(fragmentMeta.getMasterStorageUnitId()); + ShowTimeSeries showTimeSeries = new ShowTimeSeries(new GlobalSource(), + pathRegexSet, null, Integer.MAX_VALUE, 0); + RowStream rowStream = physicalEngine.execute(showTimeSeries); + SortedSet pathSet = new TreeSet<>(); + while (rowStream.hasNext()) { + Row row = rowStream.next(); + String timeSeries = new String((byte[]) row.getValue(0)); + if (fragmentMeta.getTsInterval().isContain(timeSeries)) { + pathSet.add(timeSeries); + } + } + logger.info("start to add new fragment"); + String middleTimeseries = new ArrayList<>(pathSet).get(pathSet.size() / 2); + logger.info("timeseries split middleTimeseries=" + middleTimeseries); + TimeSeriesInterval sourceTsInterval = new TimeSeriesInterval( + fragmentMeta.getTsInterval().getStartTimeSeries(), + fragmentMeta.getTsInterval().getEndTimeSeries()); + FragmentMeta newFragment = new FragmentMeta(middleTimeseries, + sourceTsInterval.getEndTimeSeries(), + fragmentMeta.getTimeInterval().getStartTime(), + fragmentMeta.getTimeInterval().getEndTime(), fragmentMeta.getMasterStorageUnit()); + logger.info("timeseries split new fragment=" + newFragment.toString()); + DefaultMetaManager.getInstance().addFragment(newFragment); + logger.info("start to add old fragment"); + DefaultMetaManager.getInstance() + .endFragmentByTimeSeriesInterval(fragmentMeta, middleTimeseries); + } finally { + migrationLogger.logMigrationExecuteTaskEnd(); } - } - logger.info("start to add new fragment"); - String middleTimeseries = new ArrayList<>(pathSet).get(pathSet.size() / 2); - logger.info("timeseries split middleTimeseries=" + middleTimeseries); - TimeSeriesInterval sourceTsInterval = new TimeSeriesInterval( - fragmentMeta.getTsInterval().getStartTimeSeries(), - fragmentMeta.getTsInterval().getEndTimeSeries()); - FragmentMeta newFragment = new FragmentMeta(middleTimeseries, - sourceTsInterval.getEndTimeSeries(), - fragmentMeta.getTimeInterval().getStartTime(), - fragmentMeta.getTimeInterval().getEndTime(), fragmentMeta.getMasterStorageUnit()); - logger.info("timeseries split new fragment=" + newFragment.toString()); - DefaultMetaManager.getInstance().addFragment(newFragment); - logger.info("start to add old fragment"); - DefaultMetaManager.getInstance() - .endFragmentByTimeSeriesInterval(fragmentMeta, middleTimeseries); - } finally { - migrationLogger.logMigrationExecuteTaskEnd(); } - } - - /** - * 在时间序列层面将分片在同一个du下分为两块(已知时间序列) - */ - public void reshardQueryByTimeseries(FragmentMeta fragmentMeta, - Map timeseriesLoadMap) { - try { - migrationLogger.logMigrationExecuteTaskStart( - new MigrationExecuteTask(fragmentMeta, fragmentMeta.getMasterStorageUnitId(), 0L, 0L, - MigrationExecuteType.RESHARD_TIME_SERIES)); - long totalLoad = 0L; - for (Entry timeseriesLoadEntry : timeseriesLoadMap.entrySet()) { - totalLoad += timeseriesLoadEntry.getValue(); - } - String middleTimeseries = null; - long currLoad = 0L; - for (Entry timeseriesLoadEntry : timeseriesLoadMap.entrySet()) { - currLoad += timeseriesLoadEntry.getValue(); - if (currLoad >= totalLoad / 2) { - middleTimeseries = timeseriesLoadEntry.getKey(); - break; + + /** + * 在时间序列层面将分片在同一个du下分为两块(已知时间序列) + */ + public void reshardQueryByTimeseries(FragmentMeta fragmentMeta, + Map timeseriesLoadMap) { + try { + migrationLogger.logMigrationExecuteTaskStart( + new MigrationExecuteTask(fragmentMeta, fragmentMeta.getMasterStorageUnitId(), 0L, 0L, + MigrationExecuteType.RESHARD_TIME_SERIES)); + long totalLoad = 0L; + for (Entry timeseriesLoadEntry : timeseriesLoadMap.entrySet()) { + totalLoad += timeseriesLoadEntry.getValue(); + } + String middleTimeseries = null; + long currLoad = 0L; + for (Entry timeseriesLoadEntry : timeseriesLoadMap.entrySet()) { + currLoad += timeseriesLoadEntry.getValue(); + if (currLoad >= totalLoad / 2) { + middleTimeseries = timeseriesLoadEntry.getKey(); + break; + } + } + + TimeSeriesInterval sourceTsInterval = new TimeSeriesInterval( + fragmentMeta.getTsInterval().getStartTimeSeries(), + fragmentMeta.getTsInterval().getEndTimeSeries()); + FragmentMeta newFragment = new FragmentMeta(middleTimeseries, + sourceTsInterval.getEndTimeSeries(), + fragmentMeta.getTimeInterval().getStartTime(), + fragmentMeta.getTimeInterval().getEndTime(), fragmentMeta.getMasterStorageUnit()); + DefaultMetaManager.getInstance().addFragment(newFragment); + DefaultMetaManager.getInstance() + .endFragmentByTimeSeriesInterval(fragmentMeta, middleTimeseries); + DefaultMetaManager.getInstance().updateFragmentByTsInterval(sourceTsInterval, fragmentMeta); + } finally { + migrationLogger.logMigrationExecuteTaskEnd(); } - } - - TimeSeriesInterval sourceTsInterval = new TimeSeriesInterval( - fragmentMeta.getTsInterval().getStartTimeSeries(), - fragmentMeta.getTsInterval().getEndTimeSeries()); - FragmentMeta newFragment = new FragmentMeta(middleTimeseries, - sourceTsInterval.getEndTimeSeries(), - fragmentMeta.getTimeInterval().getStartTime(), - fragmentMeta.getTimeInterval().getEndTime(), fragmentMeta.getMasterStorageUnit()); - DefaultMetaManager.getInstance().addFragment(newFragment); - DefaultMetaManager.getInstance() - .endFragmentByTimeSeriesInterval(fragmentMeta, middleTimeseries); - DefaultMetaManager.getInstance().updateFragmentByTsInterval(sourceTsInterval, fragmentMeta); - } finally { - migrationLogger.logMigrationExecuteTaskEnd(); } - } - public void interrupt() { - executor.shutdown(); - } + public void interrupt() { + executor.shutdown(); + } - protected boolean canExecuteTargetMigrationTask(MigrationTask migrationTask, - Map nodeLoadMap) { + protected boolean canExecuteTargetMigrationTask(MigrationTask migrationTask, + Map nodeLoadMap) { // long currTargetNodeLoad = nodeLoadMap.getOrDefault(migrationTask.getTargetStorageId(), 0L); // logger.error("currTargetNodeLoad = {}", currTargetNodeLoad); // logger.error("migrationTask.getLoad() = {}", migrationTask.getLoad()); // logger.error("config.getMaxLoadThreshold() = {}", config.getMaxLoadThreshold()); - return true; - } - - protected boolean isAllQueueEmpty(List> migrationTaskQueueList) { - for (Queue migrationTaskQueue : migrationTaskQueueList) { - if (!migrationTaskQueue.isEmpty()) { - return false; - } + return true; } - return true; - } - - protected void sortQueueListByFirstItem(List> migrationTaskQueue) { - migrationTaskQueue - .sort((o1, o2) -> { - MigrationTask migrationTask1 = o1.peek(); - MigrationTask migrationTask2 = o2.peek(); - if (migrationTask1 == null || migrationTask2 == null) { - return 1; - } else { - return (int) (migrationTask2.getPriorityScore() - migrationTask1.getPriorityScore()); - } - }); - } - - protected Map calculateNodeLoadMap(Map> nodeFragmentMap, - Map fragmentWriteLoadMap, Map fragmentReadLoadMap) { - Map nodeLoadMap = new HashMap<>(); - for (Entry> nodeFragmentEntry : nodeFragmentMap.entrySet()) { - List fragmentMetas = nodeFragmentEntry.getValue(); - for (FragmentMeta fragmentMeta : fragmentMetas) { - nodeLoadMap.put(nodeFragmentEntry.getKey(), - fragmentWriteLoadMap.getOrDefault(fragmentMeta, 0L) + fragmentReadLoadMap - .getOrDefault(fragmentMeta, 0L)); - } + + protected boolean isAllQueueEmpty(List> migrationTaskQueueList) { + for (Queue migrationTaskQueue : migrationTaskQueueList) { + if (!migrationTaskQueue.isEmpty()) { + return false; + } + } + return true; } - return nodeLoadMap; - } - - protected synchronized void executeOneRoundMigration( - List> migrationTaskQueueList, - Map nodeLoadMap) { - for (Queue migrationTaskQueue : migrationTaskQueueList) { - MigrationTask migrationTask = migrationTaskQueue.peek(); - //根据负载判断是否能进行该任务 - if (migrationTask != null && canExecuteTargetMigrationTask(migrationTask, nodeLoadMap)) { - migrationTaskQueue.poll(); - this.executor.submit(() -> { - this.logger.info("start migration: {}", migrationTask); - //异步执行耗时的操作 - if (migrationTask.getMigrationType() == MigrationType.QUERY) { - // 如果之前没切过分区,需要优先切一下分区 - if (migrationTask.getFragmentMeta().getTimeInterval().getEndTime() == Long.MAX_VALUE) { - this.logger.error("start to reshard query data: {}", migrationTask); - FragmentMeta fragmentMeta = reshardFragment(migrationTask.getSourceStorageId(), - migrationTask.getTargetStorageId(), - migrationTask.getFragmentMeta()); - migrationTask.setFragmentMeta(fragmentMeta); + + protected void sortQueueListByFirstItem(List> migrationTaskQueue) { + migrationTaskQueue + .sort((o1, o2) -> { + MigrationTask migrationTask1 = o1.peek(); + MigrationTask migrationTask2 = o2.peek(); + if (migrationTask1 == null || migrationTask2 == null) { + return 1; + } else { + return (int) (migrationTask2.getPriorityScore() - migrationTask1.getPriorityScore()); + } + }); + } + + protected Map calculateNodeLoadMap(Map> nodeFragmentMap, + Map fragmentWriteLoadMap, Map fragmentReadLoadMap) { + Map nodeLoadMap = new HashMap<>(); + for (Entry> nodeFragmentEntry : nodeFragmentMap.entrySet()) { + List fragmentMetas = nodeFragmentEntry.getValue(); + for (FragmentMeta fragmentMeta : fragmentMetas) { + nodeLoadMap.put(nodeFragmentEntry.getKey(), + fragmentWriteLoadMap.getOrDefault(fragmentMeta, 0L) + fragmentReadLoadMap + .getOrDefault(fragmentMeta, 0L)); } - this.logger.error("start to migrate data: {}", migrationTask); - migrateData(migrationTask.getSourceStorageId(), - migrationTask.getTargetStorageId(), - migrationTask.getFragmentMeta()); - } else { - this.logger.error("start to migrate write data: {}", migrationTask); - reshardFragment(migrationTask.getSourceStorageId(), - migrationTask.getTargetStorageId(), - migrationTask.getFragmentMeta()); - } - this.logger - .error("complete one migration task from {} to {} with load: {}, size: {}, type: {}", - migrationTask.getSourceStorageId(), migrationTask.getTargetStorageId(), - migrationTask.getLoad(), migrationTask.getSize(), - migrationTask.getMigrationType()); - // 执行下一轮判断 - while (!isAllQueueEmpty(migrationTaskQueueList)) { - executeOneRoundMigration(migrationTaskQueueList, nodeLoadMap); - } - }); - } + } + return nodeLoadMap; } - sortQueueListByFirstItem(migrationTaskQueueList); - } - - private void migrateData(long sourceStorageId, long targetStorageId, - FragmentMeta fragmentMeta) { - try { - // 在目标节点创建新du - StorageUnitMeta storageUnitMeta; - try { - storageUnitMeta = DefaultMetaManager.getInstance() - .generateNewStorageUnitMetaByFragment(fragmentMeta, targetStorageId); - } catch (MetaStorageException e) { - logger.error("cannot create storage unit in target storage engine", e); - throw new PhysicalException(e); - } - migrationLogger.logMigrationExecuteTaskStart( - new MigrationExecuteTask(fragmentMeta, storageUnitMeta.getId(), sourceStorageId, - targetStorageId, - MigrationExecuteType.MIGRATION)); - - Set pathRegexSet = new HashSet<>(); - pathRegexSet.add(fragmentMeta.getMasterStorageUnitId()); - ShowTimeSeries showTimeSeries = new ShowTimeSeries(new GlobalSource(), pathRegexSet, null, - Integer.MAX_VALUE, 0); - RowStream rowStream = physicalEngine.execute(showTimeSeries); - SortedSet pathSet = new TreeSet<>(); - rowStream.getHeader().getFields().forEach(field -> { - String timeSeries = field.getName(); - if (fragmentMeta.getTsInterval().isContain(timeSeries)) { - pathSet.add(timeSeries); + + protected synchronized void executeOneRoundMigration( + List> migrationTaskQueueList, + Map nodeLoadMap) { + for (Queue migrationTaskQueue : migrationTaskQueueList) { + MigrationTask migrationTask = migrationTaskQueue.peek(); + //根据负载判断是否能进行该任务 + if (migrationTask != null && canExecuteTargetMigrationTask(migrationTask, nodeLoadMap)) { + migrationTaskQueue.poll(); + this.executor.submit(() -> { + this.logger.info("start migration: {}", migrationTask); + //异步执行耗时的操作 + if (migrationTask.getMigrationType() == MigrationType.QUERY) { + // 如果之前没切过分区,需要优先切一下分区 + if (migrationTask.getFragmentMeta().getTimeInterval().getEndTime() == Long.MAX_VALUE) { + this.logger.error("start to reshard query data: {}", migrationTask); + FragmentMeta fragmentMeta = reshardFragment(migrationTask.getSourceStorageId(), + migrationTask.getTargetStorageId(), + migrationTask.getFragmentMeta()); + migrationTask.setFragmentMeta(fragmentMeta); + } + this.logger.error("start to migrate data: {}", migrationTask); + migrateData(migrationTask.getSourceStorageId(), + migrationTask.getTargetStorageId(), + migrationTask.getFragmentMeta()); + } else { + this.logger.error("start to migrate write data: {}", migrationTask); + reshardFragment(migrationTask.getSourceStorageId(), + migrationTask.getTargetStorageId(), + migrationTask.getFragmentMeta()); + } + this.logger + .error("complete one migration task from {} to {} with load: {}, size: {}, type: {}", + migrationTask.getSourceStorageId(), migrationTask.getTargetStorageId(), + migrationTask.getLoad(), migrationTask.getSize(), + migrationTask.getMigrationType()); + // 执行下一轮判断 + while (!isAllQueueEmpty(migrationTaskQueueList)) { + executeOneRoundMigration(migrationTaskQueueList, nodeLoadMap); + } + }); + } } - }); - // 开始迁移数据 - Migration migration = new Migration(new GlobalSource(), sourceStorageId, targetStorageId, - fragmentMeta, new ArrayList<>(pathSet), storageUnitMeta); - physicalEngine.execute(migration); - // 迁移完开始删除原数据 - - List paths = new ArrayList<>(); - paths.add(fragmentMeta.getMasterStorageUnitId() + "*"); - List timeRanges = new ArrayList<>(); - timeRanges.add(new TimeRange(fragmentMeta.getTimeInterval().getStartTime(), true, - fragmentMeta.getTimeInterval().getEndTime(), false)); - Delete delete = new Delete(new FragmentSource(fragmentMeta), timeRanges, paths, null); - physicalEngine.execute(delete); - } catch (Exception e) { - logger.error("encounter error when migrate data from {} to {} ", sourceStorageId, - targetStorageId, e); - } finally { - migrationLogger.logMigrationExecuteTaskEnd(); + sortQueueListByFirstItem(migrationTaskQueueList); } - } - - private FragmentMeta reshardFragment(long sourceStorageId, long targetStorageId, - FragmentMeta fragmentMeta) { - try { - migrationLogger.logMigrationExecuteTaskStart( - new MigrationExecuteTask(fragmentMeta, fragmentMeta.getMasterStorageUnitId(), - sourceStorageId, targetStorageId, - MigrationExecuteType.RESHARD_TIME)); - // [startTime, +∞) & (startPath, endPath) - TimeSeriesRange tsInterval = fragmentMeta.getTsInterval(); - TimeInterval timeInterval = fragmentMeta.getTimeInterval(); - List storageEngineList = new ArrayList<>(); - storageEngineList.add(targetStorageId); - - // 排除乱序写入问题 - if (timeInterval.getEndTime() == Long.MAX_VALUE) { - operateTaskAndRequest(sourceStorageId, targetStorageId, fragmentMeta); - Pair fragmentMetaStorageUnitMetaPair = policy - .generateFragmentAndStorageUnitByTimeSeriesIntervalAndTimeInterval( - tsInterval.getStartTimeSeries(), tsInterval.getEndTimeSeries(), - DefaultMetaManager.getInstance().getMaxActiveEndTime(), Long.MAX_VALUE, - storageEngineList); - logger.info("start to splitFragmentAndStorageUnit"); - return DefaultMetaManager.getInstance() - .splitFragmentAndStorageUnit(fragmentMetaStorageUnitMetaPair.getV(), - fragmentMetaStorageUnitMetaPair.getK(), fragmentMeta); - } - return null; - } finally { - migrationLogger.logMigrationExecuteTaskEnd(); + + private void migrateData(long sourceStorageId, long targetStorageId, + FragmentMeta fragmentMeta) { + try { + // 在目标节点创建新du + StorageUnitMeta storageUnitMeta; + try { + storageUnitMeta = DefaultMetaManager.getInstance() + .generateNewStorageUnitMetaByFragment(fragmentMeta, targetStorageId); + } catch (MetaStorageException e) { + logger.error("cannot create storage unit in target storage engine", e); + throw new PhysicalException(e); + } + migrationLogger.logMigrationExecuteTaskStart( + new MigrationExecuteTask(fragmentMeta, storageUnitMeta.getId(), sourceStorageId, + targetStorageId, + MigrationExecuteType.MIGRATION)); + + Set pathRegexSet = new HashSet<>(); + pathRegexSet.add(fragmentMeta.getMasterStorageUnitId()); + ShowTimeSeries showTimeSeries = new ShowTimeSeries(new GlobalSource(), pathRegexSet, null, + Integer.MAX_VALUE, 0); + RowStream rowStream = physicalEngine.execute(showTimeSeries); + SortedSet pathSet = new TreeSet<>(); + rowStream.getHeader().getFields().forEach(field -> { + String timeSeries = field.getName(); + if (fragmentMeta.getTsInterval().isContain(timeSeries)) { + pathSet.add(timeSeries); + } + }); + // 开始迁移数据 + Migration migration = new Migration(new GlobalSource(), fragmentMeta, new ArrayList<>(pathSet), storageUnitMeta); + physicalEngine.execute(migration); + // 设置分片现在所属的du + fragmentMeta.setMasterStorageUnit(storageUnitMeta); + + // 迁移完开始删除原数据 + List paths = new ArrayList<>(); + paths.add(fragmentMeta.getMasterStorageUnitId() + "*"); + List timeRanges = new ArrayList<>(); + timeRanges.add(new TimeRange(fragmentMeta.getTimeInterval().getStartTime(), true, + fragmentMeta.getTimeInterval().getEndTime(), false)); + Delete delete = new Delete(new FragmentSource(fragmentMeta), timeRanges, paths, null); + physicalEngine.execute(delete); + } catch (Exception e) { + logger.error("encounter error when migrate data from {} to {} ", sourceStorageId, + targetStorageId, e); + } finally { + migrationLogger.logMigrationExecuteTaskEnd(); + } } - } - private void operateTaskAndRequest(long sourceStorageId, long targetStorageId, - FragmentMeta fragmentMeta) { - // TODO 暂时先不管迁移过程中的请求问题 - } + private FragmentMeta reshardFragment(long sourceStorageId, long targetStorageId, + FragmentMeta fragmentMeta) { + try { + migrationLogger.logMigrationExecuteTaskStart( + new MigrationExecuteTask(fragmentMeta, fragmentMeta.getMasterStorageUnitId(), + sourceStorageId, targetStorageId, + MigrationExecuteType.RESHARD_TIME)); + // [startTime, +∞) & (startPath, endPath) + TimeSeriesInterval tsInterval = fragmentMeta.getTsInterval(); + TimeInterval timeInterval = fragmentMeta.getTimeInterval(); + List storageEngineList = new ArrayList<>(); + storageEngineList.add(targetStorageId); + + // 排除乱序写入问题 + if (timeInterval.getEndTime() == Long.MAX_VALUE) { + operateTaskAndRequest(sourceStorageId, targetStorageId, fragmentMeta); + Pair fragmentMetaStorageUnitMetaPair = policy + .generateFragmentAndStorageUnitByTimeSeriesIntervalAndTimeInterval( + tsInterval.getStartTimeSeries(), tsInterval.getEndTimeSeries(), + DefaultMetaManager.getInstance().getMaxActiveEndTime(), Long.MAX_VALUE, + storageEngineList); + logger.info("start to splitFragmentAndStorageUnit"); + return DefaultMetaManager.getInstance() + .splitFragmentAndStorageUnit(fragmentMetaStorageUnitMetaPair.getV(), + fragmentMetaStorageUnitMetaPair.getK(), fragmentMeta); + } + return null; + } finally { + migrationLogger.logMigrationExecuteTaskEnd(); + } + } + + private void operateTaskAndRequest(long sourceStorageId, long targetStorageId, + FragmentMeta fragmentMeta) { + // TODO 暂时先不管迁移过程中的请求问题 + } } diff --git a/core/src/main/java/cn/edu/tsinghua/iginx/monitor/HotSpotMonitor.java b/core/src/main/java/cn/edu/tsinghua/iginx/monitor/HotSpotMonitor.java new file mode 100644 index 000000000..04a0383a6 --- /dev/null +++ b/core/src/main/java/cn/edu/tsinghua/iginx/monitor/HotSpotMonitor.java @@ -0,0 +1,51 @@ +package cn.edu.tsinghua.iginx.monitor; + +import cn.edu.tsinghua.iginx.conf.ConfigDescriptor; +import cn.edu.tsinghua.iginx.engine.shared.operator.OperatorType; +import cn.edu.tsinghua.iginx.metadata.entity.FragmentMeta; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +public class HotSpotMonitor implements IMonitor { + + private static final Logger logger = LoggerFactory.getLogger(HotSpotMonitor.class); + + private final boolean isEnableMonitor = ConfigDescriptor.getInstance().getConfig().isEnableMonitor(); + private final Map writeHotspotMap = new ConcurrentHashMap<>(); // 数据分区->写入总请求时间 + private final Map readHotspotMap = new ConcurrentHashMap<>(); // 数据分区->查询总请求时间 + private static final HotSpotMonitor instance = new HotSpotMonitor(); + + public static HotSpotMonitor getInstance() { + return instance; + } + + public Map getWriteHotspotMap() { + return writeHotspotMap; + } + + public Map getReadHotspotMap() { + return readHotspotMap; + } + + public void recordAfter(long taskId, FragmentMeta fragmentMeta, OperatorType operatorType) { + if (isEnableMonitor) { + long duration = (System.nanoTime() - taskId) / 1000000; + if (operatorType == OperatorType.Project) { + long prevDuration = readHotspotMap.getOrDefault(fragmentMeta, 0L); + readHotspotMap.put(fragmentMeta, prevDuration + duration); + } else if (operatorType == OperatorType.Insert) { + long prevDuration = writeHotspotMap.getOrDefault(fragmentMeta, 0L); + writeHotspotMap.put(fragmentMeta, prevDuration + duration); + } + } + } + + @Override + public void clear() { + writeHotspotMap.clear(); + readHotspotMap.clear(); + } +} \ No newline at end of file diff --git a/core/src/main/java/cn/edu/tsinghua/iginx/monitor/IMonitor.java b/core/src/main/java/cn/edu/tsinghua/iginx/monitor/IMonitor.java new file mode 100644 index 000000000..2b0ca31c2 --- /dev/null +++ b/core/src/main/java/cn/edu/tsinghua/iginx/monitor/IMonitor.java @@ -0,0 +1,5 @@ +package cn.edu.tsinghua.iginx.monitor; + +public interface IMonitor { + void clear(); +} diff --git a/core/src/main/java/cn/edu/tsinghua/iginx/monitor/MonitorManager.java b/core/src/main/java/cn/edu/tsinghua/iginx/monitor/MonitorManager.java new file mode 100644 index 000000000..d1a65459f --- /dev/null +++ b/core/src/main/java/cn/edu/tsinghua/iginx/monitor/MonitorManager.java @@ -0,0 +1,59 @@ +package cn.edu.tsinghua.iginx.monitor; + +import cn.edu.tsinghua.iginx.compaction.CompactionManager; +import cn.edu.tsinghua.iginx.conf.ConfigDescriptor; +import cn.edu.tsinghua.iginx.metadata.DefaultMetaManager; +import cn.edu.tsinghua.iginx.metadata.IMetaManager; +import cn.edu.tsinghua.iginx.metadata.entity.FragmentMeta; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; + +public class MonitorManager implements Runnable { + + private static final Logger logger = LoggerFactory.getLogger(MonitorManager.class); + + private static final int interval = ConfigDescriptor.getInstance().getConfig() + .getLoadBalanceCheckInterval(); + + private final IMetaManager metaManager = DefaultMetaManager.getInstance(); + private final CompactionManager compactionManager = CompactionManager.getInstance(); + private static MonitorManager INSTANCE; + + public static MonitorManager getInstance() { + if (INSTANCE == null) { + synchronized (MonitorManager.class) { + if (INSTANCE == null) { + INSTANCE = new MonitorManager(); + } + } + } + return INSTANCE; + } + + @Override + public void run() { + while (true) { + try { + //清空节点信息 + logger.info("start to clear monitors"); + compactionManager.clearFragment(); + metaManager.clearMonitors(); + logger.error("end clear monitors"); + Thread.sleep(interval * 1000L); + + //上传本地统计数据 + metaManager.updateFragmentRequests(RequestsMonitor.getInstance().getWriteRequestsMap(), + RequestsMonitor.getInstance() + .getReadRequestsMap()); + metaManager.submitMaxActiveEndTime(); + Map writeHotspotMap = HotSpotMonitor.getInstance().getWriteHotspotMap(); + Map readHotspotMap = HotSpotMonitor.getInstance().getReadHotspotMap(); + metaManager.updateFragmentHeat(writeHotspotMap, readHotspotMap); + } catch (Exception e) { + logger.error("monitor manager error ", e); + } + } + } +} \ No newline at end of file diff --git a/core/src/main/java/cn/edu/tsinghua/iginx/monitor/RequestsMonitor.java b/core/src/main/java/cn/edu/tsinghua/iginx/monitor/RequestsMonitor.java new file mode 100644 index 000000000..91df302c7 --- /dev/null +++ b/core/src/main/java/cn/edu/tsinghua/iginx/monitor/RequestsMonitor.java @@ -0,0 +1,52 @@ +package cn.edu.tsinghua.iginx.monitor; + +import cn.edu.tsinghua.iginx.conf.ConfigDescriptor; +import cn.edu.tsinghua.iginx.engine.shared.operator.Insert; +import cn.edu.tsinghua.iginx.engine.shared.operator.Operator; +import cn.edu.tsinghua.iginx.engine.shared.operator.OperatorType; +import cn.edu.tsinghua.iginx.metadata.entity.FragmentMeta; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +public class RequestsMonitor implements IMonitor { + + private final boolean isEnableMonitor = ConfigDescriptor.getInstance().getConfig() + .isEnableMonitor(); + private final Map writeRequestsMap = new ConcurrentHashMap<>(); // 数据分区->请求个数 + private final Map readRequestsMap = new ConcurrentHashMap<>(); // 数据分区->请求个数 + private static final RequestsMonitor instance = new RequestsMonitor(); + + public static RequestsMonitor getInstance() { + return instance; + } + + public Map getWriteRequestsMap() { + return writeRequestsMap; + } + + public Map getReadRequestsMap() { + return readRequestsMap; + } + + public void record(FragmentMeta fragmentMeta, Operator operator) { + if (isEnableMonitor) { + if (operator.getType() == OperatorType.Insert) { + Insert insert = (Insert) operator; + long count = writeRequestsMap.getOrDefault(fragmentMeta, 0L); + count += (long) insert.getData().getPathNum() * insert.getData().getTimeSize(); + writeRequestsMap.put(fragmentMeta, count); + } else if (operator.getType() == OperatorType.Project) { + long count = readRequestsMap.getOrDefault(fragmentMeta, 0L); + count++; + readRequestsMap.put(fragmentMeta, count); + } + } + } + + @Override + public void clear() { + writeRequestsMap.clear(); + readRequestsMap.clear(); + } +} \ No newline at end of file diff --git a/core/src/main/java/cn/edu/tsinghua/iginx/monitor/TimeseriesMonitor.java b/core/src/main/java/cn/edu/tsinghua/iginx/monitor/TimeseriesMonitor.java new file mode 100644 index 000000000..d4c6bcf1f --- /dev/null +++ b/core/src/main/java/cn/edu/tsinghua/iginx/monitor/TimeseriesMonitor.java @@ -0,0 +1,66 @@ +package cn.edu.tsinghua.iginx.monitor; + +import cn.edu.tsinghua.iginx.conf.ConfigDescriptor; +import cn.edu.tsinghua.iginx.engine.physical.exception.PhysicalException; +import cn.edu.tsinghua.iginx.engine.physical.task.TaskExecuteResult; +import cn.edu.tsinghua.iginx.engine.shared.data.read.Field; +import cn.edu.tsinghua.iginx.engine.shared.operator.OperatorType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +public class TimeseriesMonitor implements IMonitor { + + private static final Logger logger = LoggerFactory.getLogger(TimeseriesMonitor.class); + private final boolean isEnableMonitor = ConfigDescriptor.getInstance().getConfig() + .isEnableMonitor(); + private boolean isStartTimeseriesMonitor = false; + private final Map timeseriesLoadMap = new ConcurrentHashMap<>(); // 时间序列->总负载 + private static final TimeseriesMonitor instance = new TimeseriesMonitor(); + + public static TimeseriesMonitor getInstance() { + return instance; + } + + public void start() { + this.isStartTimeseriesMonitor = true; + } + + public void stop() { + this.isStartTimeseriesMonitor = false; + } + + public Map getTimeseriesLoadMap() { + return timeseriesLoadMap; + } + + public void recordAfter(long taskId, TaskExecuteResult result, OperatorType operatorType) { + try { + if (isEnableMonitor && isStartTimeseriesMonitor && operatorType == OperatorType.Project) { + // 构建本次访问的timeseries列表 + List timeseriesList = new ArrayList<>(); + for (Field field : result.getRowStream().getHeader().getFields()) { + timeseriesList.add(field.getName()); + } + + long duration = (System.nanoTime() - taskId) / 1000000; + long averageLoad = duration / timeseriesList.size(); //这里认为范围负载被所有时间序列均分 + for (String timeseries : timeseriesList) { + long load = timeseriesLoadMap.getOrDefault(timeseries, 0L); + timeseriesLoadMap.put(timeseries, averageLoad + load); + } + } + } catch (PhysicalException e) { + logger.error("record timeseries error:", e); + } + } + + @Override + public void clear() { + timeseriesLoadMap.clear(); + } +} \ No newline at end of file