提交 4b35aa3c 编写于 作者: X xiangdong huang

rename OverflowSupport to OverflowMemtable; rename IntervalFileNode to...

rename OverflowSupport to OverflowMemtable; rename IntervalFileNode to TsFileResource; delete OverflowSeriesImpl
上级 463b99c2
......@@ -700,7 +700,7 @@ public class FileNodeManager implements IStatistic, IService {
* @param fileNodeName the seriesPath of storage group
* @param appendFile the appended tsfile information
*/
public boolean appendFileToFileNode(String fileNodeName, IntervalFileNode appendFile,
public boolean appendFileToFileNode(String fileNodeName, TsFileResource appendFile,
String appendFilePath) throws FileNodeManagerException {
FileNodeProcessor fileNodeProcessor = getProcessor(fileNodeName, true);
try {
......@@ -729,7 +729,7 @@ public class FileNodeManager implements IStatistic, IService {
* @param fileNodeName the seriesPath of storage group
* @param appendFile the appended tsfile information
*/
public List<String> getOverlapFilesFromFileNode(String fileNodeName, IntervalFileNode appendFile,
public List<String> getOverlapFilesFromFileNode(String fileNodeName, TsFileResource appendFile,
String uuid) throws FileNodeManagerException {
FileNodeProcessor fileNodeProcessor = getProcessor(fileNodeName, true);
List<String> overlapFiles;
......
......@@ -26,7 +26,7 @@ import java.util.Map;
/**
* FileNodeProcessorStore is used to store information about FileNodeProcessor's status.
* lastUpdateTime is changed and stored by BufferWrite flush or BufferWrite close.
* emptyIntervalFileNode and newFileNodes are changed and stored by Overflow flush and
* emptyTsFileResource and newFileNodes are changed and stored by Overflow flush and
* Overflow close. fileNodeProcessorState is changed and stored by the change of FileNodeProcessor's
* status such as "work->merge merge->wait wait->work". numOfMergeFile is changed
* and stored when FileNodeProcessor's status changes from work to merge.
......@@ -39,8 +39,8 @@ public class FileNodeProcessorStore implements Serializable {
private boolean isOverflowed;
private Map<String, Long> lastUpdateTimeMap;
private IntervalFileNode emptyIntervalFileNode;
private List<IntervalFileNode> newFileNodes;
private TsFileResource emptyTsFileResource;
private List<TsFileResource> newFileNodes;
private int numOfMergeFile;
private FileNodeProcessorStatus fileNodeProcessorStatus;
......@@ -48,19 +48,19 @@ public class FileNodeProcessorStore implements Serializable {
* Constructor of FileNodeProcessorStore.
* @param isOverflowed whether this FileNode contains unmerged Overflow operations.
* @param lastUpdateTimeMap the timestamp of last data point of each device in this FileNode.
* @param emptyIntervalFileNode a place holder when the FileNode contains no TsFile.
* @param emptyTsFileResource a place holder when the FileNode contains no TsFile.
* @param newFileNodes TsFiles in the FileNode.
* @param fileNodeProcessorStatus the status of the FileNode.
* @param numOfMergeFile the number of files already merged in one merge operation.
*/
public FileNodeProcessorStore(boolean isOverflowed, Map<String, Long> lastUpdateTimeMap,
IntervalFileNode emptyIntervalFileNode,
List<IntervalFileNode> newFileNodes,
TsFileResource emptyTsFileResource,
List<TsFileResource> newFileNodes,
FileNodeProcessorStatus fileNodeProcessorStatus,
int numOfMergeFile) {
this.isOverflowed = isOverflowed;
this.lastUpdateTimeMap = lastUpdateTimeMap;
this.emptyIntervalFileNode = emptyIntervalFileNode;
this.emptyTsFileResource = emptyTsFileResource;
this.newFileNodes = newFileNodes;
this.fileNodeProcessorStatus = fileNodeProcessorStatus;
this.numOfMergeFile = numOfMergeFile;
......@@ -90,19 +90,19 @@ public class FileNodeProcessorStore implements Serializable {
this.lastUpdateTimeMap = lastUpdateTimeMap;
}
public IntervalFileNode getEmptyIntervalFileNode() {
return emptyIntervalFileNode;
public TsFileResource getEmptyTsFileResource() {
return emptyTsFileResource;
}
public void setEmptyIntervalFileNode(IntervalFileNode emptyIntervalFileNode) {
this.emptyIntervalFileNode = emptyIntervalFileNode;
public void setEmptyTsFileResource(TsFileResource emptyTsFileResource) {
this.emptyTsFileResource = emptyTsFileResource;
}
public List<IntervalFileNode> getNewFileNodes() {
public List<TsFileResource> getNewFileNodes() {
return newFileNodes;
}
public void setNewFileNodes(List<IntervalFileNode> newFileNodes) {
public void setNewFileNodes(List<TsFileResource> newFileNodes) {
this.newFileNodes = newFileNodes;
}
......
......@@ -32,7 +32,7 @@ import org.apache.iotdb.db.engine.modification.ModificationFile;
/**
* This class is used to store one bufferwrite file status.<br>
*/
public class IntervalFileNode implements Serializable {
public class TsFileResource implements Serializable {
private static final long serialVersionUID = -4309683416067212549L;
......@@ -45,7 +45,7 @@ public class IntervalFileNode implements Serializable {
private transient ModificationFile modFile;
public IntervalFileNode(Map<String, Long> startTimeMap, Map<String, Long> endTimeMap,
public TsFileResource(Map<String, Long> startTimeMap, Map<String, Long> endTimeMap,
OverflowChangeType type, int baseDirIndex, String relativePath) {
this.overflowChangeType = type;
......@@ -65,7 +65,7 @@ public class IntervalFileNode implements Serializable {
* @param type whether this file is affected by overflow and how it is affected.
* @param relativePath the path of the file relative to the FileNode.
*/
public IntervalFileNode(OverflowChangeType type, int baseDirIndex, String relativePath) {
public TsFileResource(OverflowChangeType type, int baseDirIndex, String relativePath) {
this.overflowChangeType = type;
this.baseDirIndex = baseDirIndex;
......@@ -78,7 +78,7 @@ public class IntervalFileNode implements Serializable {
+ relativePath + ModificationFile.FILE_SUFFIX);
}
public IntervalFileNode(OverflowChangeType type, String baseDir, String relativePath) {
public TsFileResource(OverflowChangeType type, String baseDir, String relativePath) {
this.overflowChangeType = type;
this.baseDirIndex = Directories.getInstance().getTsFileFolderIndex(baseDir);
......@@ -91,7 +91,7 @@ public class IntervalFileNode implements Serializable {
+ relativePath + ModificationFile.FILE_SUFFIX);
}
public IntervalFileNode(OverflowChangeType type, String relativePath) {
public TsFileResource(OverflowChangeType type, String relativePath) {
this(type, 0, relativePath);
}
......@@ -220,11 +220,11 @@ public class IntervalFileNode implements Serializable {
}
public IntervalFileNode backUp() {
public TsFileResource backUp() {
Map<String, Long> startTimeMapCopy = new HashMap<>(this.startTimeMap);
Map<String, Long> endTimeMapCopy = new HashMap<>(this.endTimeMap);
return new IntervalFileNode(startTimeMapCopy, endTimeMapCopy, overflowChangeType,
return new TsFileResource(startTimeMapCopy, endTimeMapCopy, overflowChangeType,
baseDirIndex, relativePath);
}
......@@ -244,7 +244,7 @@ public class IntervalFileNode implements Serializable {
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
IntervalFileNode fileNode = (IntervalFileNode) o;
TsFileResource fileNode = (TsFileResource) o;
return baseDirIndex == fileNode.baseDirIndex &&
overflowChangeType == fileNode.overflowChangeType &&
Objects.equals(relativePath, fileNode.relativePath) &&
......@@ -257,7 +257,7 @@ public class IntervalFileNode implements Serializable {
public String toString() {
return String.format(
"IntervalFileNode [relativePath=%s,overflowChangeType=%s, startTimeMap=%s,"
"TsFileResource [relativePath=%s,overflowChangeType=%s, startTimeMap=%s,"
+ " endTimeMap=%s, mergeChanged=%s]",
relativePath, overflowChangeType, startTimeMap, endTimeMap, mergeChanged);
}
......
......@@ -24,27 +24,28 @@ import org.apache.iotdb.db.engine.memtable.IMemTable;
import org.apache.iotdb.db.engine.memtable.PrimitiveMemTable;
import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.file.metadata.statistics.LongStatistics;
import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
import org.apache.iotdb.tsfile.read.common.BatchData;
import org.apache.iotdb.tsfile.write.record.TSRecord;
import org.apache.iotdb.tsfile.write.record.datapoint.DataPoint;
/**
* This class is used to store and query all overflow data in memory.<br>
* This just represent someone storage group.<br>
*/
public class OverflowSupport {
public class OverflowMemtable {
/**
* store update and delete data
*/
private Map<String, Map<String, OverflowSeriesImpl>> indexTrees;
private Map<String, Map<String, LongStatistics>> indexTrees;
/**
* store insert data
*/
private IMemTable memTable;
public OverflowSupport() {
public OverflowMemtable() {
indexTrees = new HashMap<>();
memTable = new PrimitiveMemTable();
}
......@@ -68,9 +69,9 @@ public class OverflowSupport {
indexTrees.put(deviceId, new HashMap<>());
}
if (!indexTrees.get(deviceId).containsKey(measurementId)) {
indexTrees.get(deviceId).put(measurementId, new OverflowSeriesImpl(measurementId, dataType));
indexTrees.get(deviceId).put(measurementId, new LongStatistics());
}
indexTrees.get(deviceId).get(measurementId).update(startTime, endTime);
indexTrees.get(deviceId).get(measurementId).updateStats(startTime, endTime);
}
public void delete(String deviceId, String measurementId, long timestamp, boolean isFlushing) {
......@@ -87,20 +88,11 @@ public class OverflowSupport {
return memTable.query(deviceId, measurementId, dataType, props);
}
public BatchData queryOverflowUpdateInMemory(String deviceId, String measurementId,
TSDataType dataType) {
if (indexTrees.containsKey(deviceId) && indexTrees.get(deviceId).containsKey(measurementId)
&& indexTrees.get(deviceId).get(measurementId).getDataType().equals(dataType)) {
return indexTrees.get(deviceId).get(measurementId).query();
}
return null;
}
public boolean isEmptyOfOverflowSeriesMap() {
return indexTrees.isEmpty();
}
public Map<String, Map<String, OverflowSeriesImpl>> getOverflowSeriesMap() {
public Map<String, Map<String, LongStatistics>> getOverflowSeriesMap() {
return indexTrees;
}
......
......@@ -73,8 +73,8 @@ public class OverflowProcessor extends Processor {
private OverflowResource workResource;
private OverflowResource mergeResource;
private OverflowSupport workSupport;
private OverflowSupport flushSupport;
private OverflowMemtable workSupport;
private OverflowMemtable flushSupport;
private volatile Future<Boolean> flushFuture = new ImmediateFuture<>(true);
private volatile boolean isMerge;
......@@ -113,7 +113,7 @@ public class OverflowProcessor extends Processor {
// recover file
recovery(processorDataDir);
// memory
workSupport = new OverflowSupport();
workSupport = new OverflowMemtable();
overflowFlushAction = parameters.get(FileNodeConstants.OVERFLOW_FLUSH_ACTION);
filenodeFlushAction = parameters
.get(FileNodeConstants.FILENODE_PROCESSOR_FLUSH_ACTION);
......@@ -380,7 +380,7 @@ public class OverflowProcessor extends Processor {
queryFlushLock.lock();
try {
flushSupport = workSupport;
workSupport = new OverflowSupport();
workSupport = new OverflowMemtable();
} finally {
queryFlushLock.unlock();
}
......
/**
* 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.engine.overflow.io;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.file.metadata.statistics.LongStatistics;
import org.apache.iotdb.tsfile.file.metadata.statistics.Statistics;
import org.apache.iotdb.tsfile.read.common.BatchData;
/**
* This class is only used to store and query overflow overflowIndex {@code IIntervalTreeOperator}
* data in memory.
*/
public class OverflowSeriesImpl {
/**
* The data of update and delete in memory for this time series.
*/
private String measurementId;
private TSDataType dataType;
private Statistics<Long> statistics;
private int valueCount;
public OverflowSeriesImpl(String measurementId, TSDataType dataType) {
this.measurementId = measurementId;
this.dataType = dataType;
statistics = new LongStatistics();
}
public void update(long startTime, long endTime) {
statistics.updateStats(startTime, endTime);
valueCount++;
}
public void delete(long timestamp) {
statistics.updateStats(timestamp, timestamp);
valueCount++;
}
public BatchData query() {
return null;
}
public long getSize() {
return 0;
}
public String getMeasurementId() {
return measurementId;
}
public TSDataType getDataType() {
return dataType;
}
public Statistics<Long> getStatistics() {
return statistics;
}
public int getValueCount() {
return valueCount;
}
}
......@@ -19,8 +19,7 @@
package org.apache.iotdb.db.engine.querycontext;
import java.util.List;
import org.apache.iotdb.db.engine.filenode.IntervalFileNode;
import org.apache.iotdb.db.engine.memtable.TimeValuePairSorter;
import org.apache.iotdb.db.engine.filenode.TsFileResource;
import org.apache.iotdb.tsfile.read.common.Path;
public class GlobalSortedSeriesDataSource {
......@@ -28,7 +27,7 @@ public class GlobalSortedSeriesDataSource {
private Path seriesPath;
// sealed tsfile
private List<IntervalFileNode> sealedTsFiles;
private List<TsFileResource> sealedTsFiles;
// unsealed tsfile
private UnsealedTsFile unsealedTsFile;
......@@ -36,7 +35,7 @@ public class GlobalSortedSeriesDataSource {
// seq mem-table
private ReadOnlyMemChunk readableChunk;
public GlobalSortedSeriesDataSource(Path seriesPath, List<IntervalFileNode> sealedTsFiles,
public GlobalSortedSeriesDataSource(Path seriesPath, List<TsFileResource> sealedTsFiles,
UnsealedTsFile unsealedTsFile,
ReadOnlyMemChunk readableChunk) {
this.seriesPath = seriesPath;
......@@ -50,11 +49,11 @@ public class GlobalSortedSeriesDataSource {
return sealedTsFiles != null && !sealedTsFiles.isEmpty();
}
public List<IntervalFileNode> getSealedTsFiles() {
public List<TsFileResource> getSealedTsFiles() {
return sealedTsFiles;
}
public void setSealedTsFiles(List<IntervalFileNode> sealedTsFiles) {
public void setSealedTsFiles(List<TsFileResource> sealedTsFiles) {
this.sealedTsFiles = sealedTsFiles;
}
......
......@@ -39,7 +39,7 @@ import org.apache.iotdb.db.conf.IoTDBConstant;
import org.apache.iotdb.db.conf.IoTDBDescriptor;
import org.apache.iotdb.db.conf.directories.Directories;
import org.apache.iotdb.db.engine.filenode.FileNodeManager;
import org.apache.iotdb.db.engine.filenode.IntervalFileNode;
import org.apache.iotdb.db.engine.filenode.TsFileResource;
import org.apache.iotdb.db.engine.filenode.OverflowChangeType;
import org.apache.iotdb.db.exception.FileNodeManagerException;
import org.apache.iotdb.db.utils.PostbackUtils;
......@@ -660,7 +660,7 @@ public class ServerServiceImpl implements ServerService.Iface {
// create a new fileNode
String header = postbackPath + uuid.get() + File.separator + "data" + File.separator;
String relativePath = path.substring(header.length());
IntervalFileNode fileNode = new IntervalFileNode(startTimeMap, endTimeMap,
TsFileResource fileNode = new TsFileResource(startTimeMap, endTimeMap,
OverflowChangeType.NO_CHANGE,
Directories.getInstance().getNextFolderIndexForTsFile(), relativePath);
// call interface of load external file
......
......@@ -21,7 +21,7 @@ package org.apache.iotdb.db.query.control;
import java.util.HashSet;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.iotdb.db.engine.filenode.IntervalFileNode;
import org.apache.iotdb.db.engine.filenode.TsFileResource;
import org.apache.iotdb.db.engine.querycontext.OverflowInsertFile;
import org.apache.iotdb.db.engine.querycontext.QueryDataSource;
......@@ -66,8 +66,8 @@ public class OpenedFilePathsManager {
* Add the unique file paths to closedFilePathsMap and unclosedFilePathsMap.
*/
public void addUsedFilesForCurrentRequestThread(long jobId, QueryDataSource dataSource) {
for (IntervalFileNode intervalFileNode : dataSource.getSeqDataSource().getSealedTsFiles()) {
String sealedFilePath = intervalFileNode.getFilePath();
for (TsFileResource tsFileResource : dataSource.getSeqDataSource().getSealedTsFiles()) {
String sealedFilePath = tsFileResource.getFilePath();
addFilePathToMap(jobId, sealedFilePath, true);
}
......
......@@ -21,7 +21,7 @@ package org.apache.iotdb.db.query.factory;
import java.io.IOException;
import java.util.List;
import org.apache.iotdb.db.engine.filenode.IntervalFileNode;
import org.apache.iotdb.db.engine.filenode.TsFileResource;
import org.apache.iotdb.db.engine.modification.Modification;
import org.apache.iotdb.db.engine.querycontext.OverflowInsertFile;
import org.apache.iotdb.db.engine.querycontext.OverflowSeriesDataSource;
......@@ -131,7 +131,7 @@ public class SeriesReaderFactory {
* This method is used to construct reader for merge process in IoTDB. To merge only one TsFile
* data and one UnSeqFile data.
*/
public IReader createSeriesReaderForMerge(IntervalFileNode intervalFileNode,
public IReader createSeriesReaderForMerge(TsFileResource tsFileResource,
OverflowSeriesDataSource overflowSeriesDataSource,
SingleSeriesExpression singleSeriesExpression,
QueryContext context)
......@@ -139,12 +139,12 @@ public class SeriesReaderFactory {
logger.debug("Create seriesReaders for merge. SeriesFilter = {}. TsFilePath = {}",
singleSeriesExpression,
intervalFileNode.getFilePath());
tsFileResource.getFilePath());
PriorityMergeReader priorityMergeReader = new PriorityMergeReader();
// Sequence reader
IReader seriesInTsFileReader = createSealedTsFileReaderForMerge(intervalFileNode,
IReader seriesInTsFileReader = createSealedTsFileReaderForMerge(tsFileResource,
singleSeriesExpression, context);
priorityMergeReader.addReaderWithPriority(seriesInTsFileReader, 1);
......@@ -156,7 +156,7 @@ public class SeriesReaderFactory {
return priorityMergeReader;
}
private IReader createSealedTsFileReaderForMerge(IntervalFileNode fileNode,
private IReader createSealedTsFileReaderForMerge(TsFileResource fileNode,
SingleSeriesExpression singleSeriesExpression,
QueryContext context)
throws IOException {
......
......@@ -22,7 +22,7 @@ package org.apache.iotdb.db.query.reader.sequence;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.iotdb.db.engine.filenode.IntervalFileNode;
import org.apache.iotdb.db.engine.filenode.TsFileResource;
import org.apache.iotdb.db.engine.modification.Modification;
import org.apache.iotdb.db.query.context.QueryContext;
import org.apache.iotdb.db.query.control.FileReaderManager;
......@@ -45,7 +45,7 @@ import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReaderWithoutFilter;
public class SealedTsFilesReader implements IReader {
private Path seriesPath;
private List<IntervalFileNode> sealedTsFiles;
private List<TsFileResource> sealedTsFiles;
private int usedIntervalFileIndex;
private FileSeriesReader seriesReader;
private Filter filter;
......@@ -53,7 +53,7 @@ public class SealedTsFilesReader implements IReader {
private boolean hasCachedData;
private QueryContext context;
public SealedTsFilesReader(Path seriesPath, List<IntervalFileNode> sealedTsFiles, Filter filter,
public SealedTsFilesReader(Path seriesPath, List<TsFileResource> sealedTsFiles, Filter filter,
QueryContext context) {
this(seriesPath, sealedTsFiles, context);
this.filter = filter;
......@@ -63,7 +63,7 @@ public class SealedTsFilesReader implements IReader {
/**
* init with seriesPath and sealedTsFiles.
*/
public SealedTsFilesReader(Path seriesPath, List<IntervalFileNode> sealedTsFiles,
public SealedTsFilesReader(Path seriesPath, List<TsFileResource> sealedTsFiles,
QueryContext context) {
this.seriesPath = seriesPath;
this.sealedTsFiles = sealedTsFiles;
......@@ -109,7 +109,7 @@ public class SealedTsFilesReader implements IReader {
while (!flag && usedIntervalFileIndex < sealedTsFiles.size()) {
// init until reach a satisfied reader
if (seriesReader == null || !seriesReader.hasNextBatch()) {
IntervalFileNode fileNode = sealedTsFiles.get(usedIntervalFileIndex++);
TsFileResource fileNode = sealedTsFiles.get(usedIntervalFileIndex++);
if (singleTsFileSatisfied(fileNode)) {
initSingleTsFileReader(fileNode, context);
} else {
......@@ -155,7 +155,7 @@ public class SealedTsFilesReader implements IReader {
}
}
private boolean singleTsFileSatisfied(IntervalFileNode fileNode) {
private boolean singleTsFileSatisfied(TsFileResource fileNode) {
if (filter == null) {
return true;
......@@ -166,7 +166,7 @@ public class SealedTsFilesReader implements IReader {
return filter.satisfyStartEndTime(startTime, endTime);
}
private void initSingleTsFileReader(IntervalFileNode fileNode, QueryContext context)
private void initSingleTsFileReader(TsFileResource fileNode, QueryContext context)
throws IOException {
// to avoid too many opened files
......
......@@ -83,13 +83,13 @@ public class SerializeUtilTest {
@Test
public void testFileStore() {
IntervalFileNode emptyIntervalFileNode = new IntervalFileNode(OverflowChangeType.NO_CHANGE,
TsFileResource emptyTsFileResource = new TsFileResource(OverflowChangeType.NO_CHANGE,
null);
List<IntervalFileNode> newFilenodes = new ArrayList<>();
List<TsFileResource> newFilenodes = new ArrayList<>();
String deviceId = "d0.s0";
for (int i = 1; i <= 3; i++) {
// i * 100, i * 100 + 99
IntervalFileNode node = new IntervalFileNode(OverflowChangeType.NO_CHANGE,
TsFileResource node = new TsFileResource(OverflowChangeType.NO_CHANGE,
"bufferfiletest" + i);
node.setStartTime(deviceId, i * 100);
node.setEndTime(deviceId, i * 100 + 99);
......@@ -100,7 +100,7 @@ public class SerializeUtilTest {
lastUpdateTimeMap.put(deviceId, (long) 500);
FileNodeProcessorStore fileNodeProcessorStore = new FileNodeProcessorStore(false,
lastUpdateTimeMap,
emptyIntervalFileNode, newFilenodes, fileNodeProcessorState, 0);
emptyTsFileResource, newFilenodes, fileNodeProcessorState, 0);
SerializeUtil<FileNodeProcessorStore> serializeUtil = new SerializeUtil<>();
......@@ -114,13 +114,13 @@ public class SerializeUtilTest {
try {
FileNodeProcessorStore fileNodeProcessorStore2 = serializeUtil.deserialize(filePath)
.orElse(new FileNodeProcessorStore(false, new HashMap<>(),
new IntervalFileNode(OverflowChangeType.NO_CHANGE, null),
new ArrayList<IntervalFileNode>(),
new TsFileResource(OverflowChangeType.NO_CHANGE, null),
new ArrayList<TsFileResource>(),
FileNodeProcessorStatus.NONE, 0));
assertEquals(fileNodeProcessorStore.getLastUpdateTimeMap(),
fileNodeProcessorStore2.getLastUpdateTimeMap());
assertEquals(fileNodeProcessorStore.getEmptyIntervalFileNode(),
fileNodeProcessorStore2.getEmptyIntervalFileNode());
assertEquals(fileNodeProcessorStore.getEmptyTsFileResource(),
fileNodeProcessorStore2.getEmptyTsFileResource());
assertEquals(fileNodeProcessorStore.getNewFileNodes(),
fileNodeProcessorStore2.getNewFileNodes());
assertEquals(fileNodeProcessorStore.getNumOfMergeFile(),
......
......@@ -28,9 +28,9 @@ import org.junit.After;
import org.junit.Before;
import org.junit.Test;
public class OverflowSupportTest {
public class OverflowMemtableTest {
private OverflowSupport support = new OverflowSupport();
private OverflowMemtable support = new OverflowMemtable();
private String deviceId1 = "d1";
private String deviceId2 = "d2";
private String measurementId1 = "s1";
......
......@@ -44,7 +44,7 @@ public class OverflowResourceTest {
private String positionFileName = "positionFile";
private String filePath = "overflow";
private String dataPath = "1";
private OverflowSupport support = new OverflowSupport();
private OverflowMemtable support = new OverflowMemtable();
@Before
public void setUp() throws Exception {
......
......@@ -47,7 +47,7 @@ public class OverflowTestUtils {
return fileSchema;
}
public static void produceInsertData(OverflowSupport support) {
public static void produceInsertData(OverflowMemtable support) {
support.insert(getData(deviceId1, measurementId1, dataType1, String.valueOf(1), 1));
support.insert(getData(deviceId1, measurementId1, dataType1, String.valueOf(3), 3));
support.insert(getData(deviceId1, measurementId1, dataType1, String.valueOf(2), 2));
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册