diff --git a/server/src/assembly/resources/conf/iotdb-engine.properties b/server/src/assembly/resources/conf/iotdb-engine.properties index d3e4cee7037a9..1e263ac231d1f 100644 --- a/server/src/assembly/resources/conf/iotdb-engine.properties +++ b/server/src/assembly/resources/conf/iotdb-engine.properties @@ -21,6 +21,8 @@ ### Web Page Configuration #################### +enable_metric_service=false + metrics_port=8181 query_cache_size_in_metric=200 diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java index 1fc73b1175b81..c8ad7936b78b2 100644 --- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java +++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBConfig.java @@ -47,6 +47,9 @@ public class IoTDBConfig { * Port which the metrics service listens to. */ private int metricsPort = 8181; + + private boolean enableMetricService = false; + private String rpcAddress = "0.0.0.0"; /** @@ -483,8 +486,6 @@ public class IoTDBConfig { //wait for 60 second by default. private int thriftServerAwaitTimeForStopService = 60; - private boolean enableMetricsWebService = true; - private int queryCacheSizeInMetric = 200; public IoTDBConfig() { @@ -495,7 +496,7 @@ public int getMemtableNumInEachStorageGroup() { return memtableNumInEachStorageGroup; } - public void setMemtableNumInEachStorageGroup(int memtableNumInEachStorageGroup) { + void setMemtableNumInEachStorageGroup(int memtableNumInEachStorageGroup) { this.memtableNumInEachStorageGroup = memtableNumInEachStorageGroup; } @@ -611,18 +612,26 @@ public String[] getDataDirs() { return dataDirs; } - void setDataDirs(String[] dataDirs) { - this.dataDirs = dataDirs; - } - public int getMetricsPort() { return metricsPort; } - public void setMetricsPort(int metricsPort) { + void setMetricsPort(int metricsPort) { this.metricsPort = metricsPort; } + public boolean isEnableMetricService() { + return enableMetricService; + } + + public void setEnableMetricService(boolean enableMetricService) { + this.enableMetricService = enableMetricService; + } + + void setDataDirs(String[] dataDirs) { + this.dataDirs = dataDirs; + } + public String getRpcAddress() { return rpcAddress; } @@ -643,7 +652,7 @@ public String getTimestampPrecision() { return timestampPrecision; } - public void setTimestampPrecision(String timestampPrecision) { + void setTimestampPrecision(String timestampPrecision) { this.timestampPrecision = timestampPrecision; } @@ -691,7 +700,7 @@ public String getSyncDir() { return syncDir; } - public void setSyncDir(String syncDir) { + void setSyncDir(String syncDir) { this.syncDir = syncDir; } @@ -699,7 +708,7 @@ public String getQueryDir() { return queryDir; } - public void setQueryDir(String queryDir) { + void setQueryDir(String queryDir) { this.queryDir = queryDir; } @@ -779,7 +788,7 @@ public int getRpcMaxConcurrentClientNum() { return rpcMaxConcurrentClientNum; } - public void setRpcMaxConcurrentClientNum(int rpcMaxConcurrentClientNum) { + void setRpcMaxConcurrentClientNum(int rpcMaxConcurrentClientNum) { this.rpcMaxConcurrentClientNum = rpcMaxConcurrentClientNum; } @@ -823,7 +832,7 @@ void setSyncServerPort(int syncServerPort) { this.syncServerPort = syncServerPort; } - public String getLanguageVersion() { + String getLanguageVersion() { return languageVersion; } @@ -835,7 +844,7 @@ public String getBaseDir() { return baseDir; } - public void setBaseDir(String baseDir) { + void setBaseDir(String baseDir) { this.baseDir = baseDir; } @@ -891,7 +900,7 @@ public long getMergeMemoryBudget() { return mergeMemoryBudget; } - public void setMergeMemoryBudget(long mergeMemoryBudget) { + void setMergeMemoryBudget(long mergeMemoryBudget) { this.mergeMemoryBudget = mergeMemoryBudget; } @@ -899,7 +908,7 @@ public int getMergeThreadNum() { return mergeThreadNum; } - public void setMergeThreadNum(int mergeThreadNum) { + void setMergeThreadNum(int mergeThreadNum) { this.mergeThreadNum = mergeThreadNum; } @@ -907,7 +916,7 @@ public boolean isContinueMergeAfterReboot() { return continueMergeAfterReboot; } - public void setContinueMergeAfterReboot(boolean continueMergeAfterReboot) { + void setContinueMergeAfterReboot(boolean continueMergeAfterReboot) { this.continueMergeAfterReboot = continueMergeAfterReboot; } @@ -915,7 +924,7 @@ public long getMergeIntervalSec() { return mergeIntervalSec; } - public void setMergeIntervalSec(long mergeIntervalSec) { + void setMergeIntervalSec(long mergeIntervalSec) { this.mergeIntervalSec = mergeIntervalSec; } @@ -935,11 +944,11 @@ public void setAllocateMemoryForWrite(long allocateMemoryForWrite) { this.allocateMemoryForWrite = allocateMemoryForWrite; } - public long getAllocateMemoryForRead() { + long getAllocateMemoryForRead() { return allocateMemoryForRead; } - public void setAllocateMemoryForRead(long allocateMemoryForRead) { + void setAllocateMemoryForRead(long allocateMemoryForRead) { this.allocateMemoryForRead = allocateMemoryForRead; } @@ -947,7 +956,7 @@ public boolean isEnableExternalSort() { return enableExternalSort; } - public void setEnableExternalSort(boolean enableExternalSort) { + void setEnableExternalSort(boolean enableExternalSort) { this.enableExternalSort = enableExternalSort; } @@ -955,7 +964,7 @@ public int getExternalSortThreshold() { return externalSortThreshold; } - public void setExternalSortThreshold(int externalSortThreshold) { + void setExternalSortThreshold(int externalSortThreshold) { this.externalSortThreshold = externalSortThreshold; } @@ -971,7 +980,7 @@ public long getPerformanceStatDisplayInterval() { return performanceStatDisplayInterval; } - public void setPerformanceStatDisplayInterval(long performanceStatDisplayInterval) { + void setPerformanceStatDisplayInterval(long performanceStatDisplayInterval) { this.performanceStatDisplayInterval = performanceStatDisplayInterval; } @@ -979,7 +988,7 @@ public int getPerformanceStatMemoryInKB() { return performanceStatMemoryInKB; } - public void setPerformanceStatMemoryInKB(int performanceStatMemoryInKB) { + void setPerformanceStatMemoryInKB(int performanceStatMemoryInKB) { this.performanceStatMemoryInKB = performanceStatMemoryInKB; } @@ -987,7 +996,7 @@ public boolean isForceFullMerge() { return forceFullMerge; } - public void setForceFullMerge(boolean forceFullMerge) { + void setForceFullMerge(boolean forceFullMerge) { this.forceFullMerge = forceFullMerge; } @@ -1020,7 +1029,7 @@ public int getMergeChunkSubThreadNum() { return mergeChunkSubThreadNum; } - public void setMergeChunkSubThreadNum(int mergeChunkSubThreadNum) { + void setMergeChunkSubThreadNum(int mergeChunkSubThreadNum) { this.mergeChunkSubThreadNum = mergeChunkSubThreadNum; } @@ -1028,7 +1037,7 @@ public long getMergeFileSelectionTimeBudget() { return mergeFileSelectionTimeBudget; } - public void setMergeFileSelectionTimeBudget(long mergeFileSelectionTimeBudget) { + void setMergeFileSelectionTimeBudget(long mergeFileSelectionTimeBudget) { this.mergeFileSelectionTimeBudget = mergeFileSelectionTimeBudget; } @@ -1036,7 +1045,7 @@ public boolean isRpcThriftCompressionEnable() { return rpcThriftCompressionEnable; } - public void setRpcThriftCompressionEnable(boolean rpcThriftCompressionEnable) { + void setRpcThriftCompressionEnable(boolean rpcThriftCompressionEnable) { this.rpcThriftCompressionEnable = rpcThriftCompressionEnable; } @@ -1052,7 +1061,7 @@ public long getAllocateMemoryForFileMetaDataCache() { return allocateMemoryForFileMetaDataCache; } - public void setAllocateMemoryForFileMetaDataCache(long allocateMemoryForFileMetaDataCache) { + void setAllocateMemoryForFileMetaDataCache(long allocateMemoryForFileMetaDataCache) { this.allocateMemoryForFileMetaDataCache = allocateMemoryForFileMetaDataCache; } @@ -1096,7 +1105,7 @@ public void setWatermarkBitString(String watermarkBitString) { this.watermarkBitString = watermarkBitString; } - public String getWatermarkMethod() { + String getWatermarkMethod() { return this.watermarkMethod; } @@ -1116,7 +1125,7 @@ public int getWatermarkParamMaxRightBit() { return Integer.parseInt(getWatermarkParamValue("embed_lsb_num", "5")); } - public String getWatermarkParamValue(String key, String defaultValue) { + private String getWatermarkParamValue(String key, String defaultValue) { String res = getWatermarkParamValue(key); if (res != null) { return res; @@ -1124,7 +1133,7 @@ public String getWatermarkParamValue(String key, String defaultValue) { return defaultValue; } - public String getWatermarkParamValue(String key) { + private String getWatermarkParamValue(String key) { String pattern = key + "=(\\w*)"; Pattern r = Pattern.compile(pattern); Matcher m = r.matcher(watermarkMethod); @@ -1146,7 +1155,7 @@ public int getDefaultStorageGroupLevel() { return defaultStorageGroupLevel; } - public void setDefaultStorageGroupLevel(int defaultStorageGroupLevel) { + void setDefaultStorageGroupLevel(int defaultStorageGroupLevel) { this.defaultStorageGroupLevel = defaultStorageGroupLevel; } @@ -1158,7 +1167,7 @@ public void setDefaultBooleanEncoding(TSEncoding defaultBooleanEncoding) { this.defaultBooleanEncoding = defaultBooleanEncoding; } - public void setDefaultBooleanEncoding(String defaultBooleanEncoding) { + void setDefaultBooleanEncoding(String defaultBooleanEncoding) { this.defaultBooleanEncoding = TSEncoding.valueOf(defaultBooleanEncoding); } @@ -1170,7 +1179,7 @@ public void setDefaultInt32Encoding(TSEncoding defaultInt32Encoding) { this.defaultInt32Encoding = defaultInt32Encoding; } - public void setDefaultInt32Encoding(String defaultInt32Encoding) { + void setDefaultInt32Encoding(String defaultInt32Encoding) { this.defaultInt32Encoding = TSEncoding.valueOf(defaultInt32Encoding); } @@ -1182,7 +1191,7 @@ public void setDefaultInt64Encoding(TSEncoding defaultInt64Encoding) { this.defaultInt64Encoding = defaultInt64Encoding; } - public void setDefaultInt64Encoding(String defaultInt64Encoding) { + void setDefaultInt64Encoding(String defaultInt64Encoding) { this.defaultInt64Encoding = TSEncoding.valueOf(defaultInt64Encoding); } @@ -1194,7 +1203,7 @@ public void setDefaultFloatEncoding(TSEncoding defaultFloatEncoding) { this.defaultFloatEncoding = defaultFloatEncoding; } - public void setDefaultFloatEncoding(String defaultFloatEncoding) { + void setDefaultFloatEncoding(String defaultFloatEncoding) { this.defaultFloatEncoding = TSEncoding.valueOf(defaultFloatEncoding); } @@ -1206,7 +1215,7 @@ public void setDefaultDoubleEncoding(TSEncoding defaultDoubleEncoding) { this.defaultDoubleEncoding = defaultDoubleEncoding; } - public void setDefaultDoubleEncoding(String defaultDoubleEncoding) { + void setDefaultDoubleEncoding(String defaultDoubleEncoding) { this.defaultDoubleEncoding = TSEncoding.valueOf(defaultDoubleEncoding); } @@ -1218,7 +1227,7 @@ public void setDefaultTextEncoding(TSEncoding defaultTextEncoding) { this.defaultTextEncoding = defaultTextEncoding; } - public void setDefaultTextEncoding(String defaultTextEncoding) { + void setDefaultTextEncoding(String defaultTextEncoding) { this.defaultTextEncoding = TSEncoding.valueOf(defaultTextEncoding); } @@ -1230,27 +1239,27 @@ public void setSystemFileStorageFs(String systemFileStorageFs) { this.systemFileStorageFs = FSType.valueOf(systemFileStorageFs); } - public FSType getTsFileStorageFs() { + FSType getTsFileStorageFs() { return tsFileStorageFs; } - public void setTsFileStorageFs(String tsFileStorageFs) { + void setTsFileStorageFs(String tsFileStorageFs) { this.tsFileStorageFs = FSType.valueOf(tsFileStorageFs); } - public String getCoreSitePath() { + String getCoreSitePath() { return coreSitePath; } - public void setCoreSitePath(String coreSitePath) { + void setCoreSitePath(String coreSitePath) { this.coreSitePath = coreSitePath; } - public String getHdfsSitePath() { + String getHdfsSitePath() { return hdfsSitePath; } - public void setHdfsSitePath(String hdfsSitePath) { + void setHdfsSitePath(String hdfsSitePath) { this.hdfsSitePath = hdfsSitePath; } @@ -1262,15 +1271,15 @@ String getRawHDFSIp() { return hdfsIp; } - public void setHdfsIp(String[] hdfsIp) { + void setHdfsIp(String[] hdfsIp) { this.hdfsIp = String.join(",", hdfsIp); } - public String getHdfsPort() { + String getHdfsPort() { return hdfsPort; } - public void setHdfsPort(String hdfsPort) { + void setHdfsPort(String hdfsPort) { this.hdfsPort = hdfsPort; } @@ -1278,15 +1287,15 @@ public int getUpgradeThreadNum() { return upgradeThreadNum; } - public void setUpgradeThreadNum(int upgradeThreadNum) { + void setUpgradeThreadNum(int upgradeThreadNum) { this.upgradeThreadNum = upgradeThreadNum; } - public String getDfsNameServices() { + String getDfsNameServices() { return dfsNameServices; } - public void setDfsNameServices(String dfsNameServices) { + void setDfsNameServices(String dfsNameServices) { this.dfsNameServices = dfsNameServices; } @@ -1298,47 +1307,47 @@ String getRawDfsHaNamenodes() { return dfsHaNamenodes; } - public void setDfsHaNamenodes(String[] dfsHaNamenodes) { + void setDfsHaNamenodes(String[] dfsHaNamenodes) { this.dfsHaNamenodes = String.join(",", dfsHaNamenodes); } - public boolean isDfsHaAutomaticFailoverEnabled() { + boolean isDfsHaAutomaticFailoverEnabled() { return dfsHaAutomaticFailoverEnabled; } - public void setDfsHaAutomaticFailoverEnabled(boolean dfsHaAutomaticFailoverEnabled) { + void setDfsHaAutomaticFailoverEnabled(boolean dfsHaAutomaticFailoverEnabled) { this.dfsHaAutomaticFailoverEnabled = dfsHaAutomaticFailoverEnabled; } - public String getDfsClientFailoverProxyProvider() { + String getDfsClientFailoverProxyProvider() { return dfsClientFailoverProxyProvider; } - public void setDfsClientFailoverProxyProvider(String dfsClientFailoverProxyProvider) { + void setDfsClientFailoverProxyProvider(String dfsClientFailoverProxyProvider) { this.dfsClientFailoverProxyProvider = dfsClientFailoverProxyProvider; } - public boolean isUseKerberos() { + boolean isUseKerberos() { return useKerberos; } - public void setUseKerberos(boolean useKerberos) { + void setUseKerberos(boolean useKerberos) { this.useKerberos = useKerberos; } - public String getKerberosKeytabFilePath() { + String getKerberosKeytabFilePath() { return kerberosKeytabFilePath; } - public void setKerberosKeytabFilePath(String kerberosKeytabFilePath) { + void setKerberosKeytabFilePath(String kerberosKeytabFilePath) { this.kerberosKeytabFilePath = kerberosKeytabFilePath; } - public String getKerberosPrincipal() { + String getKerberosPrincipal() { return kerberosPrincipal; } - public void setKerberosPrincipal(String kerberosPrincipal) { + void setKerberosPrincipal(String kerberosPrincipal) { this.kerberosPrincipal = kerberosPrincipal; } @@ -1358,14 +1367,6 @@ public void setThriftServerAwaitTimeForStopService(int thriftServerAwaitTimeForS this.thriftServerAwaitTimeForStopService = thriftServerAwaitTimeForStopService; } - public boolean isEnableMetricsWebService() { - return enableMetricsWebService; - } - - public void setEnableMetricsWebService(boolean enableMetricsWebService) { - this.enableMetricsWebService = enableMetricsWebService; - } - public int getQueryCacheSizeInMetric() { return queryCacheSizeInMetric; } diff --git a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java index aa318f6f193d4..d0131dc4899fc 100644 --- a/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java +++ b/server/src/main/java/org/apache/iotdb/db/conf/IoTDBDescriptor.java @@ -114,6 +114,9 @@ private void loadProps() { + " use default value"); } + conf.setEnableMetricService(Boolean.parseBoolean(properties + .getProperty("enable_metric_service", Boolean.toString(conf.isEnableMetricService())))); + conf.setMetricsPort(Integer.parseInt(properties.getProperty("metrics_port", Integer.toString(conf.getMetricsPort())))); diff --git a/server/src/main/java/org/apache/iotdb/db/conf/ServerConfigConsistent.java b/server/src/main/java/org/apache/iotdb/db/conf/ServerConfigConsistent.java new file mode 100644 index 0000000000000..18ad0a41a231b --- /dev/null +++ b/server/src/main/java/org/apache/iotdb/db/conf/ServerConfigConsistent.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.conf; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * Configurations using this annotation should be the same across all nodes in a cluster. + */ +@Target({ElementType.FIELD}) +@Retention(RetentionPolicy.SOURCE) +public @interface ServerConfigConsistent { + //TODO#IOTDB-436: the restarted server should check the + // configuration consistency as before. + +} + diff --git a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java index cf99279b9ff8c..18485dc21c639 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/StorageEngine.java @@ -23,8 +23,10 @@ import java.util.ArrayList; import java.util.Collections; import java.util.ConcurrentModificationException; +import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; @@ -38,6 +40,7 @@ import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory; import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.conf.ServerConfigConsistent; import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory; import org.apache.iotdb.db.engine.flush.TsFileFlushPolicy; import org.apache.iotdb.db.engine.flush.TsFileFlushPolicy.DirectFlushPolicy; @@ -91,15 +94,46 @@ public class StorageEngine implements IService { private static final ExecutorService recoveryThreadPool = IoTDBThreadPoolFactory .newFixedThreadPool(Runtime.getRuntime().availableProcessors(), "Recovery-Thread-Pool"); - private static final StorageEngine INSTANCE = new StorageEngine(); + static class InstanceHolder { + + private InstanceHolder() { + // forbidding instantiation + } + + private static final StorageEngine INSTANCE = new StorageEngine(); + } public static StorageEngine getInstance() { - return INSTANCE; + return InstanceHolder.INSTANCE; } private ScheduledExecutorService ttlCheckThread; private TsFileFlushPolicy fileFlushPolicy = new DirectFlushPolicy(); + /** + * Time range for dividing storage group, the time unit is the same with IoTDB's TimestampPrecision + */ + @ServerConfigConsistent + static long timePartitionInterval; + static { + // build time Interval to divide time partition + String timePrecision = IoTDBDescriptor.getInstance().getConfig().getTimestampPrecision(); + switch (timePrecision) { + case "ns": + timePartitionInterval = IoTDBDescriptor.getInstance(). + getConfig().getPartitionInterval() * 1000_000_000L; + break; + case "us": + timePartitionInterval = IoTDBDescriptor.getInstance(). + getConfig().getPartitionInterval() * 1000_000L; + break; + default: + timePartitionInterval = IoTDBDescriptor.getInstance(). + getConfig().getPartitionInterval() * 1000; + break; + } + } + private StorageEngine() { logger = LoggerFactory.getLogger(StorageEngine.class); systemDir = FilePathUtils.regularizePath(config.getSystemDir()) + "storage_groups"; @@ -109,7 +143,6 @@ private StorageEngine() { } catch (IOException e) { throw new StorageEngineFailureException(e); } - // recover upgrade process UpgradeUtils.recoverUpgrade(); /* @@ -160,14 +193,16 @@ private void checkTTL() { @Override public void stop() { syncCloseAllProcessor(); - ttlCheckThread.shutdownNow(); + if (ttlCheckThread != null) { + ttlCheckThread.shutdownNow(); + try { + ttlCheckThread.awaitTermination(30, TimeUnit.SECONDS); + } catch (InterruptedException e) { + logger.warn("TTL check thread still doesn't exit after 30s"); + } + } recoveryThreadPool.shutdownNow(); this.reset(); - try { - ttlCheckThread.awaitTermination(30, TimeUnit.SECONDS); - } catch (InterruptedException e) { - logger.warn("TTL check thread still doesn't exit after 30s"); - } } @Override @@ -176,7 +211,7 @@ public ServiceType getID() { } public StorageGroupProcessor getProcessor(String path) throws StorageEngineException { - String storageGroupName = ""; + String storageGroupName; try { storageGroupName = MManager.getInstance().getStorageGroupName(path); StorageGroupProcessor processor; @@ -490,4 +525,37 @@ private String getSgByEngineFile(File file) { return file.getParentFile().getParentFile().getName(); } + /** + * + * @return TsFiles (seq or unseq) grouped by their storage group. + */ + public Map> getAllClosedStorageGroupTsFile() { + Map> ret = new HashMap<>(); + for (Entry entry : processorMap + .entrySet()) { + ret.computeIfAbsent(entry.getKey(), sg -> new ArrayList<>()).addAll(entry.getValue().getSequenceFileTreeSet()); + ret.get(entry.getKey()).addAll(entry.getValue().getUnSequenceFileList()); + ret.get(entry.getKey()).removeIf(file -> !file.isClosed()); + } + return ret; + } + + public void setFileFlushPolicy(TsFileFlushPolicy fileFlushPolicy) { + this.fileFlushPolicy = fileFlushPolicy; + } + + public boolean isFileAlreadyExist(TsFileResource tsFileResource, String storageGroup) { + // TODO-Cluster#350: integrate with time partitioning + StorageGroupProcessor processor = processorMap.get(storageGroup); + return processor != null && processor.isFileAlreadyExist(tsFileResource); + } + + public static long getTimePartitionInterval() { + return timePartitionInterval; + } + + public static long fromTimeToTimePartition(long time) { + + return time / timePartitionInterval; + } } diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java index 210afa6662a53..e220f7cf99ec9 100755 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/StorageGroupProcessor.java @@ -44,6 +44,7 @@ import org.apache.iotdb.db.conf.IoTDBConstant; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.conf.directories.DirectoryManager; +import org.apache.iotdb.db.engine.StorageEngine; import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory; import org.apache.iotdb.db.engine.flush.TsFileFlushPolicy; import org.apache.iotdb.db.engine.merge.manage.MergeManager; @@ -77,7 +78,6 @@ import org.apache.iotdb.db.query.context.QueryContext; import org.apache.iotdb.db.query.control.QueryFileManager; import org.apache.iotdb.db.utils.CopyOnReadLinkedList; -import org.apache.iotdb.db.utils.TestOnly; import org.apache.iotdb.db.utils.UpgradeUtils; import org.apache.iotdb.db.writelog.recover.TsFileRecoverPerformer; import org.apache.iotdb.rpc.TSStatusCode; @@ -145,10 +145,7 @@ public class StorageGroupProcessor { * time partition id in the storage group -> tsFileProcessor for this time partition */ private final TreeMap workUnsequenceTsFileProcessors = new TreeMap<>(); - /** - * Time range for dividing storage group, unit is second - */ - private long partitionIntervalForStorageGroup; + /** * the schema of time series that belong this storage group */ @@ -240,23 +237,6 @@ public StorageGroupProcessor(String systemInfoDir, String storageGroupName, storageGroupSysDir.getPath()); } - // build time Interval to divide time partition - String timePrecision = IoTDBDescriptor.getInstance().getConfig().getTimestampPrecision(); - switch (timePrecision) { - case "ns": - partitionIntervalForStorageGroup = IoTDBDescriptor.getInstance(). - getConfig().getPartitionInterval() * 1000_000_000L; - break; - case "us": - partitionIntervalForStorageGroup = IoTDBDescriptor.getInstance(). - getConfig().getPartitionInterval() * 1000_000L; - break; - default: - partitionIntervalForStorageGroup = IoTDBDescriptor.getInstance(). - getConfig().getPartitionInterval() * 1000; - break; - } - recover(); } @@ -333,7 +313,7 @@ private long getTimePartitionFromTsFileResource(TsFileResource resource) { // just find any time of device Iterator iterator = startTimeMap.values().iterator(); if (iterator.hasNext()) { - return fromTimeToTimePartition(iterator.next()); + return StorageEngine.fromTimeToTimePartition(iterator.next()); } return -1; @@ -506,7 +486,7 @@ public void insert(InsertPlan insertPlan) throws QueryProcessException { writeLock(); try { // init map - long timePartitionId = fromTimeToTimePartition(insertPlan.getTime()); + long timePartitionId = StorageEngine.fromTimeToTimePartition(insertPlan.getTime()); latestTimeForEachDevice.computeIfAbsent(timePartitionId, l -> new HashMap<>()) .putIfAbsent(insertPlan.getDeviceId(), Long.MIN_VALUE); partitionLatestFlushedTimeForEachDevice.computeIfAbsent(timePartitionId, id -> new HashMap<>()) @@ -547,7 +527,7 @@ public Integer[] insertBatch(BatchInsertPlan batchInsertPlan) throws QueryProces // before is first start point int before = loc; // before time partition - long beforeTimePartition = fromTimeToTimePartition(batchInsertPlan.getTimes()[before]); + long beforeTimePartition = StorageEngine.fromTimeToTimePartition(batchInsertPlan.getTimes()[before]); // init map long lastFlushTime = partitionLatestFlushedTimeForEachDevice. computeIfAbsent(beforeTimePartition, id -> new HashMap<>()). @@ -556,7 +536,7 @@ public Integer[] insertBatch(BatchInsertPlan batchInsertPlan) throws QueryProces boolean isSequence = false; while (loc < batchInsertPlan.getRowCount()) { long time = batchInsertPlan.getTimes()[loc]; - long curTimePartition = fromTimeToTimePartition(time); + long curTimePartition = StorageEngine.fromTimeToTimePartition(time); results[loc] = TSStatusCode.SUCCESS_STATUS.getStatusCode(); // start next partition if (curTimePartition != beforeTimePartition) { @@ -675,7 +655,7 @@ private void insertToTsFileProcessor(InsertPlan insertPlan, boolean sequence) throws QueryProcessException { TsFileProcessor tsFileProcessor; boolean result; - long timePartitionId = fromTimeToTimePartition(insertPlan.getTime()); + long timePartitionId = StorageEngine.fromTimeToTimePartition(insertPlan.getTime()); tsFileProcessor = getOrCreateTsFileProcessor(timePartitionId, sequence); @@ -798,10 +778,6 @@ private TsFileProcessor getOrCreateTsFileProcessorIntern(long timeRangeId, } - private long fromTimeToTimePartition(long time) { - - return time / partitionIntervalForStorageGroup; - } private TsFileProcessor createTsFileProcessor(boolean sequence, long timePartitionId) throws IOException, DiskSpaceInsufficientException { @@ -847,6 +823,7 @@ private String getNewTsFileName(long timePartitionId) { } private String getNewTsFileName(long time, long version, int mergeCnt) { + allDirectFileVersions.add(version); return time + IoTDBConstant.TSFILE_NAME_SEPARATOR + version + IoTDBConstant.TSFILE_NAME_SEPARATOR + mergeCnt + TSFILE_SUFFIX; } @@ -1229,7 +1206,7 @@ public void delete(String deviceId, String measurementId, long timestamp) throws } // time partition to divide storage group - long timePartitionId = fromTimeToTimePartition(timestamp); + long timePartitionId = StorageEngine.fromTimeToTimePartition(timestamp); // write log if (IoTDBDescriptor.getInstance().getConfig().isEnableWal()) { DeletePlan deletionPlan = new DeletePlan(timestamp, new Path(deviceId, measurementId)); @@ -1683,6 +1660,40 @@ public void loadNewTsFile(TsFileResource newTsFileResource) } } + /** + * If the historical versions of a file is a sub-set of the given file's, remove it to reduce + * unnecessary merge. Only used when the file sender and the receiver share the same file + * close policy. + * @param resource + */ + public void removeFullyOverlapFiles(TsFileResource resource) { + writeLock(); + closeQueryLock.writeLock().lock(); + try { + Iterator iterator = sequenceFileTreeSet.iterator(); + removeFullyOverlapFiles(resource, iterator); + + iterator = unSequenceFileList.iterator(); + removeFullyOverlapFiles(resource, iterator); + } finally { + closeQueryLock.writeLock().unlock(); + writeUnlock(); + } + } + + private void removeFullyOverlapFiles(TsFileResource resource, Iterator iterator) { + while (iterator.hasNext()) { + TsFileResource seqFile = iterator.next(); + if (resource.getHistoricalVersions().containsAll(seqFile.getHistoricalVersions()) + && !resource.getHistoricalVersions().equals(seqFile.getHistoricalVersions()) + && seqFile.getWriteQueryLock().writeLock().tryLock()) { + iterator.remove(); + seqFile.remove(); + seqFile.getWriteQueryLock().writeLock().unlock(); + } + } + } + /** * Get an appropriate filename to ensure the order between files. The tsfile is named after * ({systemTime}-{versionNum}-{mergeNum}.tsfile). @@ -1747,7 +1758,7 @@ private void updateLatestTimeMap(TsFileResource newTsFileResource) { for (Entry entry : newTsFileResource.getEndTimeMap().entrySet()) { String device = entry.getKey(); long endTime = newTsFileResource.getEndTimeMap().get(device); - long timePartitionId = fromTimeToTimePartition(endTime); + long timePartitionId = StorageEngine.fromTimeToTimePartition(endTime); if (!latestTimeForEachDevice.computeIfAbsent(timePartitionId, id -> new HashMap<>()) .containsKey(device) || latestTimeForEachDevice.get(timePartitionId).get(device) < endTime) { @@ -1760,7 +1771,7 @@ private void updateLatestTimeMap(TsFileResource newTsFileResource) { if (!latestFlushTimeForPartition.containsKey(device) || latestFlushTimeForPartition.get(device) < endTime) { partitionLatestFlushedTimeForEachDevice - .computeIfAbsent(timePartitionId, id -> new HashMap()) + .computeIfAbsent(timePartitionId, id -> new HashMap<>()) .put(device, endTime); } if (!globalLatestFlushedTimeForEachDevice.containsKey(device) @@ -1781,7 +1792,7 @@ private void loadTsFileByType(LoadTsFileType type, File syncedTsFile, TsFileResource tsFileResource) throws TsFileProcessorException, DiskSpaceInsufficientException { File targetFile; - long timeRangeId = fromTimeToTimePartition( + long timeRangeId = StorageEngine.fromTimeToTimePartition( tsFileResource.getStartTimeMap().entrySet().iterator().next().getValue()); switch (type) { case LOAD_UNSEQUENCE: @@ -1961,12 +1972,10 @@ public void setDataTTL(long dataTTL) { checkFilesTTL(); } - @TestOnly public List getSequenceFileTreeSet() { return new ArrayList<>(sequenceFileTreeSet); } - @TestOnly public List getUnSequenceFileList() { return unSequenceFileList; } @@ -1981,6 +1990,14 @@ public interface CloseTsFileCallBack { void call(TsFileProcessor caller) throws TsFileProcessorException, IOException; } + public String getStorageGroupName() { + return storageGroupName; + } + + public boolean isFileAlreadyExist(TsFileResource tsFileResource) { + return allDirectFileVersions.containsAll(tsFileResource.getHistoricalVersions()); + } + @FunctionalInterface public interface UpdateEndTimeCallBack { diff --git a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java index e9356dbbb13f4..58ecd56d938e2 100644 --- a/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java +++ b/server/src/main/java/org/apache/iotdb/db/engine/storagegroup/TsFileResource.java @@ -60,12 +60,12 @@ public class TsFileResource { /** * device -> start time */ - private Map startTimeMap; + protected Map startTimeMap; /** * device -> end time. It is null if it's an unsealed sequence tsfile */ - private Map endTimeMap; + protected Map endTimeMap; public TsFileProcessor getProcessor() { return processor; @@ -101,6 +101,25 @@ public TsFileProcessor getProcessor() { private FSFactory fsFactory = FSFactoryProducer.getFSFactory(); + public TsFileResource() { + } + + public TsFileResource(TsFileResource other) { + this.file = other.file; + this.startTimeMap = other.startTimeMap; + this.endTimeMap = other.endTimeMap; + this.processor = other.processor; + this.modFile = other.modFile; + this.closed = other.closed; + this.deleted = other.deleted; + this.isMerging = other.isMerging; + this.chunkMetaDataList = other.chunkMetaDataList; + this.readOnlyMemChunk = other.readOnlyMemChunk; + this.writeQueryLock = other.writeQueryLock; + this.fsFactory = other.fsFactory; + this.historicalVersions = other.historicalVersions; + } + /** * for sealed TsFile, call setClosed to close TsFileResource */ @@ -162,7 +181,7 @@ public void serialize() throws IOException { fsFactory.moveFile(src, dest); } - public void deSerialize() throws IOException { + public void deserialize() throws IOException { try (InputStream inputStream = fsFactory.getBufferedInputStream( file + RESOURCE_SUFFIX)) { int size = ReadWriteIOUtils.readInt(inputStream); @@ -363,11 +382,19 @@ public boolean stillLives(long timeLowerBound) { return false; } + protected void setStartTimeMap(Map startTimeMap) { + this.startTimeMap = startTimeMap; + } + + protected void setEndTimeMap(Map endTimeMap) { + this.endTimeMap = endTimeMap; + } + /** * set a file flag indicating that the file is being closed, so during recovery we could know we * should close the file. */ - public void setCloseFlag() { + void setCloseFlag() { try { new File(file.getAbsoluteFile() + CLOSING_SUFFIX).createNewFile(); } catch (IOException e) { diff --git a/server/src/main/java/org/apache/iotdb/db/exception/StorageEngineException.java b/server/src/main/java/org/apache/iotdb/db/exception/StorageEngineException.java index 7944bd489e3b6..bd03a29c9d784 100644 --- a/server/src/main/java/org/apache/iotdb/db/exception/StorageEngineException.java +++ b/server/src/main/java/org/apache/iotdb/db/exception/StorageEngineException.java @@ -24,6 +24,10 @@ public class StorageEngineException extends IoTDBException { private static final long serialVersionUID = 9001649171768311032L; + public StorageEngineException(Throwable cause) { + super(cause, TSStatusCode.STORAGE_ENGINE_ERROR.getStatusCode()); + } + public StorageEngineException(String message) { super(message, TSStatusCode.STORAGE_ENGINE_ERROR.getStatusCode()); } @@ -33,6 +37,6 @@ public StorageEngineException(String message, int errorCode) { } public StorageEngineException(IoTDBException e) { - super(e.getMessage(), e.getErrorCode()); + super(e, e.getErrorCode()); } } diff --git a/server/src/main/java/org/apache/iotdb/db/exception/metadata/IllegalPathException.java b/server/src/main/java/org/apache/iotdb/db/exception/metadata/IllegalPathException.java index 8f9ee27490ef2..da40603161e63 100644 --- a/server/src/main/java/org/apache/iotdb/db/exception/metadata/IllegalPathException.java +++ b/server/src/main/java/org/apache/iotdb/db/exception/metadata/IllegalPathException.java @@ -15,7 +15,6 @@ * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. - * */ package org.apache.iotdb.db.exception.metadata; @@ -23,11 +22,10 @@ import org.apache.iotdb.rpc.TSStatusCode; public class IllegalPathException extends MetadataException { - private static final long serialVersionUID = 2693272249167539978L; public IllegalPathException(String path) { super(String.format("%s is not a legal path", path)); errorCode = TSStatusCode.PATH_ILLEGAL.getStatusCode(); } -} \ No newline at end of file +} diff --git a/server/src/main/java/org/apache/iotdb/db/exception/metadata/MetadataException.java b/server/src/main/java/org/apache/iotdb/db/exception/metadata/MetadataException.java index dfca5203629c6..5d8921c288149 100644 --- a/server/src/main/java/org/apache/iotdb/db/exception/metadata/MetadataException.java +++ b/server/src/main/java/org/apache/iotdb/db/exception/metadata/MetadataException.java @@ -29,6 +29,10 @@ public class MetadataException extends IoTDBException { private static final long serialVersionUID = 3415275599091623570L; + public MetadataException(Throwable cause) { + super(cause, TSStatusCode.METADATA_ERROR.getStatusCode()); + } + public MetadataException(Throwable cause, int errorCode) { super(cause, TSStatusCode.METADATA_ERROR.getStatusCode()); this.errorCode = errorCode; diff --git a/server/src/main/java/org/apache/iotdb/db/exception/metadata/PathNotExistException.java b/server/src/main/java/org/apache/iotdb/db/exception/metadata/PathNotExistException.java index fe8e476754049..7d86ade0cddf7 100644 --- a/server/src/main/java/org/apache/iotdb/db/exception/metadata/PathNotExistException.java +++ b/server/src/main/java/org/apache/iotdb/db/exception/metadata/PathNotExistException.java @@ -15,15 +15,12 @@ * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. - * */ package org.apache.iotdb.db.exception.metadata; public class PathNotExistException extends MetadataException { - private static final long serialVersionUID = 2693272249167539978L; - public PathNotExistException(String path) { super(String.format("Path [%s] does not exist", path)); } diff --git a/server/src/main/java/org/apache/iotdb/db/exception/metadata/StorageGroupAlreadySetException.java b/server/src/main/java/org/apache/iotdb/db/exception/metadata/StorageGroupAlreadySetException.java index c43884a2bbd95..b19630975b9d7 100644 --- a/server/src/main/java/org/apache/iotdb/db/exception/metadata/StorageGroupAlreadySetException.java +++ b/server/src/main/java/org/apache/iotdb/db/exception/metadata/StorageGroupAlreadySetException.java @@ -15,7 +15,6 @@ * KIND, either express or implied. See the License for the * specific language governing permissions and limitations * under the License. - * */ package org.apache.iotdb.db.exception.metadata; @@ -27,4 +26,4 @@ public class StorageGroupAlreadySetException extends MetadataException { public StorageGroupAlreadySetException(String path) { super(String.format("%s has already been set to storage group", path)); } -} \ No newline at end of file +} diff --git a/server/src/main/java/org/apache/iotdb/db/exception/query/QueryProcessException.java b/server/src/main/java/org/apache/iotdb/db/exception/query/QueryProcessException.java index 9372d15bf9dc0..f8c007c71b6a0 100644 --- a/server/src/main/java/org/apache/iotdb/db/exception/query/QueryProcessException.java +++ b/server/src/main/java/org/apache/iotdb/db/exception/query/QueryProcessException.java @@ -35,6 +35,6 @@ public QueryProcessException(String message, int errorCode) { } public QueryProcessException(IoTDBException e) { - super(e.getMessage(), e.getErrorCode()); + super(e, e.getErrorCode()); } } diff --git a/server/src/main/java/org/apache/iotdb/db/exception/runtime/StorageEngineFailureException.java b/server/src/main/java/org/apache/iotdb/db/exception/runtime/StorageEngineFailureException.java index 934ece6d0535b..aab283c2d927b 100644 --- a/server/src/main/java/org/apache/iotdb/db/exception/runtime/StorageEngineFailureException.java +++ b/server/src/main/java/org/apache/iotdb/db/exception/runtime/StorageEngineFailureException.java @@ -24,10 +24,10 @@ public class StorageEngineFailureException extends RuntimeException { private static final long serialVersionUID = -1197701024139022020L; public StorageEngineFailureException(Throwable e) { - super("Create system directory failed! " + e.getMessage()); + super("Create system directory failed! ", e); } public StorageEngineFailureException(String message, Throwable e) { - super(message + e.getMessage()); + super(message, e); } } diff --git a/server/src/main/java/org/apache/iotdb/db/exception/storageGroup/StorageGroupProcessorException.java b/server/src/main/java/org/apache/iotdb/db/exception/storageGroup/StorageGroupProcessorException.java index 249be8550a60c..9bae99d3af1ad 100644 --- a/server/src/main/java/org/apache/iotdb/db/exception/storageGroup/StorageGroupProcessorException.java +++ b/server/src/main/java/org/apache/iotdb/db/exception/storageGroup/StorageGroupProcessorException.java @@ -26,7 +26,7 @@ public class StorageGroupProcessorException extends IoTDBException { private static final long serialVersionUID = 7373978140952977661L; public StorageGroupProcessorException(Exception exception) { - super(exception.getMessage(), TSStatusCode.STORAGE_GROUP_PROCESSOR_ERROR.getStatusCode()); + super(exception, TSStatusCode.STORAGE_GROUP_PROCESSOR_ERROR.getStatusCode()); } public StorageGroupProcessorException(String message) { diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java b/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java index 0ef29eb51bd21..00b21cc0c2e93 100644 --- a/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java +++ b/server/src/main/java/org/apache/iotdb/db/metadata/MManager.java @@ -24,9 +24,12 @@ import java.io.FileReader; import java.io.FileWriter; import java.io.IOException; +import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; +import java.util.Deque; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -39,9 +42,11 @@ import org.apache.iotdb.db.conf.adapter.IoTDBConfigDynamicAdapter; import org.apache.iotdb.db.engine.fileSystem.SystemFileFactory; import org.apache.iotdb.db.exception.ConfigAdjusterException; +import org.apache.iotdb.db.exception.metadata.IllegalPathException; import org.apache.iotdb.db.exception.metadata.MetadataException; import org.apache.iotdb.db.exception.metadata.PathNotExistException; import org.apache.iotdb.db.exception.metadata.StorageGroupNotSetException; +import org.apache.iotdb.db.metadata.mnode.LeafMNode; import org.apache.iotdb.db.metadata.mnode.MNode; import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode; import org.apache.iotdb.db.monitor.MonitorConstants; @@ -841,4 +846,68 @@ List getStorageGroupByPath(String path) throws MetadataException { lock.readLock().unlock(); } } + + public void collectSeries(MNode startingNode, Collection timeseriesSchemas) { + Deque nodeDeque = new ArrayDeque<>(); + nodeDeque.addLast(startingNode); + while (!nodeDeque.isEmpty()) { + MNode node = nodeDeque.removeFirst(); + if (node instanceof LeafMNode) { + MeasurementSchema nodeSchema = node.getSchema(); + timeseriesSchemas.add(new MeasurementSchema(node.getFullPath(), + nodeSchema.getType(), nodeSchema.getEncodingType(), nodeSchema.getCompressor())); + } else if (!node.getChildren().isEmpty()) { + nodeDeque.addAll(node.getChildren().values()); + } + } + } + + public void collectSeries(String startingPath, List timeseriesSchemas) { + MNode mNode; + try { + mNode = getNodeByPath(startingPath); + } catch (MetadataException e) { + return; + } + collectSeries(mNode, timeseriesSchemas); + } + + /** + * For a path, infer all storage groups it may belong to. + * The path can have wildcards. + * + * Consider the path into two parts: (1) the sub path which can not contain a storage group name and + * (2) the sub path which is substring that begin after the storage group name. + * + * (1) Suppose the part of the path can not contain a storage group name (e.g., + * "root".contains("root.sg") == false), then: + * If the wildcard is not at the tail, then for each wildcard, only one level will be inferred + * and the wildcard will be removed. + * If the wildcard is at the tail, then the inference will go on until the storage groups are found + * and the wildcard will be kept. + * (2) Suppose the part of the path is a substring that begin after the storage group name. (e.g., + * For "root.*.sg1.a.*.b.*" and "root.x.sg1" is a storage group, then this part is "a.*.b.*"). + * For this part, keep what it is. + * + * Assuming we have three SGs: root.group1, root.group2, root.area1.group3 + * Eg1: + * for input "root.*", returns ("root.group1", "root.group1.*"), ("root.group2", "root.group2.*") + * ("root.area1.group3", "root.area1.group3.*") + * Eg2: + * for input "root.*.s1", returns ("root.group1", "root.group1.s1"), ("root.group2", "root.group2.s1") + * + * Eg3: + * for input "root.area1.*", returns ("root.area1.group3", "root.area1.group3.*") + * + * @param path can be a prefix or a full path. + * @return StorageGroupName-FullPath pairs + */ + public Map determineStorageGroup(String path) throws IllegalPathException { + lock.readLock().lock(); + try { + return mtree.determineStorageGroup(path); + } finally { + lock.readLock().unlock(); + } + } } diff --git a/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java b/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java index 9a07c20879919..87b6afda7588b 100644 --- a/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java +++ b/server/src/main/java/org/apache/iotdb/db/metadata/MTree.java @@ -32,6 +32,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Set; import java.util.TreeSet; import java.util.regex.Pattern; @@ -231,6 +232,7 @@ boolean isStorageGroup(String path) { * * @param path Format: root.node(.node)+ */ + String deleteTimeseriesAndReturnEmptyStorageGroup(String path) throws MetadataException { MNode curNode = getNodeByPath(path); if (!(curNode instanceof LeafMNode)) { @@ -755,4 +757,63 @@ private static JSONObject combineJSONObjects(JSONObject a, JSONObject b) { } return res; } + + Map determineStorageGroup(String path) throws IllegalPathException { + Map paths = new HashMap<>(); + String[] nodes = MetaUtils.getNodeNames(path); + if (nodes.length == 0 || !nodes[0].equals(root.getName())) { + throw new IllegalPathException(path); + } + + Deque nodeStack = new ArrayDeque<>(); + Deque depthStack = new ArrayDeque<>(); + if (!root.getChildren().isEmpty()) { + nodeStack.push(root); + depthStack.push(0); + } + + while (!nodeStack.isEmpty()) { + MNode mNode = nodeStack.removeFirst(); + int depth = depthStack.removeFirst(); + + determineStorageGroup(depth + 1, nodes, mNode, paths, nodeStack, depthStack); + } + return paths; + } + + /** + * Try determining the storage group using the children of a mNode. If one child is a storage + * group node, put a storageGroupName-fullPath pair into paths. Otherwise put the children that + * match the path into the queue and discard other children. + */ + private void determineStorageGroup(int depth, String[] nodes, MNode mNode, + Map paths, Deque nodeStack, Deque depthStack) { + String currNode = depth >= nodes.length ? PATH_WILDCARD : nodes[depth]; + for (Entry entry : mNode.getChildren().entrySet()) { + if (!currNode.equals(PATH_WILDCARD) && !currNode.equals(entry.getKey())) { + continue; + } + // this child is desired + MNode child = entry.getValue(); + if (child instanceof StorageGroupMNode) { + // we have found one storage group, record it + String sgName = child.getFullPath(); + // concat the remaining path with the storage group name + StringBuilder pathWithKnownSG = new StringBuilder(sgName); + for (int i = depth + 1; i < nodes.length; i++) { + pathWithKnownSG.append(IoTDBConstant.PATH_SEPARATOR).append(nodes[i]); + } + if (depth >= nodes.length - 1 && currNode.equals(PATH_WILDCARD)) { + // the we find the sg at the last node and the last node is a wildcard (find "root + // .group1", for "root.*"), also append the wildcard (to make "root.group1.*") + pathWithKnownSG.append(IoTDBConstant.PATH_SEPARATOR).append(PATH_WILDCARD); + } + paths.put(sgName, pathWithKnownSG.toString()); + } else if (!child.getChildren().isEmpty()) { + // push it back so we can traver its children later + nodeStack.push(child); + depthStack.push(depth); + } + } + } } diff --git a/server/src/main/java/org/apache/iotdb/db/qp/Planner.java b/server/src/main/java/org/apache/iotdb/db/qp/Planner.java index 19a4d69a7cf2e..bd1089f74e08b 100644 --- a/server/src/main/java/org/apache/iotdb/db/qp/Planner.java +++ b/server/src/main/java/org/apache/iotdb/db/qp/Planner.java @@ -41,7 +41,7 @@ */ public class Planner { - private ParseDriver parseDriver; + protected ParseDriver parseDriver; public Planner() { this.parseDriver = new ParseDriver(); @@ -70,7 +70,7 @@ public PhysicalPlan parseSQLToPhysicalPlan(String sqlStr, ZoneId zoneId) * @return optimized logical operator * @throws LogicalOptimizeException exception in logical optimizing */ - private Operator logicalOptimize(Operator operator) + protected Operator logicalOptimize(Operator operator) throws LogicalOperatorException { switch (operator.getType()) { case AUTHOR: @@ -111,7 +111,7 @@ private Operator logicalOptimize(Operator operator) */ private SFWOperator optimizeSFWOperator(SFWOperator root) throws LogicalOperatorException { - ConcatPathOptimizer concatPathOptimizer = new ConcatPathOptimizer(); + ConcatPathOptimizer concatPathOptimizer = getConcatPathOptimizer(); root = (SFWOperator) concatPathOptimizer.transform(root); FilterOperator filter = root.getFilterOperator(); if (filter == null) { @@ -127,4 +127,7 @@ private SFWOperator optimizeSFWOperator(SFWOperator root) return root; } + protected ConcatPathOptimizer getConcatPathOptimizer() { + return new ConcatPathOptimizer(); + } } diff --git a/server/src/main/java/org/apache/iotdb/db/qp/executor/IPlanExecutor.java b/server/src/main/java/org/apache/iotdb/db/qp/executor/IPlanExecutor.java index ec25529697226..b2466dbcda823 100644 --- a/server/src/main/java/org/apache/iotdb/db/qp/executor/IPlanExecutor.java +++ b/server/src/main/java/org/apache/iotdb/db/qp/executor/IPlanExecutor.java @@ -20,11 +20,13 @@ import java.io.IOException; import java.sql.SQLException; -import org.apache.iotdb.db.exception.metadata.MetadataException; import org.apache.iotdb.db.exception.StorageEngineException; +import org.apache.iotdb.db.exception.metadata.MetadataException; import org.apache.iotdb.db.exception.query.QueryProcessException; import org.apache.iotdb.db.qp.physical.PhysicalPlan; -import org.apache.iotdb.db.qp.physical.crud.*; +import org.apache.iotdb.db.qp.physical.crud.BatchInsertPlan; +import org.apache.iotdb.db.qp.physical.crud.DeletePlan; +import org.apache.iotdb.db.qp.physical.crud.InsertPlan; import org.apache.iotdb.db.query.context.QueryContext; import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException; import org.apache.iotdb.tsfile.read.common.Path; @@ -89,5 +91,4 @@ void update(Path path, long startTime, long endTime, String value) * @return result of each row */ Integer[] insertBatch(BatchInsertPlan batchInsertPlan) throws QueryProcessException; - } diff --git a/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java b/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java index 56047dc60dc2d..d0c1251a047f7 100644 --- a/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java +++ b/server/src/main/java/org/apache/iotdb/db/qp/executor/PlanExecutor.java @@ -34,6 +34,7 @@ import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_TTL; import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_USER; import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_VALUE; +import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.PATH_SEPARATOR; import static org.apache.iotdb.tsfile.common.constant.TsFileConstant.TSFILE_SUFFIX; import java.io.File; @@ -62,7 +63,7 @@ import org.apache.iotdb.db.engine.storagegroup.TsFileResource; import org.apache.iotdb.db.exception.StorageEngineException; import org.apache.iotdb.db.exception.metadata.MetadataException; -import org.apache.iotdb.db.exception.query.PathException; +import org.apache.iotdb.db.exception.metadata.PathNotExistException; import org.apache.iotdb.db.exception.query.QueryProcessException; import org.apache.iotdb.db.metadata.MManager; import org.apache.iotdb.db.metadata.mnode.InternalMNode; @@ -128,9 +129,7 @@ public class PlanExecutor implements IPlanExecutor { // for data query - private IQueryRouter queryRouter; - // for insert - private StorageEngine storageEngine; + protected IQueryRouter queryRouter; // for system schema private MManager mManager; // for administration @@ -138,7 +137,6 @@ public class PlanExecutor implements IPlanExecutor { public PlanExecutor() throws QueryProcessException { queryRouter = new QueryRouter(); - storageEngine = StorageEngine.getInstance(); mManager = MManager.getInstance(); try { authorizer = LocalFileAuthorizer.getInstance(); @@ -223,7 +221,7 @@ public boolean processNonQuery(PhysicalPlan plan) throws QueryProcessException { } } - private QueryDataSet processDataQuery(QueryPlan queryPlan, QueryContext context) + protected QueryDataSet processDataQuery(QueryPlan queryPlan, QueryContext context) throws StorageEngineException, QueryFilterOptimizationException, QueryProcessException, IOException { QueryDataSet queryDataSet; @@ -319,7 +317,7 @@ protected List getPaths(String path) throws MetadataException { return MManager.getInstance().getAllTimeseriesName(path); } - private List getNodesList(String schemaPattern, int level) throws MetadataException { + protected List getNodesList(String schemaPattern, int level) throws MetadataException { return MManager.getInstance().getNodesList(schemaPattern, level); } @@ -340,7 +338,7 @@ private QueryDataSet processShowDevices(ShowDevicesPlan showDevicesPlan) throws MetadataException { ListDataSet listDataSet = new ListDataSet(Collections.singletonList(new Path(COLUMN_DEVICES)), Collections.singletonList(TSDataType.TEXT)); - Set devices = MManager.getInstance().getDevices(showDevicesPlan.getPath().toString()); + Set devices = getDevices(showDevicesPlan.getPath().toString()); for (String s : devices) { RowRecord record = new RowRecord(0); Field field = new Field(TSDataType.TEXT); @@ -351,10 +349,13 @@ private QueryDataSet processShowDevices(ShowDevicesPlan showDevicesPlan) return listDataSet; } + protected Set getDevices(String path) throws MetadataException { + return MManager.getInstance().getDevices(path); + } + private QueryDataSet processShowChildPaths(ShowChildPathsPlan showChildPathsPlan) throws MetadataException { - Set childPathsList = MManager.getInstance() - .getChildNodePathInNextLevel(showChildPathsPlan.getPath().toString()); + Set childPathsList = getPathNextChildren(showChildPathsPlan.getPath().toString()); ListDataSet listDataSet = new ListDataSet( Collections.singletonList(new Path(COLUMN_CHILD_PATHS)), Collections.singletonList(TSDataType.TEXT)); @@ -368,6 +369,10 @@ private QueryDataSet processShowChildPaths(ShowChildPathsPlan showChildPathsPlan return listDataSet; } + protected Set getPathNextChildren(String path) throws MetadataException { + return MManager.getInstance().getChildNodePathInNextLevel(path); + } + private QueryDataSet processShowStorageGroup() { ListDataSet listDataSet = new ListDataSet( Collections.singletonList(new Path(COLUMN_STORAGE_GROUP)), @@ -393,8 +398,7 @@ private QueryDataSet processShowTimeseries(ShowTimeSeriesPlan timeSeriesPlan) new Path(COLUMN_TIMESERIES_COMPRESSION)), Arrays.asList(TSDataType.TEXT, TSDataType.TEXT, TSDataType.TEXT, TSDataType.TEXT, TSDataType.TEXT)); - List timeseriesList = MManager.getInstance() - .getAllTimeseriesSchema(timeSeriesPlan.getPath().toString()); + List timeseriesList = getTimeseriesSchemas(timeSeriesPlan.getPath().toString()); for (String[] list : timeseriesList) { RowRecord record = new RowRecord(0); for (String s : list) { @@ -407,6 +411,10 @@ private QueryDataSet processShowTimeseries(ShowTimeSeriesPlan timeSeriesPlan) return listDataSet; } + protected List getTimeseriesSchemas(String path) throws MetadataException { + return MManager.getInstance().getAllTimeseriesSchema(path); + } + private QueryDataSet processShowTTLQuery(ShowTTLPlan showTTLPlan) { ListDataSet listDataSet = new ListDataSet( Arrays.asList(new Path(COLUMN_STORAGE_GROUP), new Path(COLUMN_TTL)) @@ -506,18 +514,12 @@ public void delete(DeletePlan deletePlan) throws QueryProcessException { try { Set existingPaths = new HashSet<>(); for (Path p : deletePlan.getPaths()) { - existingPaths.addAll(mManager.getAllTimeseriesName(p.getFullPath())); + existingPaths.addAll(getPaths(p.getFullPath())); } if (existingPaths.isEmpty()) { throw new QueryProcessException( "TimeSeries does not exist and its data cannot be deleted"); } - for (String onePath : existingPaths) { - if (!mManager.isPathExist(onePath)) { - throw new QueryProcessException(String - .format("TimeSeries %s does not exist and its data cannot be deleted", onePath)); - } - } for (String path : existingPaths) { delete(new Path(path), deletePlan.getDeleteTime()); } @@ -614,7 +616,7 @@ private void createSchemaAutomatically(List chunkGroupMetaDa .createTimeseries(fullPath, schema.getType(), schema.getEncodingType(), schema.getCompressor(), Collections.emptyMap()); if (result) { - storageEngine + StorageEngine.getInstance() .addTimeSeries(new Path(fullPath), schema.getType(), schema.getEncodingType(), schema.getCompressor(), Collections.emptyMap()); } @@ -687,7 +689,7 @@ public void delete(Path path, long timestamp) throws QueryProcessException { String.format("Time series %s does not exist.", path.getFullPath())); } mManager.getStorageGroupName(path.getFullPath()); - storageEngine.delete(deviceId, measurementId, timestamp); + StorageEngine.getInstance().delete(deviceId, measurementId, timestamp); } catch (MetadataException | StorageEngineException e) { throw new QueryProcessException(e); } @@ -707,9 +709,7 @@ public void insert(InsertPlan insertPlan) throws QueryProcessException { String measurement = measurementList[i]; if (!node.hasChild(measurement)) { if (!IoTDBDescriptor.getInstance().getConfig().isAutoCreateSchemaEnabled()) { - throw new QueryProcessException( - String.format("Current deviceId[%s] does not contain measurement:%s", - deviceId, measurement)); + throw new PathNotExistException(deviceId + PATH_SEPARATOR + measurement); } TSDataType dataType = TypeInferenceUtils.getPredictedDataType(strValues[i]); Path path = new Path(deviceId, measurement); @@ -719,7 +719,7 @@ public void insert(InsertPlan insertPlan) throws QueryProcessException { TSFileDescriptor.getInstance().getConfig().getCompressor(), Collections.emptyMap()); if (result) { - storageEngine.addTimeSeries(path, dataType, getDefaultEncoding(dataType)); + StorageEngine.getInstance().addTimeSeries(path, dataType, getDefaultEncoding(dataType)); } } MNode measurementNode = node.getChild(measurement); @@ -731,8 +731,8 @@ public void insert(InsertPlan insertPlan) throws QueryProcessException { dataTypes[i] = measurementNode.getSchema().getType(); } insertPlan.setDataTypes(dataTypes); - storageEngine.insert(insertPlan); - } catch (PathException | StorageEngineException | MetadataException e) { + StorageEngine.getInstance().insert(insertPlan); + } catch (StorageEngineException | MetadataException e) { throw new QueryProcessException(e); } } @@ -785,7 +785,7 @@ public Integer[] insertBatch(BatchInsertPlan batchInsertPlan) throws QueryProces TSFileDescriptor.getInstance().getConfig().getCompressor(), Collections.emptyMap()); if (result) { - storageEngine + StorageEngine.getInstance() .addTimeSeries(path, dataType, getDefaultEncoding(dataType)); } } @@ -803,8 +803,8 @@ public Integer[] insertBatch(BatchInsertPlan batchInsertPlan) throws QueryProces measurementNode.getSchema().getType())); } } - return storageEngine.insertBatch(batchInsertPlan); - } catch (PathException | StorageEngineException | MetadataException e) { + return StorageEngine.getInstance().insertBatch(batchInsertPlan); + } catch (StorageEngineException | MetadataException e) { throw new QueryProcessException(e); } } @@ -892,7 +892,7 @@ private boolean createTimeSeries(CreateTimeSeriesPlan createTimeSeriesPlan) boolean result = mManager .createTimeseries(path.getFullPath(), dataType, encoding, compressor, props); if (result) { - storageEngine.addTimeSeries(path, dataType, encoding, compressor, props); + StorageEngine.getInstance().addTimeSeries(path, dataType, encoding, compressor, props); } } catch (StorageEngineException | MetadataException e) { throw new QueryProcessException(e); @@ -910,7 +910,7 @@ private boolean deleteTimeSeries(DeleteTimeSeriesPlan deleteTimeSeriesPlan) emptyStorageGroups.addAll(mManager.deleteTimeseries(path.toString())); } for (String deleteStorageGroup : emptyStorageGroups) { - storageEngine.deleteAllDataFilesInOneStorageGroup(deleteStorageGroup); + StorageEngine.getInstance().deleteAllDataFilesInOneStorageGroup(deleteStorageGroup); } } catch (MetadataException e) { throw new QueryProcessException(e); @@ -918,7 +918,7 @@ private boolean deleteTimeSeries(DeleteTimeSeriesPlan deleteTimeSeriesPlan) return true; } - private boolean setStorageGroup(SetStorageGroupPlan setStorageGroupPlan) + public boolean setStorageGroup(SetStorageGroupPlan setStorageGroupPlan) throws QueryProcessException { Path path = setStorageGroupPlan.getPath(); try { @@ -934,7 +934,7 @@ private boolean deleteStorageGroups(DeleteStorageGroupPlan deleteStorageGroupPla List deletePathList = new ArrayList<>(); try { for (Path storageGroupPath : deleteStorageGroupPlan.getPaths()) { - storageEngine.deleteStorageGroup(storageGroupPath.getFullPath()); + StorageEngine.getInstance().deleteStorageGroup(storageGroupPath.getFullPath()); deletePathList.add(storageGroupPath.getFullPath()); } mManager.deleteStorageGroups(deletePathList); diff --git a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/FilterOperator.java b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/FilterOperator.java index 67f5946b90ca3..bc22acb5c24f9 100644 --- a/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/FilterOperator.java +++ b/server/src/main/java/org/apache/iotdb/db/qp/logical/crud/FilterOperator.java @@ -44,16 +44,16 @@ public class FilterOperator extends Operator implements Comparable { // it is the symbol of token. e.g. AND is & and OR is | - protected String tokenSymbol; + String tokenSymbol; - protected List childOperators; + private List childOperators; // leaf filter operator means it doesn't have left and right child filterOperator. Leaf filter // should set FunctionOperator. - protected boolean isLeaf = false; + protected boolean isLeaf; // isSingle being true means all recursive children of this filter belong to one seriesPath. - protected boolean isSingle = false; + boolean isSingle = false; // if isSingle = false, singlePath must be null - protected Path singlePath = null; + Path singlePath = null; public FilterOperator(int tokenType) { super(tokenType); diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/PhysicalPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/PhysicalPlan.java index c86a3e0876032..f2d3fc0dfdd39 100644 --- a/server/src/main/java/org/apache/iotdb/db/qp/physical/PhysicalPlan.java +++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/PhysicalPlan.java @@ -18,6 +18,7 @@ */ package org.apache.iotdb.db.qp.physical; +import java.io.DataOutputStream; import java.io.IOException; import java.nio.ByteBuffer; import java.util.Collections; @@ -26,6 +27,8 @@ import org.apache.iotdb.db.qp.physical.crud.BatchInsertPlan; import org.apache.iotdb.db.qp.physical.crud.DeletePlan; import org.apache.iotdb.db.qp.physical.crud.InsertPlan; +import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan; +import org.apache.iotdb.db.qp.physical.sys.SetStorageGroupPlan; import org.apache.iotdb.tsfile.read.common.Path; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; @@ -34,10 +37,23 @@ */ public abstract class PhysicalPlan { + private static final String SERIALIZATION_UNIMPLEMENTED = "serialization unimplemented"; + private boolean isQuery; private Operator.OperatorType operatorType; private static final int NULL_VALUE_LEN = -1; + //for cluster mode, whether the plan may be splitted into several sub plans + protected boolean canbeSplit = true; + + /** + * whether the plan can be split into more than one Plans. + * Only used in the cluster mode. + */ + public boolean canbeSplit() { + return canbeSplit; + } + protected PhysicalPlan(boolean isQuery) { this.isQuery = isQuery; } @@ -73,12 +89,16 @@ public void setQuery(boolean query) { isQuery = query; } + public void serializeTo(DataOutputStream stream) throws IOException { + throw new UnsupportedOperationException(SERIALIZATION_UNIMPLEMENTED); + } + public void serializeTo(ByteBuffer buffer) { - throw new UnsupportedOperationException("serialize of unimplemented"); + throw new UnsupportedOperationException(SERIALIZATION_UNIMPLEMENTED); } public void deserializeFrom(ByteBuffer buffer) { - throw new UnsupportedOperationException("serialize of unimplemented"); + throw new UnsupportedOperationException(SERIALIZATION_UNIMPLEMENTED); } protected void putString(ByteBuffer buffer, String value) { @@ -89,6 +109,14 @@ protected void putString(ByteBuffer buffer, String value) { } } + protected void putString(DataOutputStream stream, String value) throws IOException { + if (value == null) { + stream.writeInt(NULL_VALUE_LEN); + } else { + ReadWriteIOUtils.write(value, stream); + } + } + protected String readString(ByteBuffer buffer) { int valueLen = buffer.getInt(); if (valueLen == NULL_VALUE_LEN) { @@ -110,6 +138,7 @@ public static PhysicalPlan create(ByteBuffer buffer) throws IOException { } PhysicalPlanType type = PhysicalPlanType.values()[typeNum]; PhysicalPlan plan; + // TODO-Cluster: support more plans switch (type) { case INSERT: plan = new InsertPlan(); @@ -123,6 +152,14 @@ public static PhysicalPlan create(ByteBuffer buffer) throws IOException { plan = new BatchInsertPlan(); plan.deserializeFrom(buffer); break; + case SET_STORAGE_GROUP: + plan = new SetStorageGroupPlan(); + plan.deserializeFrom(buffer); + break; + case CREATE_TIMESERIES: + plan = new CreateTimeSeriesPlan(); + plan.deserializeFrom(buffer); + break; default: throw new IOException("unrecognized log type " + type); } @@ -131,7 +168,7 @@ public static PhysicalPlan create(ByteBuffer buffer) throws IOException { } public enum PhysicalPlanType { - INSERT, DELETE, BATCHINSERT + INSERT, DELETE, BATCHINSERT, SET_STORAGE_GROUP, CREATE_TIMESERIES } diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/BatchInsertPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/BatchInsertPlan.java index 11382a476d31b..07a4d9767c519 100644 --- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/BatchInsertPlan.java +++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/BatchInsertPlan.java @@ -18,6 +18,8 @@ */ package org.apache.iotdb.db.qp.physical.crud; +import java.io.DataOutputStream; +import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; @@ -42,11 +44,13 @@ public class BatchInsertPlan extends PhysicalPlan { + private static final String DATATYPE_UNSUPPORTED = "Data type %s is not supported."; + private String deviceId; private String[] measurements; private TSDataType[] dataTypes; - private long[] times; + private long[] times; // times should be sorted. It is done in the session API. private ByteBuffer timeBuffer; private Object[] columns; @@ -69,6 +73,11 @@ public BatchInsertPlan(String deviceId, List measurements) { this.deviceId = deviceId; setMeasurements(measurements); } + public BatchInsertPlan(String deviceId, String[] measurements) { + super(false, OperatorType.BATCHINSERT); + this.deviceId = deviceId; + setMeasurements(measurements); + } public BatchInsertPlan(String deviceId, String[] measurements, List dataTypes) { super(false, OperatorType.BATCHINSERT); @@ -114,6 +123,94 @@ public List getPaths() { return ret; } + @Override + public void serializeTo(DataOutputStream stream) throws IOException { + int type = PhysicalPlanType.BATCHINSERT.ordinal(); + stream.writeByte((byte) type); + + putString(stream, deviceId); + + stream.writeInt(measurements.length); + for (String m : measurements) { + putString(stream, m); + } + + for (TSDataType dataType : dataTypes) { + stream.writeShort(dataType.serialize()); + } + + stream.writeInt(index.size()); + + if (timeBuffer == null) { + for(int loc : index){ + stream.writeLong(times[loc]); + } + } else { + stream.write(timeBuffer.array()); + timeBuffer = null; + } + + if (valueBuffer == null) { + serializeValues(stream); + } else { + stream.write(valueBuffer.array()); + valueBuffer = null; + } + } + + private void serializeValues(DataOutputStream stream) throws IOException { + for (int i = 0; i < measurements.length; i++) { + serializeColumn(dataTypes[i], columns[i], stream, index); + } + } + + private void serializeColumn(TSDataType dataType, Object column, DataOutputStream stream, + Set index) + throws IOException { + switch (dataType) { + case INT32: + int[] intValues = (int[]) column; + for(int loc : index){ + stream.writeInt(intValues[loc]); + } + break; + case INT64: + long[] longValues = (long[]) column; + for(int loc : index){ + stream.writeLong(longValues[loc]); + } + break; + case FLOAT: + float[] floatValues = (float[]) column; + for(int loc : index){ + stream.writeFloat(floatValues[loc]); + } + break; + case DOUBLE: + double[] doubleValues = (double[]) column; + for(int loc : index){ + stream.writeDouble(doubleValues[loc]); + } + break; + case BOOLEAN: + boolean[] boolValues = (boolean[]) column; + for(int loc : index){ + stream.write(BytesUtils.boolToByte(boolValues[loc])); + } + break; + case TEXT: + Binary[] binaryValues = (Binary[]) column; + for(int loc : index){ + stream.writeInt(binaryValues[loc].getLength()); + stream.write(binaryValues[loc].getValues()); + } + break; + default: + throw new UnSupportedDataTypeException( + String.format(DATATYPE_UNSUPPORTED, dataType)); + } + } + @Override public void serializeTo(ByteBuffer buffer) { int type = PhysicalPlanType.BATCHINSERT.ordinal(); @@ -142,57 +239,65 @@ public void serializeTo(ByteBuffer buffer) { } if (valueBuffer == null) { - for (int i = 0; i < measurements.length; i++) { - TSDataType dataType = dataTypes[i]; - switch (dataType) { - case INT32: - int[] intValues = (int[]) columns[i]; - for (int j = start; j < end; j++) { - buffer.putInt(intValues[j]); - } - break; - case INT64: - long[] longValues = (long[]) columns[i]; - for (int j = start; j < end; j++) { - buffer.putLong(longValues[j]); - } - break; - case FLOAT: - float[] floatValues = (float[]) columns[i]; - for (int j = start; j < end; j++) { - buffer.putFloat(floatValues[j]); - } - break; - case DOUBLE: - double[] doubleValues = (double[]) columns[i]; - for (int j = start; j < end; j++) { - buffer.putDouble(doubleValues[j]); - } - break; - case BOOLEAN: - boolean[] boolValues = (boolean[]) columns[i]; - for (int j = start; j < end; j++) { - buffer.putInt(BytesUtils.boolToByte(boolValues[j])); - } - break; - case TEXT: - Binary[] binaryValues = (Binary[]) columns[i]; - for (int j = start; j < end; j++) { - buffer.putInt(binaryValues[j].getLength()); - buffer.put(binaryValues[j].getValues()); - } - break; - default: - throw new UnSupportedDataTypeException( - String.format("Data type %s is not supported.", dataType)); - } - } + serializeValues(buffer); } else { buffer.put(valueBuffer.array()); valueBuffer = null; } } + private void serializeValues(ByteBuffer buffer) { + for (int i = 0; i < measurements.length; i++) { + serializeColumn(dataTypes[i], columns[i], buffer, start, end); + } + } + + private void serializeColumn(TSDataType dataType, Object column, ByteBuffer buffer, + int start, int end) { + switch (dataType) { + case INT32: + int[] intValues = (int[]) column; + for (int j = start; j < end; j++) { + buffer.putInt(intValues[j]); + } + break; + case INT64: + long[] longValues = (long[]) column; + for (int j = start; j < end; j++) { + buffer.putLong(longValues[j]); + } + break; + case FLOAT: + float[] floatValues = (float[]) column; + for (int j = start; j < end; j++) { + buffer.putFloat(floatValues[j]); + } + break; + case DOUBLE: + double[] doubleValues = (double[]) column; + for (int j = start; j < end; j++) { + buffer.putDouble(doubleValues[j]); + } + break; + case BOOLEAN: + boolean[] boolValues = (boolean[]) column; + for (int j = start; j < end; j++) { + buffer.putInt(BytesUtils.boolToByte(boolValues[j])); + } + break; + case TEXT: + Binary[] binaryValues = (Binary[]) column; + for (int j = start; j < end; j++) { + buffer.putInt(binaryValues[j].getLength()); + buffer.put(binaryValues[j].getValues()); + } + break; + default: + throw new UnSupportedDataTypeException( + String.format(DATATYPE_UNSUPPORTED, dataType)); + } + } + public void setTimeBuffer(ByteBuffer timeBuffer) { this.timeBuffer = timeBuffer; this.timeBuffer.position(0); @@ -259,6 +364,10 @@ public void setDataTypes(List dataTypes) { } } + public void setDataTypes(TSDataType[] dataTypes) { + this.dataTypes = dataTypes; + } + public Object[] getColumns() { return columns; } @@ -288,13 +397,13 @@ public long getMaxTime() { if (maxTime != null) { return maxTime; } - long maxTime = Long.MIN_VALUE; + long tmpMaxTime = Long.MIN_VALUE; for (Long time : times) { - if (time > maxTime) { - maxTime = time; + if (time > tmpMaxTime) { + tmpMaxTime = time; } } - return maxTime; + return tmpMaxTime; } public TimeValuePair composeLastTimeValuePair(int measurementIndex) { @@ -329,7 +438,7 @@ public TimeValuePair composeLastTimeValuePair(int measurementIndex) { break; default: throw new UnSupportedDataTypeException( - String.format("Data type %s is not supported.", dataTypes[measurementIndex])); + String.format(DATATYPE_UNSUPPORTED, dataTypes[measurementIndex])); } return new TimeValuePair(times[end - 1], value); } @@ -349,4 +458,5 @@ public int getRowCount() { public void setRowCount(int size) { this.rowCount = size; } + } diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/DeletePlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/DeletePlan.java index a46aaeae1af75..d2f12369b4416 100644 --- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/DeletePlan.java +++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/DeletePlan.java @@ -18,6 +18,8 @@ */ package org.apache.iotdb.db.qp.physical.crud; +import java.io.DataOutputStream; +import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; @@ -101,6 +103,14 @@ public boolean equals(Object o) { return deleteTime == that.deleteTime && Objects.equals(paths, that.paths); } + @Override + public void serializeTo(DataOutputStream stream) throws IOException { + int type = PhysicalPlanType.DELETE.ordinal(); + stream.writeByte((byte) type); + stream.writeLong(deleteTime); + putString(stream, paths.get(0).getFullPath()); + } + @Override public void serializeTo(ByteBuffer buffer) { int type = PhysicalPlanType.DELETE.ordinal(); diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertPlan.java index 6544298e7567c..4d7fc53485bd6 100644 --- a/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertPlan.java +++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/crud/InsertPlan.java @@ -18,6 +18,8 @@ */ package org.apache.iotdb.db.qp.physical.crud; +import java.io.DataOutputStream; +import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; @@ -46,6 +48,7 @@ public class InsertPlan extends PhysicalPlan { public InsertPlan() { super(false, OperatorType.INSERT); + canbeSplit = false; } @TestOnly @@ -55,6 +58,7 @@ public InsertPlan(String deviceId, long insertTime, String measurement, String i this.deviceId = deviceId; this.measurements = new String[] {measurement}; this.values = new String[] {insertValue}; + canbeSplit = false; } public InsertPlan(TSRecord tsRecord) { @@ -69,6 +73,7 @@ public InsertPlan(TSRecord tsRecord) { dataTypes[i] = tsRecord.dataPointList.get(i).getType(); values[i] = tsRecord.dataPointList.get(i).getValue().toString(); } + canbeSplit = false; } public InsertPlan(String deviceId, long insertTime, String[] measurementList, @@ -78,6 +83,7 @@ public InsertPlan(String deviceId, long insertTime, String[] measurementList, this.deviceId = deviceId; this.measurements = measurementList; this.values = insertValues; + canbeSplit = false; } public long getTime() { @@ -149,6 +155,25 @@ public int hashCode() { return Objects.hash(deviceId, time); } + @Override + public void serializeTo(DataOutputStream stream) throws IOException { + int type = PhysicalPlanType.INSERT.ordinal(); + stream.writeByte((byte) type); + stream.writeLong(time); + + putString(stream, deviceId); + + stream.writeInt(measurements.length); + for (String m : measurements) { + putString(stream, m); + } + + stream.writeInt(values.length); + for (String m : values) { + putString(stream, m); + } + } + @Override public void serializeTo(ByteBuffer buffer) { int type = PhysicalPlanType.INSERT.ordinal(); diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CountPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CountPlan.java index 044d838f3e5c5..b1851ad020416 100644 --- a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CountPlan.java +++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CountPlan.java @@ -22,6 +22,9 @@ /** * CountPlan is used to count time-series and count nodes. + * COUNT_TIMESERIES if using "COUNT TIMESERIES " and only this command supports wildcard. + * COUNT_NODE_TIMESERIES if using "COUNT TIMESERIES GROUP BY LEVEL=" + * COUNT_NODE if using "COUNT NODES LEVEL=" */ public class CountPlan extends ShowPlan { diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateTimeSeriesPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateTimeSeriesPlan.java index a6ce3cd78f9dc..7e35ff9648493 100644 --- a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateTimeSeriesPlan.java +++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/CreateTimeSeriesPlan.java @@ -18,10 +18,13 @@ */ package org.apache.iotdb.db.qp.physical.sys; +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; import java.util.Collections; import java.util.List; import java.util.Map; - +import java.util.Objects; import org.apache.iotdb.db.qp.logical.Operator; import org.apache.iotdb.db.qp.physical.PhysicalPlan; import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType; @@ -36,8 +39,13 @@ public class CreateTimeSeriesPlan extends PhysicalPlan { private TSEncoding encoding; private CompressionType compressor; private Map props; - - public CreateTimeSeriesPlan(Path path, TSDataType dataType, TSEncoding encoding, + + public CreateTimeSeriesPlan() { + super(false, Operator.OperatorType.CREATE_TIMESERIES); + canbeSplit = false; + } + + public CreateTimeSeriesPlan(Path path, TSDataType dataType, TSEncoding encoding, CompressionType compressor, Map props) { super(false, Operator.OperatorType.CREATE_TIMESERIES); this.path = path; @@ -45,6 +53,7 @@ public CreateTimeSeriesPlan(Path path, TSDataType dataType, TSEncoding encoding, this.encoding = encoding; this.compressor = compressor; this.props = props; + canbeSplit = false; } public Path getPath() { @@ -89,14 +98,8 @@ public void setProps(Map props) { @Override public String toString() { - String ret = String.format("seriesPath: %s%nresultDataType: %s%nencoding: %s%nnamespace type:" - + " ADD_PATH%nargs: ", path, dataType, encoding); - StringBuilder stringBuilder = new StringBuilder(ret.length()+50); - stringBuilder.append(ret); - for (Map.Entry prop : props.entrySet()) { - stringBuilder.append(prop.getKey()).append("=").append(prop.getValue()).append(","); - } - return stringBuilder.toString(); + return String.format("seriesPath: %s, resultDataType: %s, encoding: %s, compression: %s", path, + dataType, encoding, compressor); } @Override @@ -104,4 +107,45 @@ public List getPaths() { return Collections.singletonList(path); } + @Override + public void serializeTo(DataOutputStream stream) throws IOException { + stream.writeByte((byte) PhysicalPlanType.CREATE_TIMESERIES.ordinal()); + byte[] pathBytes = path.getFullPath().getBytes(); + stream.writeInt(pathBytes.length); + stream.write(pathBytes); + stream.write(dataType.ordinal()); + stream.write(encoding.ordinal()); + stream.write(compressor.ordinal()); + } + + @Override + public void deserializeFrom(ByteBuffer buffer) { + int length = buffer.getInt(); + byte[] pathBytes = new byte[length]; + buffer.get(pathBytes); + path = new Path(new String(pathBytes)); + dataType = TSDataType.values()[buffer.get()]; + encoding = TSEncoding.values()[buffer.get()]; + compressor = CompressionType.values()[buffer.get()]; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + CreateTimeSeriesPlan that = (CreateTimeSeriesPlan) o; + return Objects.equals(path, that.path) && + dataType == that.dataType && + encoding == that.encoding && + compressor == that.compressor; + } + + @Override + public int hashCode() { + return Objects.hash(path, dataType, encoding, compressor); + } } diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetStorageGroupPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetStorageGroupPlan.java index 72e9ebcdfb6cb..28d19cd1e960f 100644 --- a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetStorageGroupPlan.java +++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/SetStorageGroupPlan.java @@ -18,16 +18,23 @@ */ package org.apache.iotdb.db.qp.physical.sys; +import java.io.DataOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; - +import java.util.Objects; import org.apache.iotdb.db.qp.logical.Operator; import org.apache.iotdb.db.qp.physical.PhysicalPlan; import org.apache.iotdb.tsfile.read.common.Path; public class SetStorageGroupPlan extends PhysicalPlan { private Path path; - + + public SetStorageGroupPlan() { + super(false, Operator.OperatorType.SET_STORAGE_GROUP); + } + public SetStorageGroupPlan(Path path) { super(false, Operator.OperatorType.SET_STORAGE_GROUP); this.path = path; @@ -50,4 +57,41 @@ public List getPaths() { return ret; } + @Override + public void serializeTo(DataOutputStream stream) throws IOException { + stream.write((byte) PhysicalPlanType.SET_STORAGE_GROUP.ordinal()); + byte[] fullPathBytes = path.getFullPath().getBytes(); + stream.writeInt(fullPathBytes.length); + stream.write(fullPathBytes); + } + + @Override + public void deserializeFrom(ByteBuffer buffer) { + int length = buffer.getInt(); + byte[] fullPathBytes = new byte[length]; + buffer.get(fullPathBytes); + path = new Path(new String(fullPathBytes)); + } + + @Override + public String toString() { + return "SetStorageGroup{" + path + '}'; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SetStorageGroupPlan that = (SetStorageGroupPlan) o; + return Objects.equals(path, that.path); + } + + @Override + public int hashCode() { + return Objects.hash(path); + } } diff --git a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowChildPathsPlan.java b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowChildPathsPlan.java index 9f6c199757819..8490c6bb34f67 100644 --- a/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowChildPathsPlan.java +++ b/server/src/main/java/org/apache/iotdb/db/qp/physical/sys/ShowChildPathsPlan.java @@ -22,14 +22,17 @@ import org.apache.iotdb.tsfile.read.common.Path; public class ShowChildPathsPlan extends ShowPlan { - private Path path; - public ShowChildPathsPlan(ShowContentType showContentType, Path path) { + // the path could be a prefix path with wildcard + private Path prefixPath; + + public ShowChildPathsPlan(ShowContentType showContentType, Path prefixPath) { super(showContentType); - this.path = path; + this.prefixPath = prefixPath; + canbeSplit = false; } public Path getPath() { - return this.path; + return this.prefixPath; } } diff --git a/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java b/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java index e6abde7da79cb..1f2413e593585 100644 --- a/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java +++ b/server/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java @@ -83,7 +83,7 @@ import org.apache.iotdb.db.qp.physical.sys.ShowPlan.ShowContentType; import org.apache.iotdb.db.qp.physical.sys.ShowTTLPlan; import org.apache.iotdb.db.qp.physical.sys.ShowTimeSeriesPlan; -import org.apache.iotdb.db.service.TSServiceImpl; +import org.apache.iotdb.db.utils.SchemaUtils; import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; import org.apache.iotdb.tsfile.read.common.Path; import org.apache.iotdb.tsfile.read.expression.IExpression; @@ -211,6 +211,9 @@ public PhysicalPlan transformToPhysicalPlan(Operator operator) } } + protected TSDataType getSeriesType(String path) throws MetadataException { + return SchemaUtils.getSeriesType(path); + } private PhysicalPlan transformQuery(QueryOperator queryOperator) throws QueryProcessException { @@ -290,8 +293,7 @@ private PhysicalPlan transformQuery(QueryOperator queryOperator) Path fullPath = Path.addPrefixPath(suffixPath, device); try { // remove stars in SELECT to get actual paths - List actualPaths = MManager.getInstance() - .getAllTimeseriesName(fullPath.getFullPath()); + List actualPaths = getMatchedTimeseries(fullPath.getFullPath()); // for actual non exist path if (actualPaths.isEmpty() && originAggregations.isEmpty()) { @@ -313,7 +315,7 @@ private PhysicalPlan transformQuery(QueryOperator queryOperator) pathForDataType = path.getFullPath(); measurementChecked = path.getMeasurement(); } - TSDataType dataType = TSServiceImpl.getSeriesType(pathForDataType); + TSDataType dataType = getSeriesType(pathForDataType); if (dataTypeConsistencyChecker.containsKey(measurementChecked)) { if (!dataType.equals(dataTypeConsistencyChecker.get(measurementChecked))) { throw new QueryProcessException( @@ -429,7 +431,7 @@ private List removeStarsInDeviceWithUnique(List paths) Set deviceSet = new LinkedHashSet<>(); try { for (Path path : paths) { - Set tempDS = MManager.getInstance().getDevices(path.getFullPath()); + Set tempDS = getMatchedDevices(path.getFullPath()); deviceSet.addAll(tempDS); } retDevices = new ArrayList<>(deviceSet); @@ -462,7 +464,7 @@ private void generateDataTypes(QueryPlan queryPlan) throws MetadataException { List paths = queryPlan.getPaths(); List dataTypes = new ArrayList<>(paths.size()); for (Path path : paths) { - TSDataType seriesType = MManager.getInstance().getSeriesType(path.toString()); + TSDataType seriesType = getSeriesType(path.toString()); dataTypes.add(seriesType); queryPlan.addTypeMapping(path, seriesType); } @@ -530,5 +532,13 @@ private void deduplicateAggregation(AggregationPlan queryPlan) { } } } + + protected List getMatchedTimeseries(String path) throws MetadataException { + return MManager.getInstance().getAllTimeseriesName(path); + } + + protected Set getMatchedDevices(String path) throws MetadataException { + return MManager.getInstance().getDevices(path); + } } diff --git a/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java b/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java index 0217c0fe7e45c..f71c60c4ea8ed 100644 --- a/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java +++ b/server/src/main/java/org/apache/iotdb/db/qp/strategy/optimizer/ConcatPathOptimizer.java @@ -18,19 +18,26 @@ */ package org.apache.iotdb.db.qp.strategy.optimizer; +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; import org.apache.iotdb.db.exception.metadata.MetadataException; import org.apache.iotdb.db.exception.query.LogicalOptimizeException; import org.apache.iotdb.db.exception.runtime.SQLParserException; import org.apache.iotdb.db.metadata.MManager; import org.apache.iotdb.db.qp.constant.SQLConstant; import org.apache.iotdb.db.qp.logical.Operator; -import org.apache.iotdb.db.qp.logical.crud.*; +import org.apache.iotdb.db.qp.logical.crud.BasicFunctionOperator; +import org.apache.iotdb.db.qp.logical.crud.FilterOperator; +import org.apache.iotdb.db.qp.logical.crud.FromOperator; +import org.apache.iotdb.db.qp.logical.crud.FunctionOperator; +import org.apache.iotdb.db.qp.logical.crud.QueryOperator; +import org.apache.iotdb.db.qp.logical.crud.SFWOperator; +import org.apache.iotdb.db.qp.logical.crud.SelectOperator; import org.apache.iotdb.tsfile.read.common.Path; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.*; - /** * concat paths in select and from clause. */ @@ -174,7 +181,7 @@ private void concatSelect(List fromPaths, SelectOperator selectOperator) * @param seriesLimit is ensured to be positive integer * @param seriesOffset is ensured to be non-negative integer */ - public void slimitTrim(SelectOperator select, int seriesLimit, int seriesOffset) + private void slimitTrim(SelectOperator select, int seriesLimit, int seriesOffset) throws LogicalOptimizeException { List suffixList = select.getSuffixPaths(); List aggregations = select.getAggregations(); @@ -268,7 +275,7 @@ private List removeStarsInPathWithUnique(List paths) throws LogicalO LinkedHashMap pathMap = new LinkedHashMap<>(); try { for (Path path : paths) { - List all = MManager.getInstance().getAllTimeseriesName(path.getFullPath()); + List all = removeWildcard(path.getFullPath()); for (String subPath : all) { if (!pathMap.containsKey(subPath)) { pathMap.put(subPath, 1); @@ -290,7 +297,7 @@ private void removeStarsInPath(List paths, List afterConcatAggrega List newAggregations = new ArrayList<>(); for (int i = 0; i < paths.size(); i++) { try { - List actualPaths = MManager.getInstance().getAllTimeseriesName(paths.get(i).getFullPath()); + List actualPaths = removeWildcard(paths.get(i).getFullPath()); for (String actualPath : actualPaths) { retPaths.add(new Path(actualPath)); if (afterConcatAggregations != null && !afterConcatAggregations.isEmpty()) { @@ -304,4 +311,8 @@ private void removeStarsInPath(List paths, List afterConcatAggrega selectOperator.setSuffixPathList(retPaths); selectOperator.setAggregations(newAggregations); } + + protected List removeWildcard(String path) throws MetadataException { + return MManager.getInstance().getAllTimeseriesName(path); + } } diff --git a/server/src/main/java/org/apache/iotdb/db/query/aggregation/AggregateResult.java b/server/src/main/java/org/apache/iotdb/db/query/aggregation/AggregateResult.java index 5483bd3884ccd..ee083eef68c1b 100644 --- a/server/src/main/java/org/apache/iotdb/db/query/aggregation/AggregateResult.java +++ b/server/src/main/java/org/apache/iotdb/db/query/aggregation/AggregateResult.java @@ -19,6 +19,7 @@ package org.apache.iotdb.db.query.aggregation; +import java.io.IOException; import java.io.OutputStream; import java.nio.ByteBuffer; import org.apache.iotdb.db.exception.query.QueryProcessException; @@ -29,8 +30,6 @@ import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics; import org.apache.iotdb.tsfile.read.common.BatchData; import org.apache.iotdb.tsfile.utils.Binary; - -import java.io.IOException; import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; public abstract class AggregateResult { @@ -291,4 +290,8 @@ protected boolean hasResult() { return hasResult; } + @Override + public String toString() { + return String.valueOf(getResult()); + } } diff --git a/server/src/main/java/org/apache/iotdb/db/query/control/QueryResourceManager.java b/server/src/main/java/org/apache/iotdb/db/query/control/QueryResourceManager.java index 03cbecc5975ff..a6d4653e21d24 100644 --- a/server/src/main/java/org/apache/iotdb/db/query/control/QueryResourceManager.java +++ b/server/src/main/java/org/apache/iotdb/db/query/control/QueryResourceManager.java @@ -105,7 +105,7 @@ public void endQuery(long queryId) throws StorageEngineException { try { deserializer.close(); } catch (IOException e) { - throw new StorageEngineException(e.getMessage()); + throw new StorageEngineException(e); } } externalSortFileMap.remove(queryId); diff --git a/server/src/main/java/org/apache/iotdb/db/query/dataset/AlignByDeviceDataSet.java b/server/src/main/java/org/apache/iotdb/db/query/dataset/AlignByDeviceDataSet.java index 32d489f57e89c..7974bfb13266e 100644 --- a/server/src/main/java/org/apache/iotdb/db/query/dataset/AlignByDeviceDataSet.java +++ b/server/src/main/java/org/apache/iotdb/db/query/dataset/AlignByDeviceDataSet.java @@ -201,7 +201,7 @@ protected boolean hasNextWithoutConstraint() throws IOException { default: throw new IOException("unsupported DataSetType"); } - } catch (QueryProcessException | QueryFilterOptimizationException | StorageEngineException | IOException e) { + } catch (QueryProcessException | QueryFilterOptimizationException | StorageEngineException e) { throw new IOException(e); } diff --git a/server/src/main/java/org/apache/iotdb/db/query/dataset/RawQueryDataSetWithValueFilter.java b/server/src/main/java/org/apache/iotdb/db/query/dataset/RawQueryDataSetWithValueFilter.java index cd6a0b579e612..7859176c1d981 100644 --- a/server/src/main/java/org/apache/iotdb/db/query/dataset/RawQueryDataSetWithValueFilter.java +++ b/server/src/main/java/org/apache/iotdb/db/query/dataset/RawQueryDataSetWithValueFilter.java @@ -21,16 +21,16 @@ import java.io.IOException; import java.util.List; import org.apache.iotdb.db.query.reader.series.IReaderByTimestamp; -import org.apache.iotdb.db.query.timegenerator.ServerTimeGenerator; import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; import org.apache.iotdb.tsfile.read.common.Field; import org.apache.iotdb.tsfile.read.common.Path; import org.apache.iotdb.tsfile.read.common.RowRecord; import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet; +import org.apache.iotdb.tsfile.read.query.timegenerator.TimeGenerator; public class RawQueryDataSetWithValueFilter extends QueryDataSet { - private ServerTimeGenerator timeGenerator; + private TimeGenerator timeGenerator; private List seriesReaderByTimestampList; private boolean hasCachedRowRecord; private RowRecord cachedRowRecord; @@ -44,7 +44,7 @@ public class RawQueryDataSetWithValueFilter extends QueryDataSet { * @param readers readers in List(IReaderByTimeStamp) structure */ public RawQueryDataSetWithValueFilter(List paths, List dataTypes, - ServerTimeGenerator timeGenerator, List readers) { + TimeGenerator timeGenerator, List readers) { super(paths, dataTypes); this.timeGenerator = timeGenerator; this.seriesReaderByTimestampList = readers; diff --git a/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithValueFilterDataSet.java b/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithValueFilterDataSet.java index 398ef9e0ba3ec..44402cca706ce 100644 --- a/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithValueFilterDataSet.java +++ b/server/src/main/java/org/apache/iotdb/db/query/dataset/groupby/GroupByWithValueFilterDataSet.java @@ -19,9 +19,11 @@ package org.apache.iotdb.db.query.dataset.groupby; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.exception.StorageEngineException; -import org.apache.iotdb.db.exception.query.PathException; import org.apache.iotdb.db.qp.physical.crud.GroupByPlan; import org.apache.iotdb.db.query.aggregation.AggregateResult; import org.apache.iotdb.db.query.context.QueryContext; @@ -34,10 +36,6 @@ import org.apache.iotdb.tsfile.read.common.RowRecord; import org.apache.iotdb.tsfile.read.query.timegenerator.TimeGenerator; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - public class GroupByWithValueFilterDataSet extends GroupByEngineDataSet { private List allDataReaderList; diff --git a/server/src/main/java/org/apache/iotdb/db/query/executor/AggregationExecutor.java b/server/src/main/java/org/apache/iotdb/db/query/executor/AggregationExecutor.java index 9fa96f59a817f..6882e7b51d52c 100644 --- a/server/src/main/java/org/apache/iotdb/db/query/executor/AggregationExecutor.java +++ b/server/src/main/java/org/apache/iotdb/db/query/executor/AggregationExecutor.java @@ -28,7 +28,6 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.engine.querycontext.QueryDataSource; import org.apache.iotdb.db.exception.StorageEngineException; -import org.apache.iotdb.db.exception.query.PathException; import org.apache.iotdb.db.exception.query.QueryProcessException; import org.apache.iotdb.db.qp.physical.crud.AggregationPlan; import org.apache.iotdb.db.query.aggregation.AggregateResult; @@ -36,11 +35,13 @@ import org.apache.iotdb.db.query.control.QueryResourceManager; import org.apache.iotdb.db.query.dataset.SingleDataSet; import org.apache.iotdb.db.query.factory.AggregateResultFactory; -import org.apache.iotdb.db.query.reader.series.IReaderByTimestamp; +import org.apache.iotdb.db.query.filter.TsFileFilter; import org.apache.iotdb.db.query.reader.series.IAggregateReader; +import org.apache.iotdb.db.query.reader.series.IReaderByTimestamp; import org.apache.iotdb.db.query.reader.series.SeriesAggregateReader; import org.apache.iotdb.db.query.reader.series.SeriesReaderByTimestamp; import org.apache.iotdb.db.query.timegenerator.ServerTimeGenerator; +import org.apache.iotdb.db.utils.QueryUtils; import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics; import org.apache.iotdb.tsfile.read.common.BatchData; @@ -50,20 +51,21 @@ import org.apache.iotdb.tsfile.read.expression.impl.GlobalTimeExpression; import org.apache.iotdb.tsfile.read.filter.basic.Filter; import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet; +import org.apache.iotdb.tsfile.read.query.timegenerator.TimeGenerator; public class AggregationExecutor { private List selectedSeries; - private List dataTypes; + protected List dataTypes; private List aggregations; - private IExpression expression; + protected IExpression expression; /** * aggregation batch calculation size. **/ private int aggregateFetchSize; - public AggregationExecutor(AggregationPlan aggregationPlan) { + AggregationExecutor(AggregationPlan aggregationPlan) { this.selectedSeries = aggregationPlan.getDeduplicatedPaths(); this.dataTypes = aggregationPlan.getDeduplicatedDataTypes(); this.aggregations = aggregationPlan.getDeduplicatedAggregations(); @@ -76,7 +78,7 @@ public AggregationExecutor(AggregationPlan aggregationPlan) { * * @param context query context */ - public QueryDataSet executeWithoutValueFilter(QueryContext context) + QueryDataSet executeWithoutValueFilter(QueryContext context) throws StorageEngineException, IOException, QueryProcessException { Filter timeFilter = null; @@ -112,87 +114,112 @@ private List aggregateOneSeries( Filter timeFilter, QueryContext context) throws IOException, QueryProcessException, StorageEngineException { List aggregateResultList = new ArrayList<>(); - List isCalculatedList = new ArrayList<>(); + Path seriesPath = pathToAggrIndexes.getKey(); TSDataType tsDataType = dataTypes.get(pathToAggrIndexes.getValue().get(0)); + for (int i : pathToAggrIndexes.getValue()) { + // construct AggregateResult + AggregateResult aggregateResult = AggregateResultFactory + .getAggrResultByName(aggregations.get(i), tsDataType); + aggregateResultList.add(aggregateResult); + } + aggregateOneSeries(seriesPath, context, timeFilter, tsDataType, aggregateResultList, null); + return aggregateResultList; + } + + private static void aggregateOneSeries(Path seriesPath, QueryContext context, Filter timeFilter, + TSDataType tsDataType, List aggregateResultList, TsFileFilter fileFilter) + throws StorageEngineException, IOException, QueryProcessException { + // construct series reader without value filter QueryDataSource queryDataSource = QueryResourceManager.getInstance() .getQueryDataSource(seriesPath, context, timeFilter); + if (fileFilter != null) { + QueryUtils.filterQueryDataSource(queryDataSource, fileFilter); + } // update filter by TTL timeFilter = queryDataSource.updateFilterUsingTTL(timeFilter); - IAggregateReader seriesReader = new SeriesAggregateReader(pathToAggrIndexes.getKey(), + IAggregateReader seriesReader = new SeriesAggregateReader(seriesPath, tsDataType, context, queryDataSource, timeFilter, null, null); + aggregateFromReader(seriesReader, aggregateResultList); + } - for (int i : pathToAggrIndexes.getValue()) { - // construct AggregateResult - AggregateResult aggregateResult = AggregateResultFactory - .getAggrResultByName(aggregations.get(i), tsDataType); - aggregateResultList.add(aggregateResult); - isCalculatedList.add(false); - } - int remainingToCalculate = pathToAggrIndexes.getValue().size(); + private static void aggregateFromReader(IAggregateReader seriesReader, + List aggregateResultList) throws QueryProcessException, IOException { + int remainingToCalculate = aggregateResultList.size(); + boolean[] isCalculatedArray = new boolean[aggregateResultList.size()]; while (seriesReader.hasNextChunk()) { // cal by chunk statistics if (seriesReader.canUseCurrentChunkStatistics()) { Statistics chunkStatistics = seriesReader.currentChunkStatistics(); - for (int i = 0; i < aggregateResultList.size(); i++) { - if (Boolean.FALSE.equals(isCalculatedList.get(i))) { - AggregateResult aggregateResult = aggregateResultList.get(i); - aggregateResult.updateResultFromStatistics(chunkStatistics); - if (aggregateResult.isCalculatedAggregationResult()) { - isCalculatedList.set(i, true); - remainingToCalculate--; - if (remainingToCalculate == 0) { - return aggregateResultList; - } - } - } + remainingToCalculate = aggregateStatistics(aggregateResultList, isCalculatedArray, + remainingToCalculate, chunkStatistics); + if (remainingToCalculate == 0) { + return; } seriesReader.skipCurrentChunk(); continue; } - while (seriesReader.hasNextPage()) { - //cal by page statistics - if (seriesReader.canUseCurrentPageStatistics()) { - Statistics pageStatistic = seriesReader.currentPageStatistics(); - for (int i = 0; i < aggregateResultList.size(); i++) { - if (Boolean.FALSE.equals(isCalculatedList.get(i))) { - AggregateResult aggregateResult = aggregateResultList.get(i); - aggregateResult.updateResultFromStatistics(pageStatistic); - if (aggregateResult.isCalculatedAggregationResult()) { - isCalculatedList.set(i, true); - remainingToCalculate--; - if (remainingToCalculate == 0) { - return aggregateResultList; - } - } - } + remainingToCalculate = aggregateOverlappedPages(seriesReader, aggregateResultList, + isCalculatedArray, remainingToCalculate); + } + } + + /** + * Aggregate each result in the list with the statistics + * @param aggregateResultList + * @param isCalculatedArray + * @param remainingToCalculate + * @param statistics + * @return new remainingToCalculate + * @throws QueryProcessException + */ + private static int aggregateStatistics(List aggregateResultList, + boolean[] isCalculatedArray, int remainingToCalculate, Statistics statistics) + throws QueryProcessException { + int newRemainingToCalculate = remainingToCalculate; + for (int i = 0; i < aggregateResultList.size(); i++) { + if (!isCalculatedArray[i]) { + AggregateResult aggregateResult = aggregateResultList.get(i); + aggregateResult.updateResultFromStatistics(statistics); + if (aggregateResult.isCalculatedAggregationResult()) { + isCalculatedArray[i] = true; + newRemainingToCalculate--; + if (newRemainingToCalculate == 0) { + return newRemainingToCalculate; } - seriesReader.skipCurrentPage(); - } else { - // cal by page data - BatchData batchData = seriesReader.nextPage(); - for (int i = 0; i < aggregateResultList.size(); i++) { - if (Boolean.FALSE.equals(isCalculatedList.get(i))) { - AggregateResult aggregateResult = aggregateResultList.get(i); - aggregateResult.updateResultFromPageData(batchData); - batchData.resetBatchData(); - if (aggregateResult.isCalculatedAggregationResult()) { - isCalculatedList.set(i, true); - remainingToCalculate--; - if (remainingToCalculate == 0) { - return aggregateResultList; - } - } + } + } + } + return newRemainingToCalculate; + } + + private static int aggregateOverlappedPages(IAggregateReader seriesReader, + List aggregateResultList, boolean[] isCalculatedArray, int remainingToCalculate) + throws IOException { + // cal by page data + int newRemainingToCalculate = remainingToCalculate; + while (seriesReader.hasNextPage()) { + BatchData nextOverlappedPageData = seriesReader.nextPage(); + for (int i = 0; i < aggregateResultList.size(); i++) { + if (!isCalculatedArray[i]) { + AggregateResult aggregateResult = aggregateResultList.get(i); + aggregateResult.updateResultFromPageData(nextOverlappedPageData); + nextOverlappedPageData.resetBatchData(); + if (aggregateResult.isCalculatedAggregationResult()) { + isCalculatedArray[i] = true; + newRemainingToCalculate--; + if (newRemainingToCalculate == 0) { + return newRemainingToCalculate; } } } } } - return aggregateResultList; + return newRemainingToCalculate; } /** @@ -200,16 +227,15 @@ private List aggregateOneSeries( * * @param context query context. */ - public QueryDataSet executeWithValueFilter(QueryContext context) - throws StorageEngineException, PathException, IOException { + QueryDataSet executeWithValueFilter(QueryContext context) + throws StorageEngineException, IOException { - ServerTimeGenerator timestampGenerator = new ServerTimeGenerator(expression, context); + TimeGenerator timestampGenerator = getTimeGenerator(context); List readersOfSelectedSeries = new ArrayList<>(); for (int i = 0; i < selectedSeries.size(); i++) { Path path = selectedSeries.get(i); - SeriesReaderByTimestamp seriesReaderByTimestamp = new SeriesReaderByTimestamp(path, - dataTypes.get(i), context, - QueryResourceManager.getInstance().getQueryDataSource(path, context, null), null); + IReaderByTimestamp seriesReaderByTimestamp = getReaderByTime(path, + dataTypes.get(i), context); readersOfSelectedSeries.add(seriesReaderByTimestamp); } @@ -223,11 +249,22 @@ public QueryDataSet executeWithValueFilter(QueryContext context) return constructDataSet(aggregateResults); } + private TimeGenerator getTimeGenerator(QueryContext context) throws StorageEngineException { + return new ServerTimeGenerator(expression, context); + } + + private IReaderByTimestamp getReaderByTime(Path path, TSDataType dataType, + QueryContext context) throws StorageEngineException { + return new SeriesReaderByTimestamp(path, + dataType, context, + QueryResourceManager.getInstance().getQueryDataSource(path, context, null), null); + } + /** * calculate aggregation result with value filter. */ private void aggregateWithValueFilter(List aggregateResults, - ServerTimeGenerator timestampGenerator, List readersOfSelectedSeries) + TimeGenerator timestampGenerator, List readersOfSelectedSeries) throws IOException { while (timestampGenerator.hasNext()) { diff --git a/server/src/main/java/org/apache/iotdb/db/query/executor/QueryRouter.java b/server/src/main/java/org/apache/iotdb/db/query/executor/QueryRouter.java index 7d9c47d489f13..98079336e8e07 100644 --- a/server/src/main/java/org/apache/iotdb/db/query/executor/QueryRouter.java +++ b/server/src/main/java/org/apache/iotdb/db/query/executor/QueryRouter.java @@ -65,7 +65,7 @@ public QueryDataSet rawDataQuery(RawDataQueryPlan queryPlan, QueryContext contex } queryPlan.setExpression(optimizedExpression); - RawDataQueryExecutor rawDataQueryExecutor = new RawDataQueryExecutor(queryPlan); + RawDataQueryExecutor rawDataQueryExecutor = getRawDataQueryExecutor(queryPlan); if (!queryPlan.isAlignByTime()) { return rawDataQueryExecutor.executeNonAlign(context); @@ -79,6 +79,10 @@ public QueryDataSet rawDataQuery(RawDataQueryPlan queryPlan, QueryContext contex return rawDataQueryExecutor.executeWithoutValueFilter(context); } + protected RawDataQueryExecutor getRawDataQueryExecutor(RawDataQueryPlan queryPlan) { + return new RawDataQueryExecutor(queryPlan); + } + @Override public QueryDataSet aggregate(AggregationPlan aggregationPlan, QueryContext context) throws QueryFilterOptimizationException, StorageEngineException, QueryProcessException, @@ -94,7 +98,7 @@ public QueryDataSet aggregate(AggregationPlan aggregationPlan, QueryContext cont aggregationPlan.setExpression(optimizedExpression); - AggregationExecutor engineExecutor = new AggregationExecutor(aggregationPlan); + AggregationExecutor engineExecutor = getAggregationExecutor(aggregationPlan); if (optimizedExpression != null && optimizedExpression.getType() != ExpressionType.GLOBAL_TIME) { @@ -104,6 +108,9 @@ public QueryDataSet aggregate(AggregationPlan aggregationPlan, QueryContext cont return engineExecutor.executeWithoutValueFilter(context); } + protected AggregationExecutor getAggregationExecutor(AggregationPlan aggregationPlan) { + return new AggregationExecutor(aggregationPlan); + } @Override public QueryDataSet groupBy(GroupByPlan groupByPlan, QueryContext context) diff --git a/server/src/main/java/org/apache/iotdb/db/query/executor/RawDataQueryExecutor.java b/server/src/main/java/org/apache/iotdb/db/query/executor/RawDataQueryExecutor.java index 8785f540c93a3..4f9ea11ba49d9 100644 --- a/server/src/main/java/org/apache/iotdb/db/query/executor/RawDataQueryExecutor.java +++ b/server/src/main/java/org/apache/iotdb/db/query/executor/RawDataQueryExecutor.java @@ -25,8 +25,8 @@ import org.apache.iotdb.db.qp.physical.crud.RawDataQueryPlan; import org.apache.iotdb.db.query.context.QueryContext; import org.apache.iotdb.db.query.control.QueryResourceManager; -import org.apache.iotdb.db.query.dataset.RawQueryDataSetWithValueFilter; import org.apache.iotdb.db.query.dataset.NonAlignEngineDataSet; +import org.apache.iotdb.db.query.dataset.RawQueryDataSetWithValueFilter; import org.apache.iotdb.db.query.dataset.RawQueryDataSetWithoutValueFilter; import org.apache.iotdb.db.query.reader.series.IReaderByTimestamp; import org.apache.iotdb.db.query.reader.series.ManagedSeriesReader; @@ -39,15 +39,16 @@ import org.apache.iotdb.tsfile.read.expression.impl.GlobalTimeExpression; import org.apache.iotdb.tsfile.read.filter.basic.Filter; import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet; +import org.apache.iotdb.tsfile.read.query.timegenerator.TimeGenerator; /** * IoTDB query executor. */ public class RawDataQueryExecutor { - private List deduplicatedPaths; - private List deduplicatedDataTypes; - private IExpression optimizedExpression; + protected List deduplicatedPaths; + protected List deduplicatedDataTypes; + protected IExpression optimizedExpression; public RawDataQueryExecutor(RawDataQueryPlan queryPlan) { this.deduplicatedPaths = queryPlan.getDeduplicatedPaths(); @@ -77,7 +78,7 @@ public QueryDataSet executeNonAlign(QueryContext context) throws StorageEngineEx readersOfSelectedSeries); } - private List initManagedSeriesReader(QueryContext context) + protected List initManagedSeriesReader(QueryContext context) throws StorageEngineException { Filter timeFilter = null; if (optimizedExpression != null) { @@ -108,19 +109,30 @@ private List initManagedSeriesReader(QueryContext context) */ public QueryDataSet executeWithValueFilter(QueryContext context) throws StorageEngineException { - ServerTimeGenerator timestampGenerator = new ServerTimeGenerator( + TimeGenerator timestampGenerator = getTimeGenerator( optimizedExpression, context); List readersOfSelectedSeries = new ArrayList<>(); for (int i = 0; i < deduplicatedPaths.size(); i++) { Path path = deduplicatedPaths.get(i); - SeriesReaderByTimestamp seriesReaderByTimestamp = new SeriesReaderByTimestamp(path, - deduplicatedDataTypes.get(i), context, - QueryResourceManager.getInstance().getQueryDataSource(path, context, null), null); + IReaderByTimestamp seriesReaderByTimestamp = getReaderByTimestamp(path, + deduplicatedDataTypes.get(i), context); readersOfSelectedSeries.add(seriesReaderByTimestamp); } return new RawQueryDataSetWithValueFilter(deduplicatedPaths, deduplicatedDataTypes, timestampGenerator, readersOfSelectedSeries); } + protected IReaderByTimestamp getReaderByTimestamp(Path path, TSDataType dataType, + QueryContext context) + throws StorageEngineException { + return new SeriesReaderByTimestamp(path, + dataType, context, + QueryResourceManager.getInstance().getQueryDataSource(path, context, null), null); + } + + protected TimeGenerator getTimeGenerator(IExpression expression, + QueryContext context) throws StorageEngineException { + return new ServerTimeGenerator(expression, context); + } } diff --git a/server/src/main/java/org/apache/iotdb/db/query/externalsort/ExternalSortJob.java b/server/src/main/java/org/apache/iotdb/db/query/externalsort/ExternalSortJob.java index 4d91289b15907..bf05fba9945c2 100644 --- a/server/src/main/java/org/apache/iotdb/db/query/externalsort/ExternalSortJob.java +++ b/server/src/main/java/org/apache/iotdb/db/query/externalsort/ExternalSortJob.java @@ -26,7 +26,7 @@ /** * This class represents an external sort job. Every job will use a separated directory. */ -public class ExternalSortJob { +class ExternalSortJob { private List partList; @@ -41,5 +41,4 @@ List executeForIPointReader() throws IOException { } return readers; } - } diff --git a/server/src/main/java/org/apache/iotdb/db/query/externalsort/ExternalSortJobScheduler.java b/server/src/main/java/org/apache/iotdb/db/query/externalsort/ExternalSortJobScheduler.java index 1db8cd3d714a9..c1e70717991ab 100644 --- a/server/src/main/java/org/apache/iotdb/db/query/externalsort/ExternalSortJobScheduler.java +++ b/server/src/main/java/org/apache/iotdb/db/query/externalsort/ExternalSortJobScheduler.java @@ -16,29 +16,28 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iotdb.db.query.externalsort; + package org.apache.iotdb.db.query.externalsort; -import java.util.concurrent.atomic.AtomicLong; + public class ExternalSortJobScheduler { -public class ExternalSortJobScheduler { - - private AtomicLong jobId = new AtomicLong(0); + private long queryId = 0; private ExternalSortJobScheduler() { } - long genJobId() { - return jobId.getAndIncrement(); + synchronized long genJobId() { + queryId++; + return queryId; } private static class ExternalSortJobSchedulerHelper { - private static final ExternalSortJobScheduler INSTANCE = new ExternalSortJobScheduler(); + private static final ExternalSortJobScheduler INSTANCE = new ExternalSortJobScheduler(); } public static ExternalSortJobScheduler getInstance() { - return ExternalSortJobSchedulerHelper.INSTANCE; + return ExternalSortJobSchedulerHelper.INSTANCE; } -} + } diff --git a/server/src/main/java/org/apache/iotdb/db/query/externalsort/SimpleExternalSortEngine.java b/server/src/main/java/org/apache/iotdb/db/query/externalsort/SimpleExternalSortEngine.java index 5fba08ba09d2a..00097a8636122 100644 --- a/server/src/main/java/org/apache/iotdb/db/query/externalsort/SimpleExternalSortEngine.java +++ b/server/src/main/java/org/apache/iotdb/db/query/externalsort/SimpleExternalSortEngine.java @@ -103,10 +103,10 @@ public ExternalSortJob createJob(long queryId, List readerWrapL int toIndex = Math.min(i + minExternalSortSourceCount, ret.size()); List partGroup = ret.subList(i, toIndex); i = toIndex; - StringBuilder tmpFilePath = new StringBuilder(queryDir).append(jobId).append("_") - .append(partId); + String tmpFilePath = queryDir + jobId + "_" + + partId; MultiSourceExternalSortJobPart part = new MultiSourceExternalSortJobPart(queryId, - tmpFilePath.toString(), partGroup); + tmpFilePath, partGroup); tmpPartList.add(part); partId++; } @@ -115,19 +115,19 @@ public ExternalSortJob createJob(long queryId, List readerWrapL return new ExternalSortJob(ret); } - public String getQueryDir() { + String getQueryDir() { return queryDir; } - public void setQueryDir(String queryDir) { + void setQueryDir(String queryDir) { this.queryDir = queryDir; } - public int getMinExternalSortSourceCount() { + int getMinExternalSortSourceCount() { return minExternalSortSourceCount; } - public void setMinExternalSortSourceCount(int minExternalSortSourceCount) { + void setMinExternalSortSourceCount(int minExternalSortSourceCount) { this.minExternalSortSourceCount = minExternalSortSourceCount; } @@ -171,10 +171,10 @@ private List convert(List pointReaderList) { private static class SimpleExternalSortJobEngineHelper { - private static SimpleExternalSortEngine INSTANCE = new SimpleExternalSortEngine(); + private static final SimpleExternalSortEngine INSTANCE = new SimpleExternalSortEngine(); } public static SimpleExternalSortEngine getInstance() { return SimpleExternalSortJobEngineHelper.INSTANCE; } -} \ No newline at end of file +} diff --git a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReaderByTimestamp.java b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReaderByTimestamp.java index b15b72f1392ff..f9d43ac7e2d8d 100644 --- a/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReaderByTimestamp.java +++ b/server/src/main/java/org/apache/iotdb/db/query/reader/series/SeriesReaderByTimestamp.java @@ -40,6 +40,10 @@ public SeriesReaderByTimestamp(Path seriesPath, TSDataType dataType, QueryContex dataSource, TimeFilter.gtEq(Long.MIN_VALUE), null, fileFilter); } + public SeriesReaderByTimestamp(SeriesReader seriesReader) { + this.seriesReader = seriesReader; + } + @Override public Object[] getValuesInTimestamps(long[] timestamps) throws IOException { Object[] result = new Object[timestamps.length]; diff --git a/server/src/main/java/org/apache/iotdb/db/query/reader/universal/PriorityMergeReader.java b/server/src/main/java/org/apache/iotdb/db/query/reader/universal/PriorityMergeReader.java index c5a7ebaa8557b..4ebf7d1db4f6a 100644 --- a/server/src/main/java/org/apache/iotdb/db/query/reader/universal/PriorityMergeReader.java +++ b/server/src/main/java/org/apache/iotdb/db/query/reader/universal/PriorityMergeReader.java @@ -129,7 +129,7 @@ public void close() throws IOException { } } - class Element { + static class Element { IPointReader reader; TimeValuePair timeValuePair; diff --git a/server/src/main/java/org/apache/iotdb/db/query/timegenerator/ServerTimeGenerator.java b/server/src/main/java/org/apache/iotdb/db/query/timegenerator/ServerTimeGenerator.java index 28fcd4ec0f8f1..5695e3fc71e14 100644 --- a/server/src/main/java/org/apache/iotdb/db/query/timegenerator/ServerTimeGenerator.java +++ b/server/src/main/java/org/apache/iotdb/db/query/timegenerator/ServerTimeGenerator.java @@ -39,7 +39,11 @@ */ public class ServerTimeGenerator extends TimeGenerator { - private QueryContext context; + protected QueryContext context; + + public ServerTimeGenerator(QueryContext context) { + this.context = context; + } /** * Constructor of EngineTimeGenerator. @@ -50,7 +54,7 @@ public ServerTimeGenerator(IExpression expression, QueryContext context) try { super.constructNode(expression); } catch (IOException e) { - throw new StorageEngineException(e.getMessage()); + throw new StorageEngineException(e); } } diff --git a/server/src/main/java/org/apache/iotdb/db/service/IoTDB.java b/server/src/main/java/org/apache/iotdb/db/service/IoTDB.java index 0d2c4c5f0a9eb..072e2cb1cd8c0 100644 --- a/server/src/main/java/org/apache/iotdb/db/service/IoTDB.java +++ b/server/src/main/java/org/apache/iotdb/db/service/IoTDB.java @@ -96,7 +96,9 @@ private void setUp() throws StartupException { registerManager.register(UpgradeSevice.getINSTANCE()); registerManager.register(MergeManager.getINSTANCE()); registerManager.register(CacheHitRatioMonitor.getInstance()); - registerManager.register(MetricsService.getInstance()); + if (IoTDBDescriptor.getInstance().getConfig().isEnableMetricService()) { + registerManager.register(MetricsService.getInstance()); + } registerManager.register(StorageEngine.getInstance()); JMXService.registerMBean(getInstance(), mbeanName); diff --git a/server/src/main/java/org/apache/iotdb/db/service/MetricsService.java b/server/src/main/java/org/apache/iotdb/db/service/MetricsService.java index 69948864c5a34..90541fee807ce 100644 --- a/server/src/main/java/org/apache/iotdb/db/service/MetricsService.java +++ b/server/src/main/java/org/apache/iotdb/db/service/MetricsService.java @@ -80,7 +80,7 @@ public void stop() { @Override public synchronized void startService() throws StartupException { - if (!IoTDBDescriptor.getInstance().getConfig().isEnableMetricsWebService()) { + if (!IoTDBDescriptor.getInstance().getConfig().isEnableMetricService()) { return; } logger.info("{}: start {}...", IoTDBConstant.GLOBAL_DB_NAME, this.getID().getName()); diff --git a/server/src/main/java/org/apache/iotdb/db/service/ServiceType.java b/server/src/main/java/org/apache/iotdb/db/service/ServiceType.java index 670ea32c69b1e..0b0dea52fd1c1 100644 --- a/server/src/main/java/org/apache/iotdb/db/service/ServiceType.java +++ b/server/src/main/java/org/apache/iotdb/db/service/ServiceType.java @@ -43,7 +43,8 @@ public enum ServiceType { generateJmxName(IoTDBConstant.IOTDB_PACKAGE, "Cache Hit Ratio")), FLUSH_SERVICE("Flush ServerService", - generateJmxName("org.apache.iotdb.db.engine.pool", "Flush Manager")); + generateJmxName("org.apache.iotdb.db.engine.pool", "Flush Manager")), + CLUSTER_MONITOR_SERVICE("Cluster Monitor ServerService", "Cluster Monitor"); private String name; private String jmxName; diff --git a/server/src/main/java/org/apache/iotdb/db/service/StaticResps.java b/server/src/main/java/org/apache/iotdb/db/service/StaticResps.java index c6005511dde77..441a73dfbe6cd 100644 --- a/server/src/main/java/org/apache/iotdb/db/service/StaticResps.java +++ b/server/src/main/java/org/apache/iotdb/db/service/StaticResps.java @@ -118,11 +118,21 @@ private StaticResps() { Collections.singletonList(COLUMN_PRIVILEGE), Collections.singletonList(TSDataType.TEXT.toString())); + static final TSExecuteStatementResp LAST_RESP = getExecuteResp( + Arrays.asList(COLUMN_TIMESERIES, COLUMN_VALUE), + Arrays.asList(TSDataType.TEXT.toString(), TSDataType.TEXT.toString()), false + ); + private static TSExecuteStatementResp getNoTimeExecuteResp(List columns, List dataTypes) { + return getExecuteResp(columns, dataTypes, true); + } + + private static TSExecuteStatementResp getExecuteResp(List columns, + List dataTypes, boolean ignoreTimeStamp) { TSExecuteStatementResp resp = TSServiceImpl.getTSExecuteStatementResp(TSServiceImpl.getStatus(TSStatusCode.SUCCESS_STATUS)); - resp.setIgnoreTimeStamp(true); + resp.setIgnoreTimeStamp(ignoreTimeStamp); resp.setColumns(columns); resp.setDataTypeList(dataTypes); return resp; diff --git a/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java b/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java index 3386c40ac4f67..071181b82e672 100644 --- a/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java +++ b/server/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java @@ -18,14 +18,6 @@ */ package org.apache.iotdb.db.service; -import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_PRIVILEGE; -import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_ROLE; -import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_STORAGE_GROUP; -import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_TTL; -import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_USER; -import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_VALUE; -import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_TIMESERIES; - import java.io.IOException; import java.nio.ByteBuffer; import java.sql.SQLException; @@ -67,7 +59,12 @@ import org.apache.iotdb.db.qp.executor.PlanExecutor; import org.apache.iotdb.db.qp.logical.Operator.OperatorType; import org.apache.iotdb.db.qp.physical.PhysicalPlan; -import org.apache.iotdb.db.qp.physical.crud.*; +import org.apache.iotdb.db.qp.physical.crud.AlignByDevicePlan; +import org.apache.iotdb.db.qp.physical.crud.BatchInsertPlan; +import org.apache.iotdb.db.qp.physical.crud.DeletePlan; +import org.apache.iotdb.db.qp.physical.crud.InsertPlan; +import org.apache.iotdb.db.qp.physical.crud.LastQueryPlan; +import org.apache.iotdb.db.qp.physical.crud.QueryPlan; import org.apache.iotdb.db.qp.physical.sys.AuthorPlan; import org.apache.iotdb.db.qp.physical.sys.CreateTimeSeriesPlan; import org.apache.iotdb.db.qp.physical.sys.DeleteStorageGroupPlan; @@ -81,6 +78,7 @@ import org.apache.iotdb.db.tools.watermark.GroupedLSBWatermarkEncoder; import org.apache.iotdb.db.tools.watermark.WatermarkEncoder; import org.apache.iotdb.db.utils.QueryDataSetUtils; +import org.apache.iotdb.db.utils.SchemaUtils; import org.apache.iotdb.rpc.TSStatusCode; import org.apache.iotdb.service.rpc.thrift.ServerProperties; import org.apache.iotdb.service.rpc.thrift.TSBatchInsertionReq; @@ -137,8 +135,8 @@ public class TSServiceImpl implements TSIService.Iface, ServerContext { "meet error while parsing SQL to physical plan: {}"; public static Vector sqlArgumentsList = new Vector<>(); - private Planner processor; - private IPlanExecutor executor; + protected Planner processor; + protected IPlanExecutor executor; // Record the username for every rpc connection (session). private Map sessionIdUsernameMap = new ConcurrentHashMap<>(); @@ -167,52 +165,6 @@ public TSServiceImpl() throws QueryProcessException { executor = new PlanExecutor(); } - public static TSDataType getSeriesType(String path) throws QueryProcessException { - switch (path.toLowerCase()) { - // authorization queries - case COLUMN_ROLE: - case COLUMN_USER: - case COLUMN_PRIVILEGE: - case COLUMN_STORAGE_GROUP: - return TSDataType.TEXT; - case COLUMN_TTL: - return TSDataType.INT64; - default: - // do nothing - } - - if (path.contains("(") && !path.startsWith("(") && path.endsWith(")")) { - // aggregation - int leftBracketIndex = path.indexOf('('); - String aggrType = path.substring(0, leftBracketIndex); - String innerPath = path.substring(leftBracketIndex + 1, path.length() - 1); - switch (aggrType.toLowerCase()) { - case SQLConstant.MIN_TIME: - case SQLConstant.MAX_TIME: - case SQLConstant.COUNT: - return TSDataType.INT64; - case SQLConstant.LAST_VALUE: - case SQLConstant.FIRST_VALUE: - case SQLConstant.MIN_VALUE: - case SQLConstant.MAX_VALUE: - return getSeriesType(innerPath); - case SQLConstant.AVG: - case SQLConstant.SUM: - return TSDataType.DOUBLE; - default: - throw new QueryProcessException("aggregate does not support " + aggrType + " function."); - } - } - TSDataType dataType; - try { - dataType = MManager.getInstance().getSeriesType(path); - } catch (MetadataException e) { - throw new QueryProcessException(e); - } - - return dataType; - } - @Override public TSOpenSessionResp openSession(TSOpenSessionReq req) throws TException { logger.info( @@ -349,7 +301,7 @@ public TSStatus closeOperation(TSCloseOperationReq req) { /** * release single operation resource */ - private void releaseQueryResource(long queryId) throws StorageEngineException { + protected void releaseQueryResource(long queryId) throws StorageEngineException { // remove the corresponding Physical Plan queryId2DataSet.remove(queryId); QueryResourceManager.getInstance().endQuery(queryId); @@ -376,6 +328,14 @@ private TSStatus getStatus(TSStatusCode statusType, String appendMessage) { return new TSStatus(statusCodeAndMessage); } + protected TSDataType getSeriesType(String path) throws QueryProcessException, MetadataException { + try { + return SchemaUtils.getSeriesType(path); + } catch (MetadataException e) { + throw new QueryProcessException(e); + } + } + @Override public TSFetchMetadataResp fetchMetadata(TSFetchMetadataReq req) { TSStatus status; @@ -405,7 +365,7 @@ public TSFetchMetadataResp fetchMetadata(TSFetchMetadataReq req) { status = getStatus(TSStatusCode.METADATA_ERROR, req.getType()); break; } - } catch (QueryProcessException | MetadataException | OutOfMemoryError e) { + } catch (MetadataException | OutOfMemoryError | QueryProcessException e) { logger.error( String.format("Failed to fetch timeseries %s's metadata", req.getColumnPath()), e); status = getStatus(TSStatusCode.METADATA_ERROR, e.getMessage()); @@ -620,14 +580,8 @@ private TSExecuteStatementResp internalExecuteQueryStatement( long statementId, PhysicalPlan plan, int fetchSize, String username) { long t1 = System.currentTimeMillis(); try { - TSExecuteStatementResp resp; // column headers - if (plan instanceof AuthorPlan) { - resp = getAuthQueryColumnHeaders(plan); - } else if (plan instanceof ShowPlan) { - resp = getShowQueryColumnHeaders((ShowPlan) plan); - } else { - resp = getQueryColumnHeaders(plan, username); - } + TSExecuteStatementResp resp = getQueryResp(plan, username); // column headers + if (plan instanceof QueryPlan && !((QueryPlan) plan).isAlignByTime()) { if (plan.getOperatorType() == OperatorType.AGGREGATION) { throw new QueryProcessException("Aggregation doesn't support disable align clause."); @@ -654,12 +608,11 @@ private TSExecuteStatementResp internalExecuteQueryStatement( if (plan instanceof QueryPlan && !((QueryPlan) plan).isAlignByTime()) { TSQueryNonAlignDataSet result = fillRpcNonAlignReturnData(fetchSize, newDataSet, username); resp.setNonAlignQueryDataSet(result); - resp.setQueryId(queryId); } else { TSQueryDataSet result = fillRpcReturnData(fetchSize, newDataSet, username); resp.setQueryDataSet(result); - resp.setQueryId(queryId); } + resp.setQueryId(queryId); return resp; } catch (Exception e) { logger.error("{}: Internal server error: ", IoTDBConstant.GLOBAL_DB_NAME, e); @@ -670,6 +623,17 @@ private TSExecuteStatementResp internalExecuteQueryStatement( } } + private TSExecuteStatementResp getQueryResp(PhysicalPlan plan, String username) + throws QueryProcessException, AuthException, TException { + if (plan instanceof AuthorPlan) { + return getAuthQueryColumnHeaders(plan); + } else if (plan instanceof ShowPlan) { + return getShowQueryColumnHeaders((ShowPlan) plan); + } else { + return getQueryColumnHeaders(plan, username); + } + } + @Override public TSExecuteStatementResp executeQueryStatement(TSExecuteStatementReq req) { long startTime = System.currentTimeMillis(); @@ -784,7 +748,7 @@ private TSExecuteStatementResp getQueryColumnHeaders(PhysicalPlan physicalPlan, if (plan instanceof AlignByDevicePlan) { getAlignByDeviceQueryHeaders((AlignByDevicePlan) plan, respColumns, columnsTypes); } else if (plan instanceof LastQueryPlan) { - getLastQueryHeaders(plan, respColumns, columnsTypes); + return StaticResps.LAST_RESP; } else { getWideQueryHeaders(plan, respColumns, columnsTypes); } @@ -824,7 +788,11 @@ private void getWideQueryHeaders( } for (String column : respColumns) { - columnTypes.add(getSeriesType(column).toString()); + try { + columnTypes.add(getSeriesType(column).toString()); + } catch (MetadataException e) { + throw new QueryProcessException(e); + } } } @@ -856,11 +824,11 @@ private void getAlignByDeviceQueryHeaders( while (loc < totalSize) { boolean isNonExist = false; boolean isConstant = false; - TSDataType type = null; - String column = null; + TSDataType type; + String column; // not exist - if (notExistMeasurementsLoc < plan.getNotExistMeasurements().size() - && loc == plan.getPositionOfNotExistMeasurements().get(notExistMeasurementsLoc)) { + if (isOneMeasurementIn(loc, + notExistMeasurementsLoc, plan.getPositionOfNotExistMeasurements())) { // for shifting plan.getPositionOfNotExistMeasurements().set(notExistMeasurementsLoc, loc - shiftLoc); @@ -870,8 +838,8 @@ private void getAlignByDeviceQueryHeaders( isNonExist = true; } // constant - else if (constMeasurementsLoc < plan.getConstMeasurements().size() - && loc == plan.getPositionOfConstMeasurements().get(constMeasurementsLoc)) { + else if (isOneMeasurementIn(loc, + constMeasurementsLoc, plan.getPositionOfConstMeasurements())) { // for shifting plan.getPositionOfConstMeasurements().set(constMeasurementsLoc, loc - shiftLoc); @@ -926,13 +894,17 @@ else if (constMeasurementsLoc < plan.getConstMeasurements().size() plan.setDataTypeConsistencyChecker(null); } - private void getLastQueryHeaders( - QueryPlan plan, List respColumns, List columnTypes) - throws TException, QueryProcessException { - respColumns.add(COLUMN_TIMESERIES); - respColumns.add(COLUMN_VALUE); - columnTypes.add(TSDataType.TEXT.toString()); - columnTypes.add(TSDataType.TEXT.toString()); + /** + * + * @param subLoc + * @param totalLoc + * @param measurementPositions + * @return true if the measurement at totalLoc is the subLoc measurement in measurementPositions, + * false otherwise + */ + private boolean isOneMeasurementIn(int totalLoc, + int subLoc, List measurementPositions) { + return subLoc < measurementPositions.size() && totalLoc == measurementPositions.get(subLoc); } @Override @@ -1066,12 +1038,16 @@ private QueryDataSet createQueryDataSet(long queryId, PhysicalPlan physicalPlan) throws QueryProcessException, QueryFilterOptimizationException, StorageEngineException, IOException, MetadataException, SQLException { - QueryContext context = new QueryContext(queryId); + QueryContext context = genQueryContext(queryId); QueryDataSet queryDataSet = executor.processQuery(physicalPlan, context); queryId2DataSet.put(queryId, queryDataSet); return queryDataSet; } + protected QueryContext genQueryContext(long queryId) { + return new QueryContext(queryId); + } + @Override public TSExecuteStatementResp executeUpdateStatement(TSExecuteStatementReq req) { try { @@ -1168,7 +1144,7 @@ private TSFetchResultsResp getTSFetchResultsResp(TSStatus status) { return resp; } - void handleClientExit() { + protected void handleClientExit() { Long sessionId = currSessionId.get(); if (sessionId != null) { TSCloseSessionReq req = new TSCloseSessionReq(sessionId); @@ -1454,7 +1430,7 @@ private TSStatus checkAuthority(PhysicalPlan plan, long sessionId) { return null; } - private TSStatus executePlan(PhysicalPlan plan) { + protected TSStatus executePlan(PhysicalPlan plan) { boolean execRet; try { execRet = executeNonQuery(plan); diff --git a/server/src/main/java/org/apache/iotdb/db/tools/IoTDBDataDirViewer.java b/server/src/main/java/org/apache/iotdb/db/tools/IoTDBDataDirViewer.java index 780603fbfdb50..262de5f8c04c4 100644 --- a/server/src/main/java/org/apache/iotdb/db/tools/IoTDBDataDirViewer.java +++ b/server/src/main/java/org/apache/iotdb/db/tools/IoTDBDataDirViewer.java @@ -118,7 +118,7 @@ private static void printFilesInStorageGroupDir(File storageGroup, PrintWriter p private static void printResource(String filename, PrintWriter pw) throws IOException { filename = filename.substring(0, filename.length() - 9); TsFileResource resource = new TsFileResource(SystemFileFactory.INSTANCE.getFile(filename)); - resource.deSerialize(); + resource.deserialize(); // sort device strings SortedSet keys = new TreeSet<>(resource.getStartTimeMap().keySet()); for (String device : keys) { diff --git a/server/src/main/java/org/apache/iotdb/db/tools/TsFileResourcePrinter.java b/server/src/main/java/org/apache/iotdb/db/tools/TsFileResourcePrinter.java index d52a220d7bc79..03e97d14d9aab 100644 --- a/server/src/main/java/org/apache/iotdb/db/tools/TsFileResourcePrinter.java +++ b/server/src/main/java/org/apache/iotdb/db/tools/TsFileResourcePrinter.java @@ -54,7 +54,7 @@ public static void printResource(String filename) throws IOException { filename = filename.substring(0, filename.length() - 9); TsFileResource resource = new TsFileResource(SystemFileFactory.INSTANCE.getFile(filename)); System.err.println(String.format("analyzing %s ...", filename)); - resource.deSerialize(); + resource.deserialize(); System.out.println("historicalVersions: " + resource.getHistoricalVersions()); diff --git a/server/src/main/java/org/apache/iotdb/db/utils/FileLoaderUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/FileLoaderUtils.java index 7b93c944d76f0..765c17af1169e 100644 --- a/server/src/main/java/org/apache/iotdb/db/utils/FileLoaderUtils.java +++ b/server/src/main/java/org/apache/iotdb/db/utils/FileLoaderUtils.java @@ -53,7 +53,7 @@ public static void checkTsFileResource(TsFileResource tsFileResource) throws IOE // write .resource file tsFileResource.serialize(); } else { - tsFileResource.deSerialize(); + tsFileResource.deserialize(); } } diff --git a/server/src/main/java/org/apache/iotdb/db/utils/SchemaUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/SchemaUtils.java index c18ce5856522e..8ae97d88c4e45 100644 --- a/server/src/main/java/org/apache/iotdb/db/utils/SchemaUtils.java +++ b/server/src/main/java/org/apache/iotdb/db/utils/SchemaUtils.java @@ -18,15 +18,34 @@ */ package org.apache.iotdb.db.utils; +import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_PRIVILEGE; +import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_ROLE; +import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_STORAGE_GROUP; +import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_TTL; +import static org.apache.iotdb.db.conf.IoTDBConstant.COLUMN_USER; + +import java.util.Collections; import java.util.List; +import org.apache.iotdb.db.engine.StorageEngine; +import org.apache.iotdb.db.exception.StorageEngineException; import org.apache.iotdb.db.exception.metadata.MetadataException; +import org.apache.iotdb.db.exception.metadata.PathAlreadyExistException; import org.apache.iotdb.db.metadata.MManager; +import org.apache.iotdb.db.qp.constant.SQLConstant; +import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType; +import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; +import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; +import org.apache.iotdb.tsfile.read.common.Path; import org.apache.iotdb.tsfile.write.schema.MeasurementSchema; import org.apache.iotdb.tsfile.write.schema.Schema; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class SchemaUtils { + private static final Logger logger = LoggerFactory.getLogger(SchemaUtils.class); + private SchemaUtils() { } @@ -48,11 +67,75 @@ public static Schema constructSchema(String processorName) throws MetadataExcept * @param schemaList the schema of the columns in this file. * @return a Schema contains the provided schemas. */ - public static Schema getSchemaFromColumnSchema(List schemaList) { + private static Schema getSchemaFromColumnSchema(List schemaList) { Schema schema = new Schema(); for (MeasurementSchema measurementSchema : schemaList) { schema.registerMeasurement(measurementSchema); } return schema; } + + public static void registerTimeseries(MeasurementSchema schema) { + try { + logger.debug("Registering timeseries {}", schema); + String path = schema.getMeasurementId(); + TSDataType dataType = schema.getType(); + TSEncoding encoding = schema.getEncodingType(); + CompressionType compressionType = schema.getCompressor(); + boolean result = MManager.getInstance().createTimeseries(path, dataType, encoding, + compressionType, Collections.emptyMap()); + if (result) { + StorageEngine.getInstance().addTimeSeries(new Path(path), dataType, encoding, + compressionType, Collections.emptyMap()); + } + } catch (PathAlreadyExistException ignored) { + // ignore added timeseries + } catch (MetadataException | StorageEngineException e) { + logger.error("Cannot create timeseries {} in snapshot, ignored", schema.getMeasurementId(), + e); + } + + } + + public static TSDataType getSeriesType(String path) + throws MetadataException { + switch (path.toLowerCase()) { + // authorization queries + case COLUMN_ROLE: + case COLUMN_USER: + case COLUMN_PRIVILEGE: + case COLUMN_STORAGE_GROUP: + return TSDataType.TEXT; + case SQLConstant.RESERVED_TIME: + case COLUMN_TTL: + return TSDataType.INT64; + default: + // do nothing + } + + if (path.contains("(") && !path.startsWith("(") && path.endsWith(")")) { + // aggregation + int leftBracketIndex = path.indexOf('('); + String aggrType = path.substring(0, leftBracketIndex); + String innerPath = path.substring(leftBracketIndex + 1, path.length() - 1); + switch (aggrType.toLowerCase()) { + case SQLConstant.MIN_TIME: + case SQLConstant.MAX_TIME: + case SQLConstant.COUNT: + return TSDataType.INT64; + case SQLConstant.LAST_VALUE: + case SQLConstant.FIRST_VALUE: + case SQLConstant.MIN_VALUE: + case SQLConstant.MAX_VALUE: + return getSeriesType(innerPath); + case SQLConstant.AVG: + case SQLConstant.SUM: + return TSDataType.DOUBLE; + default: + throw new MetadataException( + "aggregate does not support " + aggrType + " function."); + } + } + return MManager.getInstance().getSeriesType(path); + } } diff --git a/server/src/main/java/org/apache/iotdb/db/utils/TestOnly.java b/server/src/main/java/org/apache/iotdb/db/utils/TestOnly.java index d4058bad20500..fa1acc22019ad 100644 --- a/server/src/main/java/org/apache/iotdb/db/utils/TestOnly.java +++ b/server/src/main/java/org/apache/iotdb/db/utils/TestOnly.java @@ -27,5 +27,9 @@ @Target({ElementType.METHOD, ElementType.CONSTRUCTOR}) @Retention(RetentionPolicy.SOURCE) +/** + * TestOnly implies that the method should only be used in the tests, otherwise its functionality + * is not guaranteed and may interfere with the normal code. + */ public @interface TestOnly { } diff --git a/server/src/main/java/org/apache/iotdb/db/writelog/recover/TsFileRecoverPerformer.java b/server/src/main/java/org/apache/iotdb/db/writelog/recover/TsFileRecoverPerformer.java index 941b494566a44..bd0c2d6c4299c 100644 --- a/server/src/main/java/org/apache/iotdb/db/writelog/recover/TsFileRecoverPerformer.java +++ b/server/src/main/java/org/apache/iotdb/db/writelog/recover/TsFileRecoverPerformer.java @@ -150,7 +150,7 @@ public RestorableTsFileIOWriter recover() throws StorageGroupProcessorException private void recoverResourceFromFile() throws IOException { try { - tsFileResource.deSerialize(); + tsFileResource.deserialize(); } catch (IOException e) { logger.warn("Cannot deserialize TsFileResource {}, construct it using " + "TsFileSequenceReader", tsFileResource.getFile(), e); diff --git a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeLogTest.java b/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeLogTest.java index cf4bd93496019..4ad099171af3e 100644 --- a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeLogTest.java +++ b/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeLogTest.java @@ -33,7 +33,6 @@ import org.apache.iotdb.db.engine.merge.task.MergeTask; import org.apache.iotdb.db.engine.storagegroup.TsFileResource; import org.apache.iotdb.db.exception.metadata.MetadataException; -import org.apache.iotdb.db.exception.query.PathException; import org.apache.iotdb.db.exception.StorageEngineException; import org.apache.iotdb.tsfile.exception.write.WriteProcessException; import org.junit.After; @@ -45,7 +44,7 @@ public class MergeLogTest extends MergeTest { File tempSGDir; @Before - public void setUp() throws IOException, WriteProcessException, MetadataException, PathException { + public void setUp() throws IOException, WriteProcessException, MetadataException { super.setUp(); tempSGDir = new File(TestConstant.BASE_OUTPUT_PATH.concat("tempSG")); tempSGDir.mkdirs(); diff --git a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeOverLapTest.java b/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeOverLapTest.java index a4b5e7c0cbff7..656a2155fea69 100644 --- a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeOverLapTest.java +++ b/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeOverLapTest.java @@ -26,9 +26,8 @@ import org.apache.iotdb.db.engine.merge.manage.MergeResource; import org.apache.iotdb.db.engine.merge.task.MergeTask; import org.apache.iotdb.db.engine.storagegroup.TsFileResource; -import org.apache.iotdb.db.exception.StorageEngineException; import org.apache.iotdb.db.exception.metadata.MetadataException; -import org.apache.iotdb.db.exception.query.PathException; +import org.apache.iotdb.db.exception.StorageEngineException; import org.apache.iotdb.db.query.context.QueryContext; import org.apache.iotdb.db.query.reader.series.SeriesRawDataBatchReader; import org.apache.iotdb.tsfile.exception.write.WriteProcessException; @@ -57,7 +56,7 @@ public class MergeOverLapTest extends MergeTest { @Before public void setUp() - throws IOException, WriteProcessException, MetadataException, PathException { + throws IOException, WriteProcessException, MetadataException { ptNum = 1000; super.setUp(); tempSGDir = new File(TestConstant.BASE_OUTPUT_PATH.concat("tempSG")); diff --git a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeTaskTest.java b/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeTaskTest.java index e333c66254060..7b61725fd5ab6 100644 --- a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeTaskTest.java +++ b/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeTaskTest.java @@ -25,10 +25,9 @@ import org.apache.iotdb.db.engine.merge.manage.MergeResource; import org.apache.iotdb.db.engine.merge.task.MergeTask; import org.apache.iotdb.db.engine.modification.Deletion; +import org.apache.iotdb.db.exception.metadata.MetadataException; import org.apache.iotdb.db.engine.storagegroup.TsFileResource; import org.apache.iotdb.db.exception.StorageEngineException; -import org.apache.iotdb.db.exception.metadata.MetadataException; -import org.apache.iotdb.db.exception.query.PathException; import org.apache.iotdb.db.query.context.QueryContext; import org.apache.iotdb.db.query.reader.series.SeriesRawDataBatchReader; import org.apache.iotdb.tsfile.exception.write.WriteProcessException; @@ -51,7 +50,7 @@ public class MergeTaskTest extends MergeTest { private File tempSGDir; @Before - public void setUp() throws IOException, WriteProcessException, MetadataException, PathException { + public void setUp() throws IOException, WriteProcessException, MetadataException, MetadataException { super.setUp(); tempSGDir = new File(TestConstant.BASE_OUTPUT_PATH.concat("tempSG")); tempSGDir.mkdirs(); diff --git a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeTest.java b/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeTest.java index 21a468314a5f0..028f0d0fdcddd 100644 --- a/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeTest.java +++ b/server/src/test/java/org/apache/iotdb/db/engine/merge/MergeTest.java @@ -34,7 +34,6 @@ import org.apache.iotdb.db.engine.merge.manage.MergeManager; import org.apache.iotdb.db.engine.storagegroup.TsFileResource; import org.apache.iotdb.db.exception.metadata.MetadataException; -import org.apache.iotdb.db.exception.query.PathException; import org.apache.iotdb.db.exception.StorageEngineException; import org.apache.iotdb.db.metadata.MManager; import org.apache.iotdb.db.query.control.FileReaderManager; @@ -71,7 +70,7 @@ abstract class MergeTest { private int prevMergeChunkThreshold; @Before - public void setUp() throws IOException, WriteProcessException, MetadataException, PathException { + public void setUp() throws IOException, WriteProcessException, MetadataException, MetadataException { MManager.getInstance().init(); prevMergeChunkThreshold = IoTDBDescriptor.getInstance().getConfig().getChunkMergePointThreshold(); @@ -94,7 +93,7 @@ public void tearDown() throws IOException, StorageEngineException { MergeManager.getINSTANCE().stop(); } - private void prepareSeries() throws MetadataException, PathException { + private void prepareSeries() throws MetadataException, MetadataException { measurementSchemas = new MeasurementSchema[measurementNum]; for (int i = 0; i < measurementNum; i++) { measurementSchemas[i] = new MeasurementSchema("sensor" + i, TSDataType.DOUBLE, diff --git a/server/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java b/server/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java index 5be94eaf58974..1ad0d2229cef9 100644 --- a/server/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java +++ b/server/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java @@ -35,19 +35,16 @@ import org.apache.iotdb.db.engine.StorageEngine; import org.apache.iotdb.db.engine.modification.io.LocalTextModificationAccessor; import org.apache.iotdb.db.engine.querycontext.QueryDataSource; -import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk; +import org.apache.iotdb.db.exception.metadata.MetadataException; import org.apache.iotdb.db.exception.StartupException; import org.apache.iotdb.db.exception.StorageEngineException; -import org.apache.iotdb.db.exception.metadata.MetadataException; -import org.apache.iotdb.db.exception.query.PathException; +import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk; import org.apache.iotdb.db.exception.query.QueryProcessException; -import org.apache.iotdb.db.exception.storageGroup.StorageGroupException; import org.apache.iotdb.db.metadata.MManager; import org.apache.iotdb.db.qp.physical.crud.InsertPlan; import org.apache.iotdb.db.query.control.QueryResourceManager; import org.apache.iotdb.db.utils.EnvironmentUtils; import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor; -import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType; import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; import org.apache.iotdb.tsfile.read.reader.IPointReader; @@ -232,7 +229,7 @@ public void testDeleteInOverflowCache() @Test public void testDeleteInOverflowFile() - throws StorageEngineException, QueryProcessException, IOException { + throws StorageEngineException, QueryProcessException { // insert into BufferWrite for (int i = 101; i <= 200; i++) { TSRecord record = new TSRecord(i, processorName); diff --git a/server/src/test/java/org/apache/iotdb/db/engine/modification/DeletionQueryTest.java b/server/src/test/java/org/apache/iotdb/db/engine/modification/DeletionQueryTest.java index 9e4f16a5de8d7..ce79cebbd5f59 100644 --- a/server/src/test/java/org/apache/iotdb/db/engine/modification/DeletionQueryTest.java +++ b/server/src/test/java/org/apache/iotdb/db/engine/modification/DeletionQueryTest.java @@ -30,7 +30,6 @@ import org.apache.iotdb.db.exception.StartupException; import org.apache.iotdb.db.exception.StorageEngineException; import org.apache.iotdb.db.exception.metadata.MetadataException; -import org.apache.iotdb.db.exception.query.PathException; import org.apache.iotdb.db.exception.query.QueryProcessException; import org.apache.iotdb.db.exception.storageGroup.StorageGroupException; import org.apache.iotdb.db.metadata.MManager; @@ -39,7 +38,6 @@ import org.apache.iotdb.db.query.executor.QueryRouter; import org.apache.iotdb.db.utils.EnvironmentUtils; import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor; -import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType; import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; import org.apache.iotdb.tsfile.read.common.Path; @@ -66,8 +64,8 @@ public class DeletionQueryTest { } @Before - public void setup() throws MetadataException, - PathException, IOException, StorageEngineException, StartupException, StorageGroupException { + public void setup() throws + MetadataException, StorageEngineException, StartupException { EnvironmentUtils.envSetUp(); MManager.getInstance().setStorageGroup(processorName); diff --git a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/FileNodeManagerBenchmark.java b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/FileNodeManagerBenchmark.java index 7fea61d522c38..fc8d93aa8853f 100644 --- a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/FileNodeManagerBenchmark.java +++ b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/FileNodeManagerBenchmark.java @@ -24,7 +24,6 @@ import org.apache.iotdb.db.engine.StorageEngine; import org.apache.iotdb.db.exception.metadata.MetadataException; import org.apache.iotdb.db.exception.StorageEngineException; -import org.apache.iotdb.db.exception.query.PathException; import org.apache.iotdb.db.exception.query.QueryProcessException; import org.apache.iotdb.db.exception.storageGroup.StorageGroupException; import org.apache.iotdb.db.metadata.MManager; @@ -65,7 +64,7 @@ public class FileNodeManagerBenchmark { } private static void prepare() - throws MetadataException, PathException, IOException, StorageGroupException { + throws MetadataException { MManager manager = MManager.getInstance(); manager.setStorageGroup(prefix); for (String device : devices) { @@ -81,8 +80,8 @@ private static void tearDown() throws IOException, StorageEngineException { } public static void main(String[] args) - throws InterruptedException, IOException, MetadataException, - PathException, StorageEngineException, StorageGroupException { + throws InterruptedException, IOException, + MetadataException, StorageEngineException { tearDown(); prepare(); long startTime = System.currentTimeMillis(); diff --git a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TTLTest.java b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TTLTest.java index 832bde43ae31f..042a9fbf31c69 100644 --- a/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TTLTest.java +++ b/server/src/test/java/org/apache/iotdb/db/engine/storagegroup/TTLTest.java @@ -38,9 +38,7 @@ import org.apache.iotdb.db.exception.StorageEngineException; import org.apache.iotdb.db.exception.metadata.MetadataException; import org.apache.iotdb.db.exception.query.OutOfTTLException; -import org.apache.iotdb.db.exception.query.PathException; import org.apache.iotdb.db.exception.query.QueryProcessException; -import org.apache.iotdb.db.exception.storageGroup.StorageGroupException; import org.apache.iotdb.db.exception.storageGroup.StorageGroupProcessorException; import org.apache.iotdb.db.metadata.MManager; import org.apache.iotdb.db.metadata.mnode.StorageGroupMNode; @@ -77,7 +75,7 @@ public class TTLTest { @Before public void setUp() - throws MetadataException, IOException, StartupException, PathException, StorageGroupProcessorException { + throws MetadataException, IOException, StartupException, StorageGroupProcessorException { IoTDBDescriptor.getInstance().getConfig().setPartitionInterval(86400); EnvironmentUtils.envSetUp(); createSchemas(); diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiSeriesIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiSeriesIT.java index d0500f24c1ca9..a10d9f3c34536 100644 --- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiSeriesIT.java +++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiSeriesIT.java @@ -431,7 +431,7 @@ public void selectWhereUnknownTimeSeriesFromRoot() throws ClassNotFoundException } catch (SQLException e) { e.printStackTrace(); assertEquals( - "401: Statement format is not right: Path [root.vehicle.d0.s10] does not exist", + "401: Statement format is not right: org.apache.iotdb.db.exception.metadata.PathNotExistException: Path [root.vehicle.d0.s10] does not exist", e.getMessage()); } } diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSequenceDataQueryIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSequenceDataQueryIT.java index 7bcafa4e0bbc9..da1c01160bf15 100644 --- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSequenceDataQueryIT.java +++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSequenceDataQueryIT.java @@ -26,7 +26,6 @@ import java.io.IOException; import java.sql.Connection; import java.sql.DriverManager; -import java.sql.SQLException; import java.sql.Statement; import java.util.ArrayList; import java.util.List; @@ -99,7 +98,7 @@ public static void tearDown() throws Exception { EnvironmentUtils.cleanEnv(); } - private static void insertData() throws ClassNotFoundException, SQLException { + private static void insertData() throws ClassNotFoundException { Class.forName(Config.JDBC_DRIVER_NAME); try (Connection connection = DriverManager .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root"); @@ -134,7 +133,7 @@ private static void insertData() throws ClassNotFoundException, SQLException { // insert data (time from 1200-1499) for (long time = 1200; time < 1500; time++) { - String sql = null; + String sql; if (time % 2 == 0) { sql = String .format("insert into root.vehicle.d0(timestamp,s0) values(%s,%s)", time, time % 17); diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSeriesReaderIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSeriesReaderIT.java index 4163e8ae1b26d..c2cbd1523b631 100644 --- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSeriesReaderIT.java +++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBSeriesReaderIT.java @@ -105,7 +105,7 @@ public static void tearDown() throws Exception { EnvironmentUtils.cleanEnv(); } - private static void insertData() throws ClassNotFoundException, SQLException { + private static void insertData() throws ClassNotFoundException { Class.forName(Config.JDBC_DRIVER_NAME); try (Connection connection = DriverManager .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root"); diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTtlIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTtlIT.java index 341627aba2c0d..4fd5f133c0e52 100644 --- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTtlIT.java +++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBTtlIT.java @@ -142,7 +142,6 @@ public void testShowTTL() throws SQLException { Statement statement = connection.createStatement()) { statement.execute("SET STORAGE GROUP TO root.group1"); statement.execute("SET STORAGE GROUP TO root.group2"); - String result = doQuery(statement, "SHOW ALL TTL"); assertEquals("root.group1,null\n" + "root.group2,null\n", result); diff --git a/server/src/test/java/org/apache/iotdb/db/qp/plan/ConcatOptimizerTest.java b/server/src/test/java/org/apache/iotdb/db/qp/plan/ConcatOptimizerTest.java index f6d6b67bb2743..015a094a1cbf4 100644 --- a/server/src/test/java/org/apache/iotdb/db/qp/plan/ConcatOptimizerTest.java +++ b/server/src/test/java/org/apache/iotdb/db/qp/plan/ConcatOptimizerTest.java @@ -23,7 +23,6 @@ import java.io.IOException; import org.antlr.v4.runtime.RecognitionException; import org.apache.iotdb.db.exception.metadata.MetadataException; -import org.apache.iotdb.db.exception.query.PathException; import org.apache.iotdb.db.exception.query.QueryProcessException; import org.apache.iotdb.db.metadata.MManager; import org.apache.iotdb.db.qp.Planner; @@ -49,7 +48,7 @@ public class ConcatOptimizerTest { private Planner processor; @Before - public void before() throws MetadataException, PathException { + public void before() throws MetadataException { processor = new Planner(); MManager.getInstance().init(); MManager.getInstance().setStorageGroup("root.laptop"); diff --git a/server/src/test/java/org/apache/iotdb/db/qp/plan/PhysicalPlanTest.java b/server/src/test/java/org/apache/iotdb/db/qp/plan/PhysicalPlanTest.java index 15276852e9cde..1805579506f4c 100644 --- a/server/src/test/java/org/apache/iotdb/db/qp/plan/PhysicalPlanTest.java +++ b/server/src/test/java/org/apache/iotdb/db/qp/plan/PhysicalPlanTest.java @@ -93,8 +93,7 @@ public void testMetadata() throws QueryProcessException { String metadata = "create timeseries root.vehicle.d1.s2 with datatype=INT32,encoding=RLE"; Planner processor = new Planner(); CreateTimeSeriesPlan plan = (CreateTimeSeriesPlan) processor.parseSQLToPhysicalPlan(metadata); - assertEquals(String.format("seriesPath: root.vehicle.d1.s2%n" + "resultDataType: INT32%n" + - "encoding: RLE%nnamespace type: ADD_PATH%n" + "args: "), plan.toString()); + assertEquals("seriesPath: root.vehicle.d1.s2, resultDataType: INT32, encoding: RLE, compression: SNAPPY", plan.toString()); } @Test @@ -102,8 +101,7 @@ public void testMetadata2() throws QueryProcessException { String metadata = "create timeseries root.vehicle.d1.s2 with datatype=int32,encoding=rle"; Planner processor = new Planner(); CreateTimeSeriesPlan plan = (CreateTimeSeriesPlan) processor.parseSQLToPhysicalPlan(metadata); - assertEquals(String.format("seriesPath: root.vehicle.d1.s2%n" + "resultDataType: INT32%n" + - "encoding: RLE%nnamespace type: ADD_PATH%n" + "args: "), plan.toString()); + assertEquals("seriesPath: root.vehicle.d1.s2, resultDataType: INT32, encoding: RLE, compression: SNAPPY", plan.toString()); } @Test diff --git a/server/src/test/java/org/apache/iotdb/db/utils/EnvironmentUtils.java b/server/src/test/java/org/apache/iotdb/db/utils/EnvironmentUtils.java index 5d7e0251f7f37..d3df4a0609155 100644 --- a/server/src/test/java/org/apache/iotdb/db/utils/EnvironmentUtils.java +++ b/server/src/test/java/org/apache/iotdb/db/utils/EnvironmentUtils.java @@ -24,7 +24,6 @@ import java.io.IOException; import java.net.InetSocketAddress; import java.net.Socket; -import java.net.SocketAddress; import javax.management.remote.JMXConnector; import javax.management.remote.JMXConnectorFactory; import javax.management.remote.JMXServiceURL; @@ -40,7 +39,6 @@ import org.apache.iotdb.db.engine.StorageEngine; import org.apache.iotdb.db.engine.cache.DeviceMetaDataCache; import org.apache.iotdb.db.engine.cache.TsFileMetaDataCache; -import org.apache.iotdb.db.exception.StartupException; import org.apache.iotdb.db.exception.StorageEngineException; import org.apache.iotdb.db.metadata.MManager; import org.apache.iotdb.db.query.context.QueryContext; @@ -187,12 +185,12 @@ public static void closeStatMonitor() { /** * disable memory control
this function should be called before all code in the setup */ - public static void envSetUp() throws StartupException { + public static void envSetUp() { logger.warn("EnvironmentUtil setup..."); System.setProperty(IoTDBConstant.REMOTE_JMX_PORT_NAME, "31999"); IoTDBDescriptor.getInstance().getConfig().setThriftServerAwaitTimeForStopService(0); //we do not start 8181 port in test. - IoTDBDescriptor.getInstance().getConfig().setEnableMetricsWebService(false); + IoTDBDescriptor.getInstance().getConfig().setEnableMetricService(false); if (daemon == null) { daemon = new IoTDB(); } diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TimeValuePair.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TimeValuePair.java index c52ee5b572359..035d3a5aae958 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TimeValuePair.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/TimeValuePair.java @@ -78,4 +78,5 @@ public int getSize() { public int compareTo(TimeValuePair o) { return Long.compare(this.getTimestamp(), o.getTimestamp()); } + } diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/BatchData.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/BatchData.java index eb483acc93beb..a4c316fee51cd 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/BatchData.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/BatchData.java @@ -18,16 +18,20 @@ */ package org.apache.iotdb.tsfile.read.common; +import java.io.Serializable; +import java.util.ArrayList; import org.apache.iotdb.tsfile.common.conf.TSFileConfig; import org.apache.iotdb.tsfile.exception.write.UnSupportedDataTypeException; import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; import org.apache.iotdb.tsfile.read.reader.BatchDataIterator; import org.apache.iotdb.tsfile.utils.Binary; import org.apache.iotdb.tsfile.utils.TsPrimitiveType; -import org.apache.iotdb.tsfile.utils.TsPrimitiveType.*; - -import java.io.Serializable; -import java.util.ArrayList; +import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsBinary; +import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsBoolean; +import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsDouble; +import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsFloat; +import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsInt; +import org.apache.iotdb.tsfile.utils.TsPrimitiveType.TsLong; /** * BatchData is a self-defined data structure which is optimized for different type of diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/TimeRange.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/TimeRange.java index a3261c0bfb61c..924a87158f046 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/TimeRange.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/TimeRange.java @@ -97,6 +97,10 @@ public boolean contains(TimeRange r) { return min <= r.min && max >= r.max; } + public boolean contains(long min, long max) { + return this.min <= min && this.max >= max; + } + /** * Set a closed interval [min,max]. diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/GroupByFilter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/GroupByFilter.java index e8cdb81056c02..f2d82bf282188 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/GroupByFilter.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/GroupByFilter.java @@ -146,4 +146,12 @@ public boolean equals(Object obj) { public int hashCode() { return Objects.hash(interval, slidingStep, startTime, endTime); } + + public long getStartTime() { + return startTime; + } + + public long getEndTime() { + return endTime; + } } diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/In.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/In.java index 1bb76d4345955..3767f0167f038 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/In.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/filter/operator/In.java @@ -118,4 +118,8 @@ public String toString() { public FilterSerializeId getSerializeId() { return FilterSerializeId.IN; } + + public Set getValues() { + return values; + } } diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/Murmur128Hash.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/Murmur128Hash.java index 2bb5118bdbeaa..9bc11499255cf 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/Murmur128Hash.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/Murmur128Hash.java @@ -18,34 +18,51 @@ */ package org.apache.iotdb.tsfile.utils; -import java.nio.ByteBuffer; - public class Murmur128Hash { + private Murmur128Hash() { + // util class + } + /** - * get hashcode of value by seed + * get hashcode of value by seed + * * @param value value - * @param seed seend + * @param seed seed * @return hashcode of value */ - public static int hash(String value, int seed){ - return (int) hash3_x64_128(ByteBuffer.wrap(value.getBytes()), 0, value.length(), seed); + public static int hash(String value, int seed) { + return (int) innerHash(value.getBytes(), 0, value.getBytes().length, seed); + } + + /** + * get hashcode of two values by seed + * + * @param value1 the first value + * @param value2 the second value + * @param seed seed + * @return hashcode of value + */ + public static int hash(String value1, long value2, int seed) { + return (int) innerHash( + BytesUtils.concatByteArray(value1.getBytes(), BytesUtils.longToBytes(value2)), 0, + value1.length() + 8, seed); } /************************************** * Methods to perform murmur 128 hash. **************************************/ - private static long getBlock(ByteBuffer key, int offset, int index) { - int i_8 = index << 3; - int blockOffset = offset + i_8; - return ((long) key.get(blockOffset) & 0xff) + (((long) key.get(blockOffset + 1) & 0xff) << 8) + private static long getBlock(byte[] key, int offset, int index) { + int i8 = index << 3; + int blockOffset = offset + i8; + return ((long) key[blockOffset] & 0xff) + (((long) key[blockOffset + 1] & 0xff) << 8) + - (((long) key.get(blockOffset + 2) & 0xff) << 16) + ( - ((long) key.get(blockOffset + 3) & 0xff) << 24) + - (((long) key.get(blockOffset + 4) & 0xff) << 32) + ( - ((long) key.get(blockOffset + 5) & 0xff) << 40) + - (((long) key.get(blockOffset + 6) & 0xff) << 48) + ( - ((long) key.get(blockOffset + 7) & 0xff) << 56); + (((long) key[blockOffset + 2] & 0xff) << 16) + ( + ((long) key[blockOffset + 3] & 0xff) << 24) + + (((long) key[blockOffset + 4] & 0xff) << 32) + ( + ((long) key[blockOffset + 5] & 0xff) << 40) + + (((long) key[blockOffset + 6] & 0xff) << 48) + ( + ((long) key[blockOffset + 7] & 0xff) << 56); } private static long rotl64(long v, int n) { @@ -61,7 +78,7 @@ private static long fmix(long k) { return k; } - private static long hash3_x64_128(ByteBuffer key, int offset, int length, long seed) { + private static long innerHash(byte[] key, int offset, int length, long seed) { final int nblocks = length >> 4; // Process as 128-bit blocks. long h1 = seed; long h2 = seed; @@ -70,7 +87,7 @@ private static long hash3_x64_128(ByteBuffer key, int offset, int length, long s // ---------- // body for (int i = 0; i < nblocks; i++) { - long k1 = getBlock(key, offset, i * 2 + 0); + long k1 = getBlock(key, offset, i * 2); long k2 = getBlock(key, offset, i * 2 + 1); k1 *= c1; k1 = rotl64(k1, 31); @@ -95,53 +112,53 @@ private static long hash3_x64_128(ByteBuffer key, int offset, int length, long s long k2 = 0; switch (length & 15) { case 15: - k2 ^= ((long) key.get(offset + 14)) << 48; + k2 ^= ((long) key[offset + 14]) << 48; // fallthrough case 14: - k2 ^= ((long) key.get(offset + 13)) << 40; + k2 ^= ((long) key[offset + 13]) << 40; // fallthrough case 13: - k2 ^= ((long) key.get(offset + 12)) << 32; + k2 ^= ((long) key[offset + 12]) << 32; // fallthrough case 12: - k2 ^= ((long) key.get(offset + 11)) << 24; + k2 ^= ((long) key[offset + 11]) << 24; // fallthrough case 11: - k2 ^= ((long) key.get(offset + 10)) << 16; + k2 ^= ((long) key[offset + 10]) << 16; // fallthrough case 10: - k2 ^= ((long) key.get(offset + 9)) << 8; + k2 ^= ((long) key[offset + 9]) << 8; // fallthrough case 9: - k2 ^= ((long) key.get(offset + 8)); + k2 ^= key[offset + 8]; k2 *= c2; k2 = rotl64(k2, 33); k2 *= c1; h2 ^= k2; // fallthrough case 8: - k1 ^= ((long) key.get(offset + 7)) << 56; + k1 ^= ((long) key[offset + 7]) << 56; // fallthrough case 7: - k1 ^= ((long) key.get(offset + 6)) << 48; + k1 ^= ((long) key[offset + 6]) << 48; // fallthrough case 6: - k1 ^= ((long) key.get(offset + 5)) << 40; + k1 ^= ((long) key[offset + 5]) << 40; // fallthrough case 5: - k1 ^= ((long) key.get(offset + 4)) << 32; + k1 ^= ((long) key[offset + 4]) << 32; // fallthrough case 4: - k1 ^= ((long) key.get(offset + 3)) << 24; + k1 ^= ((long) key[offset + 3]) << 24; // fallthrough case 3: - k1 ^= ((long) key.get(offset + 2)) << 16; + k1 ^= ((long) key[offset + 2]) << 16; // fallthrough case 2: - k1 ^= ((long) key.get(offset + 1)) << 8; + k1 ^= ((long) key[offset + 1]) << 8; // fallthrough case 1: - k1 ^= (key.get(offset)); + k1 ^= (key[offset]); k1 *= c1; k1 = rotl64(k1, 31); k1 *= c2; @@ -164,4 +181,5 @@ private static long hash3_x64_128(ByteBuffer key, int offset, int length, long s return h1 + h2; } + } diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/ReadWriteIOUtils.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/ReadWriteIOUtils.java index 9d89621eac3fd..2e305d97c1cf8 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/ReadWriteIOUtils.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/utils/ReadWriteIOUtils.java @@ -25,6 +25,7 @@ import static org.apache.iotdb.tsfile.utils.ReadWriteIOUtils.ClassSerializeId.FLOAT; import static org.apache.iotdb.tsfile.utils.ReadWriteIOUtils.ClassSerializeId.INTEGER; import static org.apache.iotdb.tsfile.utils.ReadWriteIOUtils.ClassSerializeId.LONG; +import static org.apache.iotdb.tsfile.utils.ReadWriteIOUtils.ClassSerializeId.NULL; import static org.apache.iotdb.tsfile.utils.ReadWriteIOUtils.ClassSerializeId.STRING; import java.io.DataOutputStream; @@ -781,7 +782,7 @@ public static boolean checkIfMagicString(InputStream inputStream) throws IOExcep } enum ClassSerializeId { - LONG, DOUBLE, INTEGER, FLOAT, BINARY, BOOLEAN, STRING + LONG, DOUBLE, INTEGER, FLOAT, BINARY, BOOLEAN, STRING, NULL } public static void writeObject(Object value, DataOutputStream outputStream) { @@ -806,6 +807,8 @@ public static void writeObject(Object value, DataOutputStream outputStream) { } else if (value instanceof Boolean) { outputStream.write(BOOLEAN.ordinal()); outputStream.write(((Boolean) value) ? 1 : 0); + } else if (value == null) { + outputStream.write(NULL.ordinal()); } else { outputStream.write(STRING.ordinal()); byte[] bytes = value.toString().getBytes(); @@ -835,6 +838,8 @@ public static Object readObject(ByteBuffer buffer) { byte[] bytes = new byte[length]; buffer.get(bytes); return new Binary(bytes); + case NULL: + return null; case STRING: default: length = buffer.getInt(); diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/schema/MeasurementSchema.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/schema/MeasurementSchema.java index b0c4a3434023a..91d64f6dcfc5c 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/schema/MeasurementSchema.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/schema/MeasurementSchema.java @@ -318,4 +318,5 @@ public String toString() { sc.addTail("]"); return sc.toString(); } + }