From a677f89b90a90d0f1e9ddeb83f0d22d2c8dffb5c Mon Sep 17 00:00:00 2001 From: suyue <2335813255@qq.com> Date: Tue, 26 Feb 2019 11:28:02 +0800 Subject: [PATCH 1/8] optimize value-filter query --- .../EngineDataSetWithTimeGenerator.java | 6 +- .../EngineExecutorWithTimeGenerator.java | 33 ++--- .../db/query/factory/SeriesReaderFactory.java | 72 +++++++-- .../reader/mem/MemChunkReaderByTimestamp.java | 22 ++- .../reader/merge/EngineReaderByTimeStamp.java | 16 +- .../db/query/reader/merge/MergeElement.java | 64 ++++++++ .../merge/PriorityMergeReaderByTimestamp.java | 137 ++++++++++++++---- .../SealedTsFilesReaderByTimestamp.java | 102 +++++++++++++ .../SequenceDataReaderByTimestamp.java | 104 +++++++++++++ .../UnSealedTsFilesReaderByTimestamp.java | 73 ++++++++++ .../EngineChunkReaderByTimestamp.java | 97 +++++++++++++ .../PriorityMergeReaderByTimestampTest.java | 68 +++++---- .../iotdb/tsfile/read/common/BatchData.java | 25 ++++ .../reader/chunk/ChunkReaderByTimestamp.java | 5 + .../series/SeriesReaderByTimestamp.java | 47 +++++- 15 files changed, 770 insertions(+), 101 deletions(-) create mode 100644 iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/MergeElement.java create mode 100644 iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReaderByTimestamp.java create mode 100644 iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReaderByTimestamp.java create mode 100644 iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFilesReaderByTimestamp.java create mode 100644 iotdb/src/main/java/org/apache/iotdb/db/query/reader/unsequence/EngineChunkReaderByTimestamp.java diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithTimeGenerator.java b/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithTimeGenerator.java index 20d93e89a8f21..a658eb0777879 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithTimeGenerator.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithTimeGenerator.java @@ -62,11 +62,11 @@ public RowRecord next() throws IOException { RowRecord rowRecord = new RowRecord(timestamp); for (int i = 0; i < readers.size(); i++) { EngineReaderByTimeStamp reader = readers.get(i); - TsPrimitiveType tsPrimitiveType = reader.getValueInTimestamp(timestamp); - if (tsPrimitiveType == null) { + Object value = reader.getValueInTimestamp(timestamp); + if (value == null) { rowRecord.addField(new Field(null)); } else { - rowRecord.addField(getField(tsPrimitiveType.getValue(), dataTypes.get(i))); + rowRecord.addField(getField(value, dataTypes.get(i))); } } diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineExecutorWithTimeGenerator.java b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineExecutorWithTimeGenerator.java index 86d36dc393b33..1b4cc84451e47 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineExecutorWithTimeGenerator.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineExecutorWithTimeGenerator.java @@ -1,19 +1,15 @@ /** - * 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 + * 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 + * 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 + * 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.query.executor; @@ -30,9 +26,8 @@ import org.apache.iotdb.db.query.dataset.EngineDataSetWithTimeGenerator; import org.apache.iotdb.db.query.factory.SeriesReaderFactory; import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp; -import org.apache.iotdb.db.query.reader.merge.PriorityMergeReader; import org.apache.iotdb.db.query.reader.merge.PriorityMergeReaderByTimestamp; -import org.apache.iotdb.db.query.reader.sequence.SequenceDataReader; +import org.apache.iotdb.db.query.reader.sequence.SequenceDataReaderByTimestamp; import org.apache.iotdb.db.query.timegenerator.EngineTimeGenerator; import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; import org.apache.iotdb.tsfile.read.common.Path; @@ -102,13 +97,13 @@ private List getReadersOfSelectedPaths(List paths PriorityMergeReaderByTimestamp mergeReaderByTimestamp = new PriorityMergeReaderByTimestamp(); // reader for sequence data - SequenceDataReader tsFilesReader = new SequenceDataReader(queryDataSource.getSeqDataSource(), - null); + SequenceDataReaderByTimestamp tsFilesReader = new SequenceDataReaderByTimestamp( + queryDataSource.getSeqDataSource()); mergeReaderByTimestamp.addReaderWithPriority(tsFilesReader, 1); // reader for unSequence data - PriorityMergeReader unSeqMergeReader = SeriesReaderFactory.getInstance() - .createUnSeqMergeReader(queryDataSource.getOverflowSeriesDataSource(), null); + PriorityMergeReaderByTimestamp unSeqMergeReader = SeriesReaderFactory.getInstance() + .createUnSeqMergeReaderByTimestamp(queryDataSource.getOverflowSeriesDataSource()); mergeReaderByTimestamp.addReaderWithPriority(unSeqMergeReader, 2); readersOfSelectedSeries.add(mergeReaderByTimestamp); diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/factory/SeriesReaderFactory.java b/iotdb/src/main/java/org/apache/iotdb/db/query/factory/SeriesReaderFactory.java index 8062172a3cb33..090adb7d17d04 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/factory/SeriesReaderFactory.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/factory/SeriesReaderFactory.java @@ -1,19 +1,15 @@ /** - * 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 + * 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 + * 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 + * 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.query.factory; @@ -25,11 +21,14 @@ import org.apache.iotdb.db.engine.querycontext.OverflowSeriesDataSource; import org.apache.iotdb.db.query.control.FileReaderManager; import org.apache.iotdb.db.query.reader.IReader; +import org.apache.iotdb.db.query.reader.mem.MemChunkReaderByTimestamp; import org.apache.iotdb.db.query.reader.mem.MemChunkReaderWithFilter; import org.apache.iotdb.db.query.reader.mem.MemChunkReaderWithoutFilter; import org.apache.iotdb.db.query.reader.merge.PriorityMergeReader; +import org.apache.iotdb.db.query.reader.merge.PriorityMergeReaderByTimestamp; import org.apache.iotdb.db.query.reader.sequence.SealedTsFilesReader; import org.apache.iotdb.db.query.reader.unsequence.EngineChunkReader; +import org.apache.iotdb.db.query.reader.unsequence.EngineChunkReaderByTimestamp; import org.apache.iotdb.tsfile.common.constant.StatisticConstant; import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData; import org.apache.iotdb.tsfile.read.TsFileSequenceReader; @@ -41,6 +40,7 @@ import org.apache.iotdb.tsfile.read.filter.DigestForFilter; import org.apache.iotdb.tsfile.read.filter.basic.Filter; import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReader; +import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReaderByTimestamp; import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReaderWithFilter; import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReaderWithoutFilter; import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReader; @@ -121,7 +121,51 @@ public PriorityMergeReader createUnSeqMergeReader( return unSeqMergeReader; } - // TODO createUnSeqMergeReaderByTime a method with filter + /** + * This method is used to create overflow insert reader by timestamp for IoTDB request, such as + * query, aggregation and groupby request. + */ + public PriorityMergeReaderByTimestamp createUnSeqMergeReaderByTimestamp( + OverflowSeriesDataSource overflowSeriesDataSource) + throws IOException { + + PriorityMergeReaderByTimestamp unSeqMergeReader = new PriorityMergeReaderByTimestamp(); + + int priorityValue = 1; + + for (OverflowInsertFile overflowInsertFile : overflowSeriesDataSource + .getOverflowInsertFileList()) { + + // store only one opened file stream into manager, to avoid too many opened files + TsFileSequenceReader unClosedTsFileReader = FileReaderManager.getInstance() + .get(overflowInsertFile.getFilePath(), true); + + ChunkLoaderImpl chunkLoader = new ChunkLoaderImpl(unClosedTsFileReader); + + for (ChunkMetaData chunkMetaData : overflowInsertFile.getChunkMetaDataList()) { + + Chunk chunk = chunkLoader.getChunk(chunkMetaData); + ChunkReaderByTimestamp chunkReader = new ChunkReaderByTimestamp(chunk, + chunkMetaData.getMaxTombstoneTime()); + + unSeqMergeReader + .addReaderWithPriority( + new EngineChunkReaderByTimestamp(chunkReader, unClosedTsFileReader), + priorityValue); + priorityValue++; + } + } + + // add reader for MemTable + if (overflowSeriesDataSource.hasRawChunk()) { + unSeqMergeReader.addReaderWithPriority( + new MemChunkReaderByTimestamp(overflowSeriesDataSource.getReadableMemChunk()), + priorityValue); + } + + // TODO add external sort when needed + return unSeqMergeReader; + } /** * This method is used to construct reader for merge process in IoTDB. To merge only one TsFile diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderByTimestamp.java index dbd6beb7da5d4..2a097c5c320ec 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderByTimestamp.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderByTimestamp.java @@ -16,17 +16,19 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.iotdb.db.query.reader.mem; import java.io.IOException; import java.util.Iterator; import org.apache.iotdb.db.engine.memtable.TimeValuePairSorter; +import org.apache.iotdb.db.query.reader.IReader; import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp; import org.apache.iotdb.db.utils.TimeValuePair; -import org.apache.iotdb.db.utils.TsPrimitiveType; import org.apache.iotdb.tsfile.read.common.BatchData; +import org.apache.iotdb.tsfile.utils.Pair; -public class MemChunkReaderByTimestamp implements EngineReaderByTimeStamp { +public class MemChunkReaderByTimestamp implements EngineReaderByTimeStamp, IReader { private Iterator timeValuePairIterator; private boolean hasCachedTimeValuePair; @@ -67,12 +69,12 @@ public void close() { // TODO consider change timeValuePairIterator to List structure, and use binary search instead of // sequential search @Override - public TsPrimitiveType getValueInTimestamp(long timestamp) throws IOException { + public Object getValueInTimestamp(long timestamp) throws IOException { while (hasNext()) { TimeValuePair timeValuePair = next(); long time = timeValuePair.getTimestamp(); if (time == timestamp) { - return timeValuePair.getValue(); + return timeValuePair.getValue().getValue(); } else if (time > timestamp) { hasCachedTimeValuePair = true; cachedTimeValuePair = timeValuePair; @@ -82,6 +84,18 @@ public TsPrimitiveType getValueInTimestamp(long timestamp) throws IOException { return null; } + @Override + public Pair getValueGtEqTimestamp(long timestamp) throws IOException { + while (hasNext()) { + TimeValuePair timeValuePair = next(); + long time = timeValuePair.getTimestamp(); + if (time >= timestamp) { + return new Pair<>(time, timeValuePair.getValue().getValue()); + } + } + return null; + } + @Override public boolean hasNextBatch() { return false; diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/EngineReaderByTimeStamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/EngineReaderByTimeStamp.java index 8ddf0add06b72..b7087fa964826 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/EngineReaderByTimeStamp.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/EngineReaderByTimeStamp.java @@ -16,17 +16,25 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.iotdb.db.query.reader.merge; import java.io.IOException; -import org.apache.iotdb.db.query.reader.IReader; -import org.apache.iotdb.db.utils.TsPrimitiveType; +import org.apache.iotdb.tsfile.utils.Pair; -public interface EngineReaderByTimeStamp extends IReader { +public interface EngineReaderByTimeStamp { /** * Given a timestamp, the reader is supposed to return the corresponding value in the timestamp. * If no value in this timestamp, null will be returned. */ - TsPrimitiveType getValueInTimestamp(long timestamp) throws IOException; + Object getValueInTimestamp(long timestamp) throws IOException; + + /** + * Given a timestamp, the reader is supposed to return the first value whose time is greater or + * equal to timestamp. If no point with time greater or equal to timestamp, null will be returned. + */ + Pair getValueGtEqTimestamp(long timestamp) throws IOException; + + void close() throws IOException; } diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/MergeElement.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/MergeElement.java new file mode 100644 index 0000000000000..33b9097eef7d3 --- /dev/null +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/MergeElement.java @@ -0,0 +1,64 @@ +/** + * 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.query.reader.merge; + +public class MergeElement implements Comparable { + + int index; + long time; + Object value; + Integer priority; + + public MergeElement(int index, long time, Object value, int priority) { + this.index = index; + this.time = time; + this.value = value; + this.priority = priority; + } + + @Override + public int compareTo(MergeElement o) { + + if (this.time > o.time) { + return 1; + } + + if (this.time < o.time) { + return -1; + } + + return o.priority.compareTo(this.priority); + } + + @Override + public boolean equals(Object o){ + if (o instanceof MergeElement){ + MergeElement element = (MergeElement) o; + if (this.time == element.time && this.priority.equals(element.priority)){ + return true; + } + } + return false; + } + + @Override + public int hashCode(){ + return (int) (time * 31 + priority.hashCode()); + } +} diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestamp.java index 244291a11856c..1551bf754730a 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestamp.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestamp.java @@ -1,54 +1,74 @@ /** - * 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 + * 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 + * 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 + * 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.query.reader.merge; import java.io.IOException; -import org.apache.iotdb.db.utils.TimeValuePair; -import org.apache.iotdb.db.utils.TsPrimitiveType; +import java.util.ArrayList; +import java.util.List; +import java.util.PriorityQueue; +import org.apache.iotdb.tsfile.utils.Pair; /** - * TODO the process of PriorityMergeReaderByTimestamp can be optimized. + *

+ * Usage: Get value in timestamp by sorting time-value pair in multiple readers with time and + * priority. (1) merge multiple chunk group readers in the unsequence file (2)merge sequence reader, + * unsequence reader and mem reader + *

*/ -public class PriorityMergeReaderByTimestamp extends PriorityMergeReader implements - EngineReaderByTimeStamp { +public class PriorityMergeReaderByTimestamp implements EngineReaderByTimeStamp { private boolean hasCachedTimeValuePair; - private TimeValuePair cachedTimeValuePair; + private Pair cachedTimeValuePair; + + private List readerList = new ArrayList<>(); + private List priorityList = new ArrayList<>(); + private PriorityQueue heap = new PriorityQueue<>(); + + /** + * The bigger the priority value is, the higher the priority of this reader is + */ + public void addReaderWithPriority(EngineReaderByTimeStamp reader, int priority) + throws IOException { + Pair timeValuePair = reader.getValueGtEqTimestamp(Long.MIN_VALUE); + + if (timeValuePair != null) { + heap.add( + new MergeElement(readerList.size(), timeValuePair.left, timeValuePair.right, priority)); + } + readerList.add(reader); + priorityList.add(priority); + } @Override - public TsPrimitiveType getValueInTimestamp(long timestamp) throws IOException { + public Object getValueInTimestamp(long timestamp) throws IOException { if (hasCachedTimeValuePair) { - if (cachedTimeValuePair.getTimestamp() == timestamp) { + if (cachedTimeValuePair.left == timestamp) { hasCachedTimeValuePair = false; - return cachedTimeValuePair.getValue(); - } else if (cachedTimeValuePair.getTimestamp() > timestamp) { + return cachedTimeValuePair.right; + } else if (cachedTimeValuePair.left > timestamp) { return null; } } while (hasNext()) { - cachedTimeValuePair = next(); - if (cachedTimeValuePair.getTimestamp() == timestamp) { + cachedTimeValuePair = next(timestamp); + if (cachedTimeValuePair.left == timestamp) { hasCachedTimeValuePair = false; - return cachedTimeValuePair.getValue(); - } else if (cachedTimeValuePair.getTimestamp() > timestamp) { + return cachedTimeValuePair.right; + } else if (cachedTimeValuePair.left > timestamp) { hasCachedTimeValuePair = true; return null; } @@ -56,4 +76,67 @@ public TsPrimitiveType getValueInTimestamp(long timestamp) throws IOException { return null; } + + @Override + public Pair getValueGtEqTimestamp(long timestamp) throws IOException { + + if (hasCachedTimeValuePair) { + if (cachedTimeValuePair.left >= timestamp) { + hasCachedTimeValuePair = false; + return cachedTimeValuePair; + } + } + + while (hasNext()) { + cachedTimeValuePair = next(timestamp); + if (cachedTimeValuePair.left >= timestamp) { + hasCachedTimeValuePair = false; + return cachedTimeValuePair; + } + } + + return null; + } + + @Override + public void close() throws IOException { + for (EngineReaderByTimeStamp reader : readerList) { + reader.close(); + } + } + + public boolean hasNext() { + return !heap.isEmpty(); + } + + /** + * Get the top element of the heap and update the element of the stack with a value whose time is greater than + * that timestamp + */ + private Pair next(long timestamp) throws IOException { + MergeElement top = heap.peek(); + updateHeap(top, timestamp); + return new Pair<>(top.time, top.value); + } + + /** + * This method is only used in test. + */ + public Pair next() throws IOException { + MergeElement top = heap.peek(); + updateHeap(top, top.time); + return new Pair<>(top.time, top.value); + } + + private void updateHeap(MergeElement top, long timestamp) throws IOException { + while (!heap.isEmpty() && heap.peek().time == top.time) { + MergeElement e = heap.poll(); + EngineReaderByTimeStamp reader = readerList.get(e.index); + Pair timeValuePair = reader.getValueGtEqTimestamp(timestamp); + if (timeValuePair != null) { + heap.add(new MergeElement(e.index, timeValuePair.left, timeValuePair.right, + priorityList.get(e.index))); + } + } + } } diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReaderByTimestamp.java new file mode 100644 index 0000000000000..278a7263fa5f8 --- /dev/null +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReaderByTimestamp.java @@ -0,0 +1,102 @@ +package org.apache.iotdb.db.query.reader.sequence; + +import java.io.IOException; +import java.util.List; +import org.apache.iotdb.db.engine.filenode.IntervalFileNode; +import org.apache.iotdb.db.query.control.FileReaderManager; +import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp; +import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData; +import org.apache.iotdb.tsfile.read.TsFileSequenceReader; +import org.apache.iotdb.tsfile.read.common.Path; +import org.apache.iotdb.tsfile.read.controller.ChunkLoader; +import org.apache.iotdb.tsfile.read.controller.ChunkLoaderImpl; +import org.apache.iotdb.tsfile.read.controller.MetadataQuerierByFileImpl; +import org.apache.iotdb.tsfile.read.reader.series.SeriesReaderByTimestamp; +import org.apache.iotdb.tsfile.utils.Pair; + +public class SealedTsFilesReaderByTimestamp implements EngineReaderByTimeStamp { + + private Path seriesPath; + private List sealedTsFiles; + private int usedIntervalFileIndex; + private SeriesReaderByTimestamp seriesReader; + + /** + * init with seriesPath and sealedTsFiles. + */ + public SealedTsFilesReaderByTimestamp(Path seriesPath, List sealedTsFiles) { + this.seriesPath = seriesPath; + this.sealedTsFiles = sealedTsFiles; + this.usedIntervalFileIndex = 0; + this.seriesReader = null; + } + + @Override + public Object getValueInTimestamp(long timestamp) throws IOException { + Object value = null; + if (seriesReader != null) { + value = seriesReader.getValueInTimestamp(timestamp); + if (value != null || seriesReader.hasNext()) { + return value; + } + } + constructReader(timestamp); + if (seriesReader != null) { + value = seriesReader.getValueInTimestamp(timestamp); + if (value != null || seriesReader.hasNext()) { + return value; + } + } + + return value; + } + + @Override + public Pair getValueGtEqTimestamp(long timestamp) throws IOException { + Object value = getValueInTimestamp(timestamp); + if (value != null) { + return new Pair<>(timestamp, value); + } + if (seriesReader != null && seriesReader.hasNext()) { + return seriesReader.next(); + } + return null; + } + + @Override + public void close() throws IOException { + // file streams are managed uniformly. + } + + // construct reader from the file that might overlap this timestamp + private void constructReader(long timestamp) throws IOException { + while (usedIntervalFileIndex < sealedTsFiles.size()) { + if (singleTsFileSatisfied(sealedTsFiles.get(usedIntervalFileIndex), timestamp)) { + initSingleTsFileReader(sealedTsFiles.get(usedIntervalFileIndex)); + } + usedIntervalFileIndex++; + } + } + + /** + * Judge whether the file should be skipped + */ + private boolean singleTsFileSatisfied(IntervalFileNode fileNode, long timestamp) { + long endTime = fileNode.getEndTime(seriesPath.getDevice()); + return endTime >= timestamp; + } + + private void initSingleTsFileReader(IntervalFileNode fileNode) throws IOException { + + // to avoid too many opened files + TsFileSequenceReader tsFileReader = FileReaderManager.getInstance() + .get(fileNode.getFilePath(), false); + + MetadataQuerierByFileImpl metadataQuerier = new MetadataQuerierByFileImpl(tsFileReader); + List metaDataList = metadataQuerier.getChunkMetaDataList(seriesPath); + ChunkLoader chunkLoader = new ChunkLoaderImpl(tsFileReader); + + seriesReader = new SeriesReaderByTimestamp(chunkLoader, metaDataList); + + } +} diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReaderByTimestamp.java new file mode 100644 index 0000000000000..a6a7984c67473 --- /dev/null +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReaderByTimestamp.java @@ -0,0 +1,104 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license + * agreements. See the NOTICE file distributed with this work for additional information regarding + * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with the License. You may obtain + * a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iotdb.db.query.reader.sequence; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import org.apache.iotdb.db.engine.querycontext.GlobalSortedSeriesDataSource; +import org.apache.iotdb.db.query.reader.mem.MemChunkReaderByTimestamp; +import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp; +import org.apache.iotdb.tsfile.utils.Pair; + +public class SequenceDataReaderByTimestamp implements EngineReaderByTimeStamp { + + private List seriesReaders; + private int nextSeriesReaderIndex; + private EngineReaderByTimeStamp currentSeriesReader; + + private boolean hasCached; + private Pair cachedTimeValuePair; + + /** + * init with globalSortedSeriesDataSource and filter. + */ + public SequenceDataReaderByTimestamp(GlobalSortedSeriesDataSource sources) + throws IOException { + seriesReaders = new ArrayList<>(); + + nextSeriesReaderIndex = 0; + + // add reader for sealed TsFiles + if (sources.hasSealedTsFiles()) { + seriesReaders.add( + new SealedTsFilesReaderByTimestamp(sources.getSeriesPath(), sources.getSealedTsFiles())); + } + + // add reader for unSealed TsFile + if (sources.hasUnsealedTsFile()) { + seriesReaders.add(new UnSealedTsFilesReaderByTimestamp(sources.getUnsealedTsFile())); + } + + // add data in memTable + if (sources.hasRawSeriesChunk()) { + seriesReaders.add(new MemChunkReaderByTimestamp(sources.getReadableChunk())); + } + + } + + @Override + public Object getValueInTimestamp(long timestamp) throws IOException { + cachedTimeValuePair = getValueGtEqTimestamp(timestamp); + if (cachedTimeValuePair == null || cachedTimeValuePair.left == timestamp) { + return cachedTimeValuePair; + } else { + hasCached = true; + return null; + } + } + + @Override + public Pair getValueGtEqTimestamp(long timestamp) throws IOException { + if (hasCached && cachedTimeValuePair.left >= timestamp) { + hasCached = false; + return cachedTimeValuePair; + } + + if(currentSeriesReader != null){ + cachedTimeValuePair = currentSeriesReader.getValueGtEqTimestamp(timestamp); + if (cachedTimeValuePair != null) { + return cachedTimeValuePair; + } + } + + while (nextSeriesReaderIndex < seriesReaders.size()) { + currentSeriesReader = seriesReaders.get(nextSeriesReaderIndex++); + cachedTimeValuePair = currentSeriesReader.getValueGtEqTimestamp(timestamp); + if (cachedTimeValuePair != null) { + return cachedTimeValuePair; + } + } + + return null; + } + + @Override + public void close() throws IOException { + for (EngineReaderByTimeStamp seriesReader : seriesReaders) { + seriesReader.close(); + } + } +} diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFilesReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFilesReaderByTimestamp.java new file mode 100644 index 0000000000000..49b4e0bf45442 --- /dev/null +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFilesReaderByTimestamp.java @@ -0,0 +1,73 @@ +/** + * 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.query.reader.sequence; + +import java.io.IOException; +import org.apache.iotdb.db.engine.querycontext.UnsealedTsFile; +import org.apache.iotdb.db.query.control.FileReaderManager; +import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp; +import org.apache.iotdb.tsfile.read.TsFileSequenceReader; +import org.apache.iotdb.tsfile.read.common.Path; +import org.apache.iotdb.tsfile.read.controller.ChunkLoader; +import org.apache.iotdb.tsfile.read.controller.ChunkLoaderImpl; +import org.apache.iotdb.tsfile.read.reader.series.SeriesReaderByTimestamp; +import org.apache.iotdb.tsfile.utils.Pair; + +public class UnSealedTsFilesReaderByTimestamp implements EngineReaderByTimeStamp { + + protected Path seriesPath; + private SeriesReaderByTimestamp unSealedReader; + + /** + * Construct funtion for UnSealedTsFileReader. + * + * @param unsealedTsFile -param to initial + */ + public UnSealedTsFilesReaderByTimestamp(UnsealedTsFile unsealedTsFile) throws IOException { + + TsFileSequenceReader unClosedTsFileReader = FileReaderManager.getInstance() + .get(unsealedTsFile.getFilePath(), + true); + ChunkLoader chunkLoader = new ChunkLoaderImpl(unClosedTsFileReader); + unSealedReader = new SeriesReaderByTimestamp(chunkLoader, + unsealedTsFile.getChunkMetaDataList()); + + + } + + @Override + public Object getValueInTimestamp(long timestamp) throws IOException { + return unSealedReader.getValueInTimestamp(timestamp); + } + + @Override + public Pair getValueGtEqTimestamp(long timestamp) throws IOException { + Object value = unSealedReader.getValueInTimestamp(timestamp); + if (value != null) { + return new Pair<>(timestamp, value); + } + if (unSealedReader.hasNext()) { + return unSealedReader.next(); + } + return null; + } + + @Override + public void close() throws IOException { + + } + +} diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/unsequence/EngineChunkReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/unsequence/EngineChunkReaderByTimestamp.java new file mode 100644 index 0000000000000..f163779760ebd --- /dev/null +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/unsequence/EngineChunkReaderByTimestamp.java @@ -0,0 +1,97 @@ +/** + * 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.query.reader.unsequence; + +import java.io.IOException; +import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp; +import org.apache.iotdb.tsfile.read.TsFileSequenceReader; +import org.apache.iotdb.tsfile.read.common.BatchData; +import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReaderByTimestamp; +import org.apache.iotdb.tsfile.utils.Pair; + +public class EngineChunkReaderByTimestamp implements EngineReaderByTimeStamp { + + private ChunkReaderByTimestamp chunkReader; + private BatchData data; + + /** + * Each EngineChunkReader has a corresponding UnClosedTsFileReader, when EngineChunkReader is + * closed, UnClosedTsFileReader also should be closed in meanwhile. + */ + private TsFileSequenceReader unClosedTsFileReader; + + public EngineChunkReaderByTimestamp(ChunkReaderByTimestamp chunkReader, + TsFileSequenceReader unClosedTsFileReader) { + this.chunkReader = chunkReader; + this.unClosedTsFileReader = unClosedTsFileReader; + } + + /** + * get value with time equals timestamp. If there is no such point, return null. + */ + @Override + public Object getValueInTimestamp(long timestamp) throws IOException { + + while (data != null) { + Object value = data.getValueInTimestamp(timestamp); + if (value != null) { + return value; + } + if (data.hasNext()) { + return null; + } else { + chunkReader.setCurrentTimestamp(timestamp); + if (chunkReader.hasNextBatch()) { + data = chunkReader.nextBatch(); + } else { + return null; + } + } + } + + return null; + } + + @Override + public Pair getValueGtEqTimestamp(long timestamp) throws IOException { + + Object value = getValueInTimestamp(timestamp); + if (value != null) { + return new Pair(timestamp, value); + } + while (data != null) { + if (data.hasNext()) { + Pair tvPair = new Pair(data.currentTime(), data.currentValue()); + data.next(); + return tvPair; + } else { + chunkReader.setCurrentTimestamp(timestamp); + if (chunkReader.hasNextBatch()) { + data = chunkReader.nextBatch(); + } else { + return null; + } + } + } + return null; + } + + @Override + public void close() throws IOException { + this.chunkReader.close(); + this.unClosedTsFileReader.close(); + } +} diff --git a/iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestampTest.java b/iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestampTest.java index 03771e80fa748..8c3d3dc661846 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestampTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestampTest.java @@ -1,19 +1,15 @@ /** - * 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 + * 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 + * 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 + * 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.query.reader.merge; @@ -23,10 +19,12 @@ import java.util.Iterator; import java.util.List; import java.util.Random; +import org.apache.iotdb.db.query.reader.IReader; import org.apache.iotdb.db.utils.TimeValuePair; import org.apache.iotdb.db.utils.TsPrimitiveType; import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; import org.apache.iotdb.tsfile.read.common.BatchData; +import org.apache.iotdb.tsfile.utils.Pair; import org.junit.Assert; import org.junit.Test; @@ -50,22 +48,22 @@ public void test() throws IOException { Random random = new Random(); for (long time = 4; time < 1080 + 200 * 13 + 600; ) { - TsPrimitiveType value = priorityReader.getValueInTimestamp(time); - // System.out.println("time = " + time + " value = " + value); + Long value = (Long) priorityReader.getValueInTimestamp(time); + //System.out.println("time = " + time + " value = " + value); if (time < 100) { // null Assert.assertNull(value); } else if (time < 850) { // reader 1 if ((time - 100) % 5 == 0) { - Assert.assertEquals(time % 11, value.getLong()); + Assert.assertEquals(time % 11, value.longValue()); } } else if (time < 1080) { // reader 2, reader 1 if (time >= 850 && (time - 850) % 7 == 0) { - Assert.assertEquals(time % 19, value.getLong()); + Assert.assertEquals(time % 19, value.longValue()); } else if (time < 1100 && (time - 100) % 5 == 0) { - Assert.assertEquals(time % 11, value.getLong()); + Assert.assertEquals(time % 11, value.longValue()); } else { Assert.assertNull(value); } @@ -73,11 +71,11 @@ public void test() throws IOException { } else if (time < 1080 + 200 * 13) { // reader 3, reader 2, reader 1 if (time >= 1080 && (time - 1080) % 13 == 0) { - Assert.assertEquals(time % 31, value.getLong()); + Assert.assertEquals(time % 31, value.longValue()); } else if (time < 850 + 200 * 7 && (time - 850) % 7 == 0) { - Assert.assertEquals(time % 19, value.getLong()); + Assert.assertEquals(time % 19, value.longValue()); } else if (time < 1100 && (time - 100) % 5 == 0) { - Assert.assertEquals(time % 11, value.getLong()); + Assert.assertEquals(time % 11, value.longValue()); } else { Assert.assertNull(value); } @@ -89,9 +87,9 @@ public void test() throws IOException { } while (priorityReader.hasNext()) { - TimeValuePair timeValuePair = priorityReader.next(); - long time = timeValuePair.getTimestamp(); - long value = timeValuePair.getValue().getLong(); + Pair timeValuePair = priorityReader.next(); + long time = timeValuePair.left; + long value = (long) timeValuePair.right; if (time < 850) { Assert.assertEquals(time % 11, value); } else if (time < 1080) { @@ -104,7 +102,8 @@ public void test() throws IOException { } - public static class FakedPrioritySeriesReaderByTimestamp implements EngineReaderByTimeStamp { + public static class FakedPrioritySeriesReaderByTimestamp implements EngineReaderByTimeStamp, + IReader { private Iterator iterator; private long currentTimeStamp = Long.MIN_VALUE; @@ -159,17 +158,17 @@ public void close() { } @Override - public TsPrimitiveType getValueInTimestamp(long timestamp) throws IOException { + public Object getValueInTimestamp(long timestamp) throws IOException { this.currentTimeStamp = timestamp; if (hasCachedTimeValuePair && cachedTimeValuePair.getTimestamp() == timestamp) { hasCachedTimeValuePair = false; - return cachedTimeValuePair.getValue(); + return cachedTimeValuePair.getValue().getValue(); } if (hasNext()) { cachedTimeValuePair = next(); if (cachedTimeValuePair.getTimestamp() == timestamp) { - return cachedTimeValuePair.getValue(); + return cachedTimeValuePair.getValue().getValue(); } else if (cachedTimeValuePair.getTimestamp() > timestamp) { hasCachedTimeValuePair = true; } @@ -177,6 +176,19 @@ public TsPrimitiveType getValueInTimestamp(long timestamp) throws IOException { return null; } + @Override + public Pair getValueGtEqTimestamp(long timestamp) throws IOException { + Object value = getValueInTimestamp(timestamp); + if (value != null) { + return new Pair<>(timestamp, value); + } + if (hasNext()) { + TimeValuePair timeValuePair = next(); + return new Pair<>(timeValuePair.getTimestamp(), timeValuePair.getValue().getValue()); + } + return null; + } + @Override public boolean hasNextBatch() { return false; 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 ffb37e263903f..bbf8f3e44f78c 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 @@ -19,6 +19,7 @@ package org.apache.iotdb.tsfile.read.common; import java.util.ArrayList; +import java.util.Arrays; 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; @@ -527,4 +528,28 @@ public void setAnObject(int idx, Comparable v) { public int length() { return this.timeLength; } + + public Object getValueInTimestamp(long key){ + int max, min, mid; + min = curIdx ; + max = length() - 1; + mid = (max + min) / 2; + while ( max >= min) { + long midKey = timeRet.get(mid / timeCapacity)[mid % timeCapacity]; + if (key > midKey) { + min = mid + 1; + } else if (key < midKey) { + max = mid - 1; + } + else { + curIdx = mid; + Object value = currentValue(); + curIdx++; + return value; + } + mid = (min + max) / 2; + } + curIdx = min; + return null; + } } diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReaderByTimestamp.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReaderByTimestamp.java index 0c33e0a37abae..b85263fb925ff 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReaderByTimestamp.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReaderByTimestamp.java @@ -29,6 +29,11 @@ public ChunkReaderByTimestamp(Chunk chunk) { super(chunk); } + public ChunkReaderByTimestamp(Chunk chunk, long maxTombstoneTime) { + super(chunk); + setMaxTombstoneTime(maxTombstoneTime); + } + @Override public boolean pageSatisfied(PageHeader pageHeader) { long maxTimestamp = pageHeader.getMaxTimestamp(); diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/SeriesReaderByTimestamp.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/SeriesReaderByTimestamp.java index 2d2e11ca1cfe9..80b58b6e20305 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/SeriesReaderByTimestamp.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/SeriesReaderByTimestamp.java @@ -27,6 +27,7 @@ import org.apache.iotdb.tsfile.read.controller.ChunkLoader; import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReader; import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReaderByTimestamp; +import org.apache.iotdb.tsfile.utils.Pair; /** *

@@ -34,7 +35,7 @@ * a series with given timestamps. *

*/ -public class SeriesReaderByTimestamp { +public class SeriesReaderByTimestamp{ protected ChunkLoader chunkLoader; protected List chunkMetaDataList; @@ -86,8 +87,10 @@ public Object getValueInTimestamp(long timestamp) throws IOException { } if (data.hasNext()) { + Object value = data.currentValue(); if (data.currentTime() == timestamp) { - return data.currentValue(); + data.next(); + return value; } return null; } else { @@ -102,6 +105,46 @@ public Object getValueInTimestamp(long timestamp) throws IOException { return null; } + /** + * Judge if the series reader has next time-value pair. + * @return true if has next, false if not. + * @throws IOException + */ + public boolean hasNext() throws IOException { + + if(chunkReader != null){ + if(data != null && data.hasNext()){ + return true; + } + while (chunkReader.hasNextBatch()){ + data = chunkReader.nextBatch(); + if(data != null && data.hasNext()){ + return true; + } + } + } + + while(constructNextSatisfiedChunkReader()){ + while (chunkReader.hasNextBatch()){ + data = chunkReader.nextBatch(); + if(data != null && data.hasNext()){ + return true; + } + } + } + return false; + } + + /** + * Get next point. + * @return Pair: left is timestamp, right is value. + */ + public Pair next(){ + Pair pair = new Pair<>(data.currentTime(), data.currentValue()); + data.next(); + return pair; + } + private boolean constructNextSatisfiedChunkReader() throws IOException { while (currentChunkIndex < chunkMetaDataList.size()) { ChunkMetaData chunkMetaData = chunkMetaDataList.get(currentChunkIndex++); From 2eab632b034c060f21eeba867064d9e2003f2137 Mon Sep 17 00:00:00 2001 From: suyue <2335813255@qq.com> Date: Thu, 28 Feb 2019 22:33:29 +0800 Subject: [PATCH 2/8] optimizing queries by reducing the number of objects created --- .../reader/mem/MemChunkReaderByTimestamp.java | 13 -- .../reader/merge/EngineReaderByTimeStamp.java | 7 +- .../db/query/reader/merge/MergeElement.java | 64 ---------- .../merge/PriorityMergeReaderByTimestamp.java | 116 +++++------------- .../SealedTsFilesReaderByTimestamp.java | 13 +- .../SequenceDataReaderByTimestamp.java | 45 +++---- .../UnSealedTsFilesReaderByTimestamp.java | 12 +- .../EngineChunkReaderByTimestamp.java | 27 ++-- .../PriorityMergeReaderByTimestampTest.java | 34 ++--- .../series/SeriesReaderByTimestamp.java | 10 -- 10 files changed, 70 insertions(+), 271 deletions(-) delete mode 100644 iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/MergeElement.java diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderByTimestamp.java index 2a097c5c320ec..2770b324c870c 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderByTimestamp.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderByTimestamp.java @@ -26,7 +26,6 @@ import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp; import org.apache.iotdb.db.utils.TimeValuePair; import org.apache.iotdb.tsfile.read.common.BatchData; -import org.apache.iotdb.tsfile.utils.Pair; public class MemChunkReaderByTimestamp implements EngineReaderByTimeStamp, IReader { @@ -84,18 +83,6 @@ public Object getValueInTimestamp(long timestamp) throws IOException { return null; } - @Override - public Pair getValueGtEqTimestamp(long timestamp) throws IOException { - while (hasNext()) { - TimeValuePair timeValuePair = next(); - long time = timeValuePair.getTimestamp(); - if (time >= timestamp) { - return new Pair<>(time, timeValuePair.getValue().getValue()); - } - } - return null; - } - @Override public boolean hasNextBatch() { return false; diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/EngineReaderByTimeStamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/EngineReaderByTimeStamp.java index b7087fa964826..ca58f4fad7d87 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/EngineReaderByTimeStamp.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/EngineReaderByTimeStamp.java @@ -20,7 +20,6 @@ package org.apache.iotdb.db.query.reader.merge; import java.io.IOException; -import org.apache.iotdb.tsfile.utils.Pair; public interface EngineReaderByTimeStamp { @@ -30,11 +29,7 @@ public interface EngineReaderByTimeStamp { */ Object getValueInTimestamp(long timestamp) throws IOException; - /** - * Given a timestamp, the reader is supposed to return the first value whose time is greater or - * equal to timestamp. If no point with time greater or equal to timestamp, null will be returned. - */ - Pair getValueGtEqTimestamp(long timestamp) throws IOException; + boolean hasNext() throws IOException; void close() throws IOException; } diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/MergeElement.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/MergeElement.java deleted file mode 100644 index 33b9097eef7d3..0000000000000 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/MergeElement.java +++ /dev/null @@ -1,64 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iotdb.db.query.reader.merge; - -public class MergeElement implements Comparable { - - int index; - long time; - Object value; - Integer priority; - - public MergeElement(int index, long time, Object value, int priority) { - this.index = index; - this.time = time; - this.value = value; - this.priority = priority; - } - - @Override - public int compareTo(MergeElement o) { - - if (this.time > o.time) { - return 1; - } - - if (this.time < o.time) { - return -1; - } - - return o.priority.compareTo(this.priority); - } - - @Override - public boolean equals(Object o){ - if (o instanceof MergeElement){ - MergeElement element = (MergeElement) o; - if (this.time == element.time && this.priority.equals(element.priority)){ - return true; - } - } - return false; - } - - @Override - public int hashCode(){ - return (int) (time * 31 + priority.hashCode()); - } -} diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestamp.java index 1551bf754730a..c6b6b8b0a586e 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestamp.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestamp.java @@ -12,13 +12,12 @@ * or implied. See the License for the specific language governing permissions and limitations * under the License. */ + package org.apache.iotdb.db.query.reader.merge; import java.io.IOException; import java.util.ArrayList; import java.util.List; -import java.util.PriorityQueue; -import org.apache.iotdb.tsfile.utils.Pair; /** *

@@ -29,73 +28,40 @@ */ public class PriorityMergeReaderByTimestamp implements EngineReaderByTimeStamp { - private boolean hasCachedTimeValuePair; - private Pair cachedTimeValuePair; - private List readerList = new ArrayList<>(); private List priorityList = new ArrayList<>(); - private PriorityQueue heap = new PriorityQueue<>(); /** - * The bigger the priority value is, the higher the priority of this reader is + * The bigger the priority value is, the higher the priority of this reader is. */ - public void addReaderWithPriority(EngineReaderByTimeStamp reader, int priority) - throws IOException { - Pair timeValuePair = reader.getValueGtEqTimestamp(Long.MIN_VALUE); - - if (timeValuePair != null) { - heap.add( - new MergeElement(readerList.size(), timeValuePair.left, timeValuePair.right, priority)); - } - readerList.add(reader); - priorityList.add(priority); - } - - @Override - public Object getValueInTimestamp(long timestamp) throws IOException { - - if (hasCachedTimeValuePair) { - if (cachedTimeValuePair.left == timestamp) { - hasCachedTimeValuePair = false; - return cachedTimeValuePair.right; - } else if (cachedTimeValuePair.left > timestamp) { - return null; + public void addReaderWithPriority(EngineReaderByTimeStamp reader, int priority) { + int size = readerList.size(); + boolean hasAdded = false; + + //sort by priority, elements with low priority are at front of list. + for (int i = 0; i < size; i++) { + if (priority < priorityList.get(i)) { + readerList.set(i, reader); + priorityList.set(i, priority); + hasAdded = true; } } - - while (hasNext()) { - cachedTimeValuePair = next(timestamp); - if (cachedTimeValuePair.left == timestamp) { - hasCachedTimeValuePair = false; - return cachedTimeValuePair.right; - } else if (cachedTimeValuePair.left > timestamp) { - hasCachedTimeValuePair = true; - return null; - } + if (!hasAdded) { + readerList.add(reader); + priorityList.add(priority); } - - return null; } @Override - public Pair getValueGtEqTimestamp(long timestamp) throws IOException { - - if (hasCachedTimeValuePair) { - if (cachedTimeValuePair.left >= timestamp) { - hasCachedTimeValuePair = false; - return cachedTimeValuePair; - } - } - - while (hasNext()) { - cachedTimeValuePair = next(timestamp); - if (cachedTimeValuePair.left >= timestamp) { - hasCachedTimeValuePair = false; - return cachedTimeValuePair; + public Object getValueInTimestamp(long timestamp) throws IOException { + Object value = null; + for (int i = readerList.size() - 1; i >= 0; i--) { + value = readerList.get(i).getValueInTimestamp(timestamp); + if (value != null) { + return value; } } - - return null; + return value; } @Override @@ -105,38 +71,14 @@ public void close() throws IOException { } } - public boolean hasNext() { - return !heap.isEmpty(); - } - - /** - * Get the top element of the heap and update the element of the stack with a value whose time is greater than - * that timestamp - */ - private Pair next(long timestamp) throws IOException { - MergeElement top = heap.peek(); - updateHeap(top, timestamp); - return new Pair<>(top.time, top.value); - } - - /** - * This method is only used in test. - */ - public Pair next() throws IOException { - MergeElement top = heap.peek(); - updateHeap(top, top.time); - return new Pair<>(top.time, top.value); - } - - private void updateHeap(MergeElement top, long timestamp) throws IOException { - while (!heap.isEmpty() && heap.peek().time == top.time) { - MergeElement e = heap.poll(); - EngineReaderByTimeStamp reader = readerList.get(e.index); - Pair timeValuePair = reader.getValueGtEqTimestamp(timestamp); - if (timeValuePair != null) { - heap.add(new MergeElement(e.index, timeValuePair.left, timeValuePair.right, - priorityList.get(e.index))); + @Override + public boolean hasNext() throws IOException { + for (int i = readerList.size() - 1; i >= 0; i--) { + if (readerList.get(i).hasNext()) { + return true; } } + return false; } + } diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReaderByTimestamp.java index 49eeb71592013..37a78b84df2ae 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReaderByTimestamp.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReaderByTimestamp.java @@ -34,7 +34,6 @@ import org.apache.iotdb.tsfile.read.controller.ChunkLoaderImpl; import org.apache.iotdb.tsfile.read.controller.MetadataQuerierByFileImpl; import org.apache.iotdb.tsfile.read.reader.series.SeriesReaderByTimestamp; -import org.apache.iotdb.tsfile.utils.Pair; public class SealedTsFilesReaderByTimestamp implements EngineReaderByTimeStamp { @@ -77,15 +76,11 @@ public Object getValueInTimestamp(long timestamp) throws IOException { } @Override - public Pair getValueGtEqTimestamp(long timestamp) throws IOException { - Object value = getValueInTimestamp(timestamp); - if (value != null) { - return new Pair<>(timestamp, value); - } + public boolean hasNext() throws IOException { if (seriesReader != null && seriesReader.hasNext()) { - return seriesReader.next(); + return true; } - return null; + return usedIntervalFileIndex + 1 < sealedTsFiles.size(); } @Override @@ -104,7 +99,7 @@ private void constructReader(long timestamp) throws IOException { } /** - * Judge whether the file should be skipped + * Judge whether the file should be skipped. */ private boolean singleTsFileSatisfied(IntervalFileNode fileNode, long timestamp) { long endTime = fileNode.getEndTime(seriesPath.getDevice()); diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReaderByTimestamp.java index b49adbffe14fa..19923a763da4b 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReaderByTimestamp.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReaderByTimestamp.java @@ -22,7 +22,6 @@ import org.apache.iotdb.db.query.context.QueryContext; import org.apache.iotdb.db.query.reader.mem.MemChunkReaderByTimestamp; import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp; -import org.apache.iotdb.tsfile.utils.Pair; public class SequenceDataReaderByTimestamp implements EngineReaderByTimeStamp { @@ -30,9 +29,6 @@ public class SequenceDataReaderByTimestamp implements EngineReaderByTimeStamp { private int nextSeriesReaderIndex; private EngineReaderByTimeStamp currentSeriesReader; - private boolean hasCached; - private Pair cachedTimeValuePair; - /** * init with globalSortedSeriesDataSource and filter. */ @@ -63,40 +59,35 @@ public SequenceDataReaderByTimestamp(GlobalSortedSeriesDataSource sources, Query @Override public Object getValueInTimestamp(long timestamp) throws IOException { - cachedTimeValuePair = getValueGtEqTimestamp(timestamp); - if (cachedTimeValuePair == null || cachedTimeValuePair.left == timestamp) { - return cachedTimeValuePair; - } else { - hasCached = true; - return null; - } - } - - @Override - public Pair getValueGtEqTimestamp(long timestamp) throws IOException { - if (hasCached && cachedTimeValuePair.left >= timestamp) { - hasCached = false; - return cachedTimeValuePair; - } - + Object value = null; if (currentSeriesReader != null) { - cachedTimeValuePair = currentSeriesReader.getValueGtEqTimestamp(timestamp); - if (cachedTimeValuePair != null) { - return cachedTimeValuePair; + value = currentSeriesReader.getValueInTimestamp(timestamp); + if (value != null || currentSeriesReader.hasNext()) { + return value; } } while (nextSeriesReaderIndex < seriesReaders.size()) { currentSeriesReader = seriesReaders.get(nextSeriesReaderIndex++); - cachedTimeValuePair = currentSeriesReader.getValueGtEqTimestamp(timestamp); - if (cachedTimeValuePair != null) { - return cachedTimeValuePair; + if (currentSeriesReader != null) { + value = currentSeriesReader.getValueInTimestamp(timestamp); + if (value != null || currentSeriesReader.hasNext()) { + return value; + } } } + return value; + } - return null; + @Override + public boolean hasNext() throws IOException { + if (currentSeriesReader != null && currentSeriesReader.hasNext()) { + return true; + } + return nextSeriesReaderIndex < seriesReaders.size(); } + @Override public void close() throws IOException { for (EngineReaderByTimeStamp seriesReader : seriesReaders) { diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFilesReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFilesReaderByTimestamp.java index 49b4e0bf45442..251dc799dcd57 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFilesReaderByTimestamp.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFilesReaderByTimestamp.java @@ -24,7 +24,6 @@ import org.apache.iotdb.tsfile.read.controller.ChunkLoader; import org.apache.iotdb.tsfile.read.controller.ChunkLoaderImpl; import org.apache.iotdb.tsfile.read.reader.series.SeriesReaderByTimestamp; -import org.apache.iotdb.tsfile.utils.Pair; public class UnSealedTsFilesReaderByTimestamp implements EngineReaderByTimeStamp { @@ -54,15 +53,8 @@ public Object getValueInTimestamp(long timestamp) throws IOException { } @Override - public Pair getValueGtEqTimestamp(long timestamp) throws IOException { - Object value = unSealedReader.getValueInTimestamp(timestamp); - if (value != null) { - return new Pair<>(timestamp, value); - } - if (unSealedReader.hasNext()) { - return unSealedReader.next(); - } - return null; + public boolean hasNext() throws IOException { + return unSealedReader.hasNext(); } @Override diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/unsequence/EngineChunkReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/unsequence/EngineChunkReaderByTimestamp.java index f163779760ebd..286da26d7dcaa 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/unsequence/EngineChunkReaderByTimestamp.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/unsequence/EngineChunkReaderByTimestamp.java @@ -20,7 +20,6 @@ import org.apache.iotdb.tsfile.read.TsFileSequenceReader; import org.apache.iotdb.tsfile.read.common.BatchData; import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReaderByTimestamp; -import org.apache.iotdb.tsfile.utils.Pair; public class EngineChunkReaderByTimestamp implements EngineReaderByTimeStamp { @@ -66,27 +65,15 @@ public Object getValueInTimestamp(long timestamp) throws IOException { } @Override - public Pair getValueGtEqTimestamp(long timestamp) throws IOException { - - Object value = getValueInTimestamp(timestamp); - if (value != null) { - return new Pair(timestamp, value); + public boolean hasNext() throws IOException { + if (data != null & data.hasNext()) { + return true; } - while (data != null) { - if (data.hasNext()) { - Pair tvPair = new Pair(data.currentTime(), data.currentValue()); - data.next(); - return tvPair; - } else { - chunkReader.setCurrentTimestamp(timestamp); - if (chunkReader.hasNextBatch()) { - data = chunkReader.nextBatch(); - } else { - return null; - } - } + if (chunkReader != null && chunkReader.hasNextBatch()) { + data = chunkReader.nextBatch(); + return true; } - return null; + return false; } @Override diff --git a/iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestampTest.java b/iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestampTest.java index 8c3d3dc661846..e543764438a0c 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestampTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestampTest.java @@ -24,7 +24,6 @@ import org.apache.iotdb.db.utils.TsPrimitiveType; import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; import org.apache.iotdb.tsfile.read.common.BatchData; -import org.apache.iotdb.tsfile.utils.Pair; import org.junit.Assert; import org.junit.Test; @@ -86,18 +85,16 @@ public void test() throws IOException { time += random.nextInt(50) + 1; } - while (priorityReader.hasNext()) { - Pair timeValuePair = priorityReader.next(); - long time = timeValuePair.left; - long value = (long) timeValuePair.right; - if (time < 850) { - Assert.assertEquals(time % 11, value); - } else if (time < 1080) { - Assert.assertEquals(time % 19, value); - } else { - Assert.assertEquals(time % 31, value); + for(long i = 1; i < 1080 + 200 * 13; i++){ + Object value = priorityReader.getValueInTimestamp(i); + if(i >= 1080 && (i-1080) % 13 == 0){ + Assert.assertEquals(i % 31, value); + } else if( i >= 850 && i < 850 + 200 * 7 && (i-850) % 7 == 0 ){ + Assert.assertEquals(i % 19, value); + } else if( i >= 100 &&i < 100 + 200 * 5 && (i-100) % 5 == 0 ){ + Assert.assertEquals(i % 11, value); } - cnt++; + Assert.assertNull(value); } } @@ -176,19 +173,6 @@ public Object getValueInTimestamp(long timestamp) throws IOException { return null; } - @Override - public Pair getValueGtEqTimestamp(long timestamp) throws IOException { - Object value = getValueInTimestamp(timestamp); - if (value != null) { - return new Pair<>(timestamp, value); - } - if (hasNext()) { - TimeValuePair timeValuePair = next(); - return new Pair<>(timeValuePair.getTimestamp(), timeValuePair.getValue().getValue()); - } - return null; - } - @Override public boolean hasNextBatch() { return false; diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/SeriesReaderByTimestamp.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/SeriesReaderByTimestamp.java index d216a6999e463..1674dd8a3da52 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/SeriesReaderByTimestamp.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/SeriesReaderByTimestamp.java @@ -135,16 +135,6 @@ public boolean hasNext() throws IOException { return false; } - /** - * Get next point. - * @return Pair: left is timestamp, right is value. - */ - public Pair next(){ - Pair pair = new Pair<>(data.currentTime(), data.currentValue()); - data.next(); - return pair; - } - private boolean constructNextSatisfiedChunkReader() throws IOException { while (currentChunkIndex < chunkMetaDataList.size()) { ChunkMetaData chunkMetaData = chunkMetaDataList.get(currentChunkIndex++); From b0512653561d7660c7761754798c2a834dc2d42a Mon Sep 17 00:00:00 2001 From: suyue <2335813255@qq.com> Date: Fri, 1 Mar 2019 20:15:03 +0800 Subject: [PATCH 3/8] revision based on PR review opinion --- .../EngineExecutorWithTimeGenerator.java | 23 +++++++++------- .../db/query/factory/SeriesReaderFactory.java | 2 +- .../merge/PriorityMergeReaderByTimestamp.java | 26 +++++++++++-------- .../SealedTsFilesReaderByTimestamp.java | 21 ++++++++++----- .../SequenceDataReaderByTimestamp.java | 8 +++++- .../PriorityMergeReaderByTimestampTest.java | 13 ---------- .../iotdb/tsfile/read/common/BatchData.java | 26 +++++++------------ 7 files changed, 60 insertions(+), 59 deletions(-) diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineExecutorWithTimeGenerator.java b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineExecutorWithTimeGenerator.java index cab776437d0a3..45fc0b4e741e7 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineExecutorWithTimeGenerator.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/executor/EngineExecutorWithTimeGenerator.java @@ -1,17 +1,22 @@ /** - * Licensed to the Apache Software Foundation (ASF) under one or more contributor license - * agreements. See the NOTICE file distributed with this work for additional information regarding - * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with the License. You may obtain - * a copy of the License at + * 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 + * 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 + * 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.query.executor; import java.io.IOException; diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/factory/SeriesReaderFactory.java b/iotdb/src/main/java/org/apache/iotdb/db/query/factory/SeriesReaderFactory.java index 88146c6491328..f9e43b3b12b4f 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/factory/SeriesReaderFactory.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/factory/SeriesReaderFactory.java @@ -126,7 +126,7 @@ public PriorityMergeReader createUnSeqMergeReader( } /** - * This method is used to create overflow insert reader by timestamp for IoTDB request, such as + * This method is used to create unsequence insert reader by timestamp for IoTDB request, such as * query, aggregation and groupby request. */ public PriorityMergeReaderByTimestamp createUnSeqMergeReaderByTimestamp( diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestamp.java index c6b6b8b0a586e..e18cc1d7cadd4 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestamp.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestamp.java @@ -1,15 +1,19 @@ /** - * 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 + * 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 + * 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 + * 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. */ @@ -41,8 +45,8 @@ public void addReaderWithPriority(EngineReaderByTimeStamp reader, int priority) //sort by priority, elements with low priority are at front of list. for (int i = 0; i < size; i++) { if (priority < priorityList.get(i)) { - readerList.set(i, reader); - priorityList.set(i, priority); + readerList.add(i, reader); + priorityList.add(i, priority); hasAdded = true; } } diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReaderByTimestamp.java index 37a78b84df2ae..f1e75765d7f75 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReaderByTimestamp.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReaderByTimestamp.java @@ -39,7 +39,7 @@ public class SealedTsFilesReaderByTimestamp implements EngineReaderByTimeStamp { private Path seriesPath; private List sealedTsFiles; - private int usedIntervalFileIndex; + private int nextIntervalFileIndex; private SeriesReaderByTimestamp seriesReader; private QueryContext context; @@ -50,7 +50,7 @@ public SealedTsFilesReaderByTimestamp(Path seriesPath, List se QueryContext context) { this.seriesPath = seriesPath; this.sealedTsFiles = sealedTsFiles; - this.usedIntervalFileIndex = 0; + this.nextIntervalFileIndex = 0; this.seriesReader = null; this.context = context; } @@ -80,7 +80,14 @@ public boolean hasNext() throws IOException { if (seriesReader != null && seriesReader.hasNext()) { return true; } - return usedIntervalFileIndex + 1 < sealedTsFiles.size(); + while (nextIntervalFileIndex < sealedTsFiles.size()) { + initSingleTsFileReader(sealedTsFiles.get(nextIntervalFileIndex), context); + nextIntervalFileIndex++; + if(seriesReader.hasNext()){ + return true; + } + } + return false; } @Override @@ -90,11 +97,11 @@ public void close() throws IOException { // construct reader from the file that might overlap this timestamp private void constructReader(long timestamp) throws IOException { - while (usedIntervalFileIndex < sealedTsFiles.size()) { - if (singleTsFileSatisfied(sealedTsFiles.get(usedIntervalFileIndex), timestamp)) { - initSingleTsFileReader(sealedTsFiles.get(usedIntervalFileIndex), context); + while (nextIntervalFileIndex < sealedTsFiles.size()) { + if (singleTsFileSatisfied(sealedTsFiles.get(nextIntervalFileIndex), timestamp)) { + initSingleTsFileReader(sealedTsFiles.get(nextIntervalFileIndex), context); } - usedIntervalFileIndex++; + nextIntervalFileIndex++; } } diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReaderByTimestamp.java index 19923a763da4b..beaecb6e70b51 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReaderByTimestamp.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReaderByTimestamp.java @@ -84,7 +84,13 @@ public boolean hasNext() throws IOException { if (currentSeriesReader != null && currentSeriesReader.hasNext()) { return true; } - return nextSeriesReaderIndex < seriesReaders.size(); + while (nextSeriesReaderIndex < seriesReaders.size()) { + currentSeriesReader = seriesReaders.get(nextSeriesReaderIndex++); + if (currentSeriesReader != null && currentSeriesReader.hasNext()) { + return true; + } + } + return false; } diff --git a/iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestampTest.java b/iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestampTest.java index e543764438a0c..f74ae85a733f1 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestampTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestampTest.java @@ -84,19 +84,6 @@ public void test() throws IOException { } time += random.nextInt(50) + 1; } - - for(long i = 1; i < 1080 + 200 * 13; i++){ - Object value = priorityReader.getValueInTimestamp(i); - if(i >= 1080 && (i-1080) % 13 == 0){ - Assert.assertEquals(i % 31, value); - } else if( i >= 850 && i < 850 + 200 * 7 && (i-850) % 7 == 0 ){ - Assert.assertEquals(i % 19, value); - } else if( i >= 100 &&i < 100 + 200 * 5 && (i-100) % 5 == 0 ){ - Assert.assertEquals(i % 11, value); - } - Assert.assertNull(value); - } - } public static class FakedPrioritySeriesReaderByTimestamp implements EngineReaderByTimeStamp, 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 bbf8f3e44f78c..ed82aed09920b 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 @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.iotdb.tsfile.read.common; import java.util.ArrayList; @@ -529,27 +530,18 @@ public int length() { return this.timeLength; } - public Object getValueInTimestamp(long key){ - int max, min, mid; - min = curIdx ; - max = length() - 1; - mid = (max + min) / 2; - while ( max >= min) { - long midKey = timeRet.get(mid / timeCapacity)[mid % timeCapacity]; - if (key > midKey) { - min = mid + 1; - } else if (key < midKey) { - max = mid - 1; - } - else { - curIdx = mid; + public Object getValueInTimestamp(long time) { + while (hasNext()) { + if (currentTime() < time) { + next(); + } else if (currentTime() == time) { Object value = currentValue(); - curIdx++; + next(); return value; + } else { + return null; } - mid = (min + max) / 2; } - curIdx = min; return null; } } From 0020386029ee07ea0f57e759f4e73dd9cf9e4fb8 Mon Sep 17 00:00:00 2001 From: suyue <2335813255@qq.com> Date: Fri, 1 Mar 2019 21:52:58 +0800 Subject: [PATCH 4/8] Merge remote-tracking branch 'origin/master' into value_filter_query_optimization --- .../UserGuideV0.7.0/2-Concept.md | 10 +- .../4-Deployment and Management.md | 131 ++++++- example/kafka/pom.xml | 71 ++++ example/kafka/readme.md | 75 ++++ .../java/org/apache/iotdb/kafka/Constant.java | 58 +++ .../org/apache/iotdb/kafka/KafkaConsumer.java | 102 +++++ .../iotdb/kafka/KafkaConsumerThread.java | 103 +++++ .../org/apache/iotdb/kafka/KafkaProducer.java | 67 ++++ example/pom.xml | 46 +++ example/rocketmq/pom.xml | 66 ++++ example/rocketmq/readme.md | 87 +++++ .../org/apache/iotdb/example/Constant.java | 66 ++++ .../iotdb/example/RocketMQConsumer.java | 138 +++++++ .../iotdb/example/RocketMQProducer.java | 104 ++++++ .../java/org/apache/iotdb/example/Utils.java | 42 +++ .../iotdb/cli/client/AbstractClient.java | 31 +- .../org/apache/iotdb/cli/client/Client.java | 298 ++++----------- .../apache/iotdb/cli/client/WinClient.java | 148 +++++--- .../apache/iotdb/cli/tool/CsvTestDataGen.java | 125 ------- .../org/apache/iotdb/cli/tool/ExportCsv.java | 159 ++++---- .../org/apache/iotdb/cli/tool/ImportCsv.java | 353 ++++++++++-------- .../iotdb/cli/client/AbstractClientIT.java | 4 +- .../db/engine/filenode/FileNodeManager.java | 48 ++- .../db/engine/filenode/FileNodeProcessor.java | 7 +- .../engine/filenode/TimeIntervalTsFile.java | 287 -------------- .../io/LocalTextModificationAccessor.java | 1 + .../db/query/control/FileReaderManager.java | 131 ++++--- .../query/control/OpenedFilePathsManager.java | 43 ++- .../db/query/factory/SeriesReaderFactory.java | 6 +- .../reader/sequence/SealedTsFilesReader.java | 2 +- .../SealedTsFilesReaderByTimestamp.java | 2 +- .../reader/sequence/UnSealedTsFileReader.java | 2 +- .../UnSealedTsFilesReaderByTimestamp.java | 2 +- .../apache/iotdb/db/service/JDBCService.java | 12 +- .../iotdb/db/service/TSServiceImpl.java | 6 +- .../iotdb/db/utils/OpenFileNumUtil.java | 22 +- .../modification/DeletionQueryTest.java | 2 - .../iotdb/db/integration/IoTDBDeletionIT.java | 26 ++ .../db/integration/IoTDBSeriesReaderIT.java | 11 + .../query/control/FileReaderManagerTest.java | 18 +- .../iotdb/db/utils/OpenFileNumUtilTest.java | 2 +- .../apache/iotdb/jdbc/IoTDBConnection.java | 2 +- .../iotdb/jdbc/IoTDBConnectionParams.java | 2 +- pom.xml | 4 + .../read/controller/MetadataQuerier.java | 9 + .../controller/MetadataQuerierByFileImpl.java | 12 + .../read/query/executor/TsFileExecutor.java | 57 ++- .../reader/series/EmptyFileSeriesReader.java | 64 ++++ .../tsfile/write/TsFileReadWriteTest.java | 36 ++ 49 files changed, 2020 insertions(+), 1080 deletions(-) create mode 100644 example/kafka/pom.xml create mode 100644 example/kafka/readme.md create mode 100644 example/kafka/src/main/java/org/apache/iotdb/kafka/Constant.java create mode 100644 example/kafka/src/main/java/org/apache/iotdb/kafka/KafkaConsumer.java create mode 100644 example/kafka/src/main/java/org/apache/iotdb/kafka/KafkaConsumerThread.java create mode 100644 example/kafka/src/main/java/org/apache/iotdb/kafka/KafkaProducer.java create mode 100644 example/pom.xml create mode 100644 example/rocketmq/pom.xml create mode 100644 example/rocketmq/readme.md create mode 100644 example/rocketmq/src/main/java/org/apache/iotdb/example/Constant.java create mode 100644 example/rocketmq/src/main/java/org/apache/iotdb/example/RocketMQConsumer.java create mode 100644 example/rocketmq/src/main/java/org/apache/iotdb/example/RocketMQProducer.java create mode 100644 example/rocketmq/src/main/java/org/apache/iotdb/example/Utils.java delete mode 100644 iotdb-cli/src/main/java/org/apache/iotdb/cli/tool/CsvTestDataGen.java delete mode 100644 iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/TimeIntervalTsFile.java create mode 100644 tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/EmptyFileSeriesReader.java diff --git a/docs/Documentation/UserGuideV0.7.0/2-Concept.md b/docs/Documentation/UserGuideV0.7.0/2-Concept.md index d560b7d846a93..b51ad78cfa2bf 100644 --- a/docs/Documentation/UserGuideV0.7.0/2-Concept.md +++ b/docs/Documentation/UserGuideV0.7.0/2-Concept.md @@ -178,7 +178,7 @@ IoTDB can support LONG types and DATETIME-DISPLAY types when displaying timestam * Value -The value of a time series is actually the value sent by a sensor to IoTDB. This value can be stored by IoTDB according to the data type. At the same time, users can select the compression mode and the corresponding encoding mode according to the data type of this value. See [Data Type](需要连接到具体的网页链接Chapter2) and [Encoding](需要连接到具体的网页链接Chapter2) of this document for details on data type and corresponding encoding. +The value of a time series is actually the value sent by a sensor to IoTDB. This value can be stored by IoTDB according to the data type. At the same time, users can select the compression mode and the corresponding encoding mode according to the data type of this value. See [Data Type](#data-type) and [Encoding](#encoding) of this document for details on data type and corresponding encoding. * Point @@ -191,7 +191,7 @@ A column of data contains all values belonging to a time series and the timestam ## Data Type IoTDB supports six data types in total: BOOLEAN (Boolean), INT32 (Integer), INT64 (Long Integer), FLOAT (Single Precision Floating Point), DOUBLE (Double Precision Floating Point), TEXT (String). -The time series of FLOAT and DOUBLE type can specify (MAX\_POINT\_NUMBER, see [this page](需要连接到具体的网页链接Chapter5CreateTimeseriesStatement) for more information on how to specify), which is the number of digits after the decimal point of the floating point number, if the encoding method is [RLE](需要连接到具体的网页链接Chapter2RLE) or [TS\_2DIFF](需要连接到具体的网页链接Chapter2TS2DIFF) (Refer to [Create Timeseries Statement](需要连接到具体的网页链接) for more information on how to specify). If MAX\_POINT\_NUMBER is not specified, the system will use [float\_precision](需要连接到具体的网页链接chapter4float\_precision) in the configuration file "tsfile-format.properties" for configuration for the configuration method. +The time series of FLOAT and DOUBLE type can specify (MAX\_POINT\_NUMBER, see [this page](#iotdb-query-statement) for more information on how to specify), which is the number of digits after the decimal point of the floating point number, if the encoding method is [RLE](#encoding) or [TS\_2DIFF](#encoding) (Refer to [Create Timeseries Statement](#chapter-5-iotdb-sql-documentation) for more information on how to specify). If MAX\_POINT\_NUMBER is not specified, the system will use [float\_precision](#encoding) in the configuration file "tsfile-format.properties" for configuration for the configuration method. When the data type of data input by the user in the system does not correspond to the data type of the time series, the system will report type errors. As shown below, the second-order difference encoding does not support the Boolean type: @@ -211,13 +211,13 @@ PLAIN encoding, the default encoding mode, i.e, no encoding, supports multiple d Second-order differential encoding is more suitable for encoding monotonically increasing or decreasing sequence data, and is not recommended for sequence data with large fluctuations. -Second-order differential encoding can also be used to encode floating-point numbers, but it is necessary to specify reserved decimal digits (MAX\_POINT\_NUMBER, see [this page](需要连接到具体的网页链接Chapter5CreateTimeseriesStatement) for more information on how to specify) when creating time series. It is more suitable for storing sequence data where floating-point values appear continuously, monotonously increase or decrease, and it is not suitable for storing sequence data with high precision requirements after the decimal point or with large fluctuations. +Second-order differential encoding can also be used to encode floating-point numbers, but it is necessary to specify reserved decimal digits (MAX\_POINT\_NUMBER, see [this page](#iotdb-query-statement) for more information on how to specify) when creating time series. It is more suitable for storing sequence data where floating-point values appear continuously, monotonously increase or decrease, and it is not suitable for storing sequence data with high precision requirements after the decimal point or with large fluctuations. * RLE Run-length encoding is more suitable for storing sequence with continuous integer values, and is not recommended for sequence data with most of the time different values. -Run-length encoding can also be used to encode floating-point numbers, but it is necessary to specify reserved decimal digits (MAX\_POINT\_NUMBER, see [this page](需要连接到具体的网页链接Chapter5CreateTimeseriesStatement) for more information on how to specify) when creating time series. It is more suitable for storing sequence data where floating-point values appear continuously, monotonously increase or decrease, and it is not suitable for storing sequence data with high precision requirements after the decimal point or with large fluctuations. +Run-length encoding can also be used to encode floating-point numbers, but it is necessary to specify reserved decimal digits (MAX\_POINT\_NUMBER, see [this page](#iotdb-query-statement) for more information on how to specify) when creating time series. It is more suitable for storing sequence data where floating-point values appear continuously, monotonously increase or decrease, and it is not suitable for storing sequence data with high precision requirements after the decimal point or with large fluctuations. * GORILLA @@ -244,4 +244,4 @@ The four encodings described in the previous sections are applicable to differen When the time series is written and encoded as binary data according to the specified type, IoTDB compresses the data using compression technology to further improve space storage efficiency. Although both encoding and compression are designed to improve storage efficiency, encoding techniques are usually only available for specific data types (e.g., second-order differential encoding is only suitable for INT32 or INT64 data type, and storing floating-point numbers requires multiplying them by 10m to convert to integers), after which the data is converted to a binary stream. The compression method (SNAPPY) compresses the binary stream, so the use of the compression method is no longer limited by the data type. -IoTDB allows you to specify the compression method of the column when creating a time series. IoTDB now supports two kinds of compression: UNCOMPRESSOR (no compression) and SNAPPY compression. The specified syntax for compression is detailed in [Create Timeseries Statement](需要连接到具体的网页链接Chapter5). \ No newline at end of file +IoTDB allows you to specify the compression method of the column when creating a time series. IoTDB now supports two kinds of compression: UNCOMPRESSOR (no compression) and SNAPPY compression. The specified syntax for compression is detailed in [Create Timeseries Statement](#chapter-5-iotdb-sql-documentation). \ No newline at end of file diff --git a/docs/Documentation/UserGuideV0.7.0/4-Deployment and Management.md b/docs/Documentation/UserGuideV0.7.0/4-Deployment and Management.md index 0117cc0f5d3d0..2cde40d4618fc 100644 --- a/docs/Documentation/UserGuideV0.7.0/4-Deployment and Management.md +++ b/docs/Documentation/UserGuideV0.7.0/4-Deployment and Management.md @@ -33,6 +33,8 @@ - [Engine Layer](#engine-layer) - [System Monitor](#system-monitor) - [System Status Monitoring](#system-status-monitoring) + - [JMX MBean Monitoring](#jmx-mbean-monitoring) + - [MBean Monitor Attributes List](#mbean-monitor-attributes-list) - [Data Status Monitoring](#data-status-monitoring) - [Writing Data Monitor](#writing-data-monitor) - [Example](#example) @@ -542,6 +544,131 @@ Currently, IoTDB provides users to use Java's JConsole tool to monitor system st After starting JConsole tool and connecting to IoTDB server, you will have a basic look at IoTDB system status(CPU Occupation, in-memory information, etc.). See [official documentation](https://docs.oracle.com/javase/7/docs/technotes/guides/management/jconsole.html) for more informations. +#### JMX MBean Monitoring +By using JConsole tool and connecting with JMX you can see some system statistics and parameters. +This section describes how to use the JConsole ```Mbean``` tab to monitor the number of files opened by the IoTDB service process, the size of the data file, and so on. Once connected to JMX, you can find the ```MBean``` named ```org.apache.iotdb.service``` through the ```MBeans``` tab, as shown in the following Figure. + + + +There are several attributes under Monitor, including the numbers of files opened in different folders, the data file size statistics and the values of some system parameters. By double-clicking the value corresponding to an attribute it can also display a line chart of that attribute. In particular, all the opened file count statistics are currently only supported on ```MacOS``` and most ```Linux``` distro except ```CentOS```. For the OS not supported these statistics will return ```-2```. See the following section for specific introduction of the Monitor attributes. + +##### MBean Monitor Attributes List + +* DataSizeInByte + +|Name| DataSizeInByte | +|:---:|:---| +|Description| The total size of data file.| +|Unit| Byte | +|Type| Long | + +* FileNodeNum + +|Name| FileNodeNum | +|:---:|:---| +|Description| The count number of FileNode. (Currently not supported)| +|Type| Long | + +* OverflowCacheSize + +|Name| OverflowCacheSize | +|:---:|:---| +|Description| The size of out-of-order data cache. (Currently not supported)| +|Unit| Byte | +|Type| Long | + +* BufferWriteCacheSize + +|Name| BufferWriteCacheSize | +|:---:|:---| +|Description| The size of BufferWriter cache. (Currently not supported)| +|Unit| Byte | +|Type| Long | + +* BaseDirectory + +|Name| BaseDirectory | +|:---:|:---| +|Description| The absolute directory of data file. | +|Type| String | + +* WriteAheadLogStatus + +|Name| WriteAheadLogStatus | +|:---:|:---| +|Description| The status of write-ahead-log (WAL). ```True``` means WAL is enabled. | +|Type| Boolean | + +* TotalOpenFileNum + +|Name| TotalOpenFileNum | +|:---:|:---| +|Description| All the opened file number of IoTDB server process. | +|Type| Int | + +* DeltaOpenFileNum + +|Name| DeltaOpenFileNum | +|:---:|:---| +|Description| The opened TsFile file number of IoTDB server process. | +|Default Directory| /data/data/settled | +|Type| Int | + +* OverflowOpenFileNum + +|Name| OverflowOpenFileNum | +|:---:|:---| +|Description| The opened out-of-order data file number of IoTDB server process. | +|Default Directory| /data/data/overflow | +|Type| Int | + +* WalOpenFileNum + +|Name| WalOpenFileNum | +|:---:|:---| +|Description| The opened write-ahead-log file number of IoTDB server process. | +|Default Directory| /data/wal | +|Type| Int | + +* MetadataOpenFileNum + +|Name| MetadataOpenFileNum | +|:---:|:---| +|Description| The opened meta-data file number of IoTDB server process. | +|Default Directory| /data/system/schema | +|Type| Int | + +* DigestOpenFileNum + +|Name| DigestOpenFileNum | +|:---:|:---| +|Description| The opened info file number of IoTDB server process. | +|Default Directory| /data/system/info | +|Type| Int | + +* SocketOpenFileNum + +|Name| SocketOpenFileNum | +|:---:|:---| +|Description| The Socket link (TCP or UDP) number of the operation system. | +|Type| Int | + +* MergePeriodInSecond + +|Name| MergePeriodInSecond | +|:---:|:---| +|Description| The interval at which the IoTDB service process periodically triggers the merge process. | +|Unit| Second | +|Type| Long | + +* ClosePeriodInSecond + +|Name| ClosePeriodInSecond | +|:---:|:---| +|Description| The interval at which the IoTDB service process periodically flushes memory data to disk. | +|Unit| Second | +|Type| Long | + ### Data Status Monitoring This module is the statistical monitoring method provided by IoTDB for users to store data information. We will record the statistical data in the system and store it in the database. The current 0.7.0 version of IoTDB provides statistics for writing data. @@ -692,9 +819,9 @@ At the same time, in order to facilitate the debugging of the system by the deve #### Connect JMX -Here we use Jconsole to connect with JMX. +Here we use JConsole to connect with JMX. -Start the Jconsole, establish a new JMX connection with the IoTDB Server (you can select the local process or input the IP and PORT for remote connection, the default operation port of the IoTDB JMX service is 31999). Fig 4.1 shows the connection GUI of jconsole. +Start the JConsole, establish a new JMX connection with the IoTDB Server (you can select the local process or input the IP and PORT for remote connection, the default operation port of the IoTDB JMX service is 31999). Fig 4.1 shows the connection GUI of JConsole. diff --git a/example/kafka/pom.xml b/example/kafka/pom.xml new file mode 100644 index 0000000000000..04c899da9cca1 --- /dev/null +++ b/example/kafka/pom.xml @@ -0,0 +1,71 @@ + + + + 4.0.0 + + + org.apache.iotdb + iotdb-examples + 0.8.0-SNAPSHOT + ../pom.xml + + kafka-example + IoTDB-Kafka Examples + jar + + + org.apache.iotdb + iotdb-jdbc + + + org.apache.kafka + kafka_2.10 + 0.8.2.0 + + + org.slf4j + slf4j-api + + + org.slf4j + slf4j-log4j12 + + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + 8 + 8 + + + + + diff --git a/example/kafka/readme.md b/example/kafka/readme.md new file mode 100644 index 0000000000000..60f4e633a8ebf --- /dev/null +++ b/example/kafka/readme.md @@ -0,0 +1,75 @@ + +# Kafka-IoTDB Demo +## Function +``` +The example is to show how to send data from localhost to IoTDB through Kafka. +``` +## Usage +### Version usage +IoTDB: 0.8.0-SNAPSHOT +Kafka: 0.8.2.0 +### Dependencies with Maven + +``` + + + org.apache.kafka + kafka_2.10 + 0.8.2.0 + + + org.apache.iotdb + iotdb-jdbc + 0.8.0-SNAPSHOT + + +``` + +### Launch the servers + +``` +  Before you run the program, make sure you have launched the servers of Kafka and IoTDB. +  For details, please refer to http://kafka.apache.org/081/documentation.html#quickstart +``` + +### Run KafkaProducer.java + +``` + The class is to send data from localhost to Kafka clusters. +  Firstly, you have to change the parameter of TOPIC in Constant.java to what you create:(for example : "Kafka-Test") +  > public final static String TOPIC = "Kafka-Test"; +  The default format of data is "device,timestamp,value ". (for example : "sensor1,2017/10/24 19:30:00,60") + Then you need to create data in Constat.ALL_DATA +  Finally, run KafkaProducer.java +``` + +### Run KafkaConsumer.java + +``` + The class is to show how to consume data from kafka through multi-threads. + The data is sent by class KafkaProducer. +  You can set the parameter of CONSUMER_THREAD_NUM in Constant.java to make sure the number of consumer threads:(for example: "5") + > private final static int CONSUMER_THREAD_NUM = 5; +``` + +#### Notice +  If you want to use multiple consumers, please make sure that the number of topic's partition you create is more than 1. \ No newline at end of file diff --git a/example/kafka/src/main/java/org/apache/iotdb/kafka/Constant.java b/example/kafka/src/main/java/org/apache/iotdb/kafka/Constant.java new file mode 100644 index 0000000000000..90ea54c5c134e --- /dev/null +++ b/example/kafka/src/main/java/org/apache/iotdb/kafka/Constant.java @@ -0,0 +1,58 @@ +/** + * 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.kafka; + +import org.apache.iotdb.jdbc.Config; + +/** + * @author lta + */ +public class Constant { + + public static final String TOPIC = "Kafka-Test"; + public static final int CONSUMER_THREAD_NUM = 5; + public static final String IOTDB_CONNECTION_URL = Config.IOTDB_URL_PREFIX + "localhost:6667/"; + public static final String IOTDB_CONNECTION_USER = "root"; + public static final String IOTDB_CONNECTION_PASSWORD = "root"; + public static final String STORAGE_GROUP = "root.vehicle"; + public static final String[] ALL_TIMESERIES = {"root.vehicle.device.sensor1", + "root.vehicle.device.sensor2", "root.vehicle.device.sensor3", "root.vehicle.device.sensor4"}; + public static final String[] ALL_DATA = { + "sensor1,2017/10/24 19:30:00,606162908", + "sensor2,2017/10/24 19:30:00,160161162", + "sensor3,2017/10/24 19:30:00,260261262", + "sensor4,2017/10/24 19:30:00,360361362", + "sensor1,2017/10/24 19:31:00,818182346", + "sensor2,2017/10/24 19:31:00,180181182", + "sensor3,2017/10/24 19:31:00,280281282", + "sensor4,2017/10/24 19:31:00,380381382", + "sensor1,2017/10/24 19:32:00,505152421", + "sensor2,2017/10/24 19:32:00,150151152", + "sensor3,2017/10/24 19:32:00,250251252", + "sensor4,2017/10/24 19:32:00,350351352", + "sensor1,2017/10/24 19:33:00,404142234", + "sensor2,2017/10/24 19:33:00,140141142", + "sensor3,2017/10/24 19:33:00,240241242", + "sensor4,2017/10/24 19:33:00,340341342", + "sensor1,2017/10/24 19:34:00,101112567", + "sensor2,2017/10/24 19:34:00,110111112", + "sensor3,2017/10/24 19:34:00,210211212", + "sensor4,2017/10/24 19:34:00,310311312", + }; +} diff --git a/example/kafka/src/main/java/org/apache/iotdb/kafka/KafkaConsumer.java b/example/kafka/src/main/java/org/apache/iotdb/kafka/KafkaConsumer.java new file mode 100644 index 0000000000000..194449be34f00 --- /dev/null +++ b/example/kafka/src/main/java/org/apache/iotdb/kafka/KafkaConsumer.java @@ -0,0 +1,102 @@ +/** + * 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.kafka; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import kafka.consumer.ConsumerConfig; +import kafka.consumer.KafkaStream; +import kafka.javaapi.consumer.ConsumerConnector; +import kafka.serializer.StringDecoder; +import kafka.utils.VerifiableProperties; + +/** + * The class is to show how to get data from kafka through multi-threads. The data is sent by class + * KafkaProducer. + * + * @author lta + */ +public class KafkaConsumer { + + private ConsumerConnector consumer; + + private KafkaConsumer() { + /** + * Consumer configuration + */ + Properties props = new Properties(); + + /** + * Zookeeper configuration + */ + props.put("zookeeper.connect", "127.0.0.1:2181"); + props.put("group.id", "consumeGroup"); + props.put("zookeeper.session.timeout.ms", "400"); + props.put("zookeeper.sync.time.ms", "200"); + props.put("rebalance.max.retries", "5"); + props.put("rebalance.backoff.ms", "1200"); + props.put("auto.commit.interval.ms", "1000"); + + /** + * What to do when there is no initial offset in ZooKeeper or if an offset is out of range + * smallest : automatically reset the offset to the smallest offset + */ + props.put("auto.offset.reset", "smallest"); + + /** + * serializer class + */ + props.put("serializer.class", "kafka.serializer.StringEncoder"); + + ConsumerConfig config = new ConsumerConfig(props); + consumer = kafka.consumer.Consumer.createJavaConsumerConnector(config); + } + + public static void main(String[] args) { + new KafkaConsumer().consume(); + } + + private void consume() { + /** + * Specify the number of consumer thread + */ + Map topicCountMap = new HashMap<>(); + topicCountMap.put(Constant.TOPIC, Constant.CONSUMER_THREAD_NUM); + + /** + * Specify data decoder + */ + StringDecoder keyDecoder = new StringDecoder(new VerifiableProperties()); + StringDecoder valueDecoder = new StringDecoder(new VerifiableProperties()); + + Map>> consumerMap = consumer + .createMessageStreams(topicCountMap, keyDecoder, + valueDecoder); + + List> streams = consumerMap.get(Constant.TOPIC); + ExecutorService executor = Executors.newFixedThreadPool(Constant.CONSUMER_THREAD_NUM); + for (final KafkaStream stream : streams) { + executor.submit(new KafkaConsumerThread(stream)); + } + } +} \ No newline at end of file diff --git a/example/kafka/src/main/java/org/apache/iotdb/kafka/KafkaConsumerThread.java b/example/kafka/src/main/java/org/apache/iotdb/kafka/KafkaConsumerThread.java new file mode 100644 index 0000000000000..3e5d18068a1da --- /dev/null +++ b/example/kafka/src/main/java/org/apache/iotdb/kafka/KafkaConsumerThread.java @@ -0,0 +1,103 @@ +/** + * 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.kafka; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.Statement; +import kafka.consumer.KafkaStream; +import kafka.message.MessageAndMetadata; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * @author lta + */ +public class KafkaConsumerThread implements Runnable { + + private Connection connection = null; + private Statement statement = null; + private KafkaStream stream; + private static boolean setStorageGroup = true; + private static boolean createTimeSeries = true; + private String createStorageGroupSqlTemplate = "SET STORAGE GROUP TO %s"; + private String createTimeseriesSqlTemplate = "CREATE TIMESERIES %s WITH DATATYPE=TEXT, ENCODING=PLAIN"; + private String insertDataSqlTemplate = "INSERT INTO root.vehicle.device(timestamp,%s) VALUES (%s,'%s')"; + private static final Logger LOGGER = LoggerFactory.getLogger(KafkaConsumerThread.class); + + public KafkaConsumerThread(KafkaStream stream) { + this.stream = stream; + /** + * Establish JDBC connection of IoTDB + */ + initIoTDB(); + } + + private void initIoTDB() { + try { + Class.forName("org.apache.iotdb.jdbc.IoTDBDriver"); + connection = DriverManager + .getConnection(Constant.IOTDB_CONNECTION_URL, Constant.IOTDB_CONNECTION_USER, + Constant.IOTDB_CONNECTION_PASSWORD); + statement = connection.createStatement(); + if (setStorageGroup) { + try { + statement.execute(String.format(createStorageGroupSqlTemplate, Constant.STORAGE_GROUP)); + } catch (SQLException e) { + } + setStorageGroup = false; + } + if (createTimeSeries) { + for (String timeseries : Constant.ALL_TIMESERIES) { + statement.addBatch(String.format(createTimeseriesSqlTemplate, timeseries)); + } + statement.executeBatch(); + statement.clearBatch(); + createTimeSeries = false; + } + } catch (ClassNotFoundException | SQLException e) { + LOGGER.error(e.getMessage()); + } + } + + /** + * Write data to IoTDB + */ + private void writeData(String message) { + + String[] items = message.split(","); + + try { + String sql = String.format(insertDataSqlTemplate, items[0], items[1], items[2]); + statement.execute(sql); + } catch (SQLException e) { + LOGGER.error(e.getMessage()); + } + } + + public void run() { + for (MessageAndMetadata consumerIterator : stream) { + String uploadMessage = consumerIterator.message(); + LOGGER.info(String.format("%s from partiton[%d]: %s", Thread.currentThread().getName(), + consumerIterator.partition(), uploadMessage)); + writeData(uploadMessage); + } + } +} \ No newline at end of file diff --git a/example/kafka/src/main/java/org/apache/iotdb/kafka/KafkaProducer.java b/example/kafka/src/main/java/org/apache/iotdb/kafka/KafkaProducer.java new file mode 100644 index 0000000000000..ec8d8b472a916 --- /dev/null +++ b/example/kafka/src/main/java/org/apache/iotdb/kafka/KafkaProducer.java @@ -0,0 +1,67 @@ +/** + * 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.kafka; + +import java.util.Properties; +import kafka.javaapi.producer.Producer; +import kafka.producer.KeyedMessage; +import kafka.producer.ProducerConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The class is to show how to send data to kafka through multi-threads. + * + * @author lta + */ +public class KafkaProducer { + + private final Producer producer; + private static final Logger LOGGER = LoggerFactory.getLogger(KafkaProducer.class); + + public KafkaProducer() { + + Properties props = new Properties(); + props.put("metadata.broker.list", "127.0.0.1:9092"); + props.put("zk.connect", "127.0.0.1:2181"); + props.put("serializer.class", "kafka.serializer.StringEncoder"); + props.put("key.serializer.class", "kafka.serializer.StringEncoder"); + props.put("request.required.acks", "-1"); + + producer = new Producer<>(new ProducerConfig(props)); + } + + public static void main(String[] args) { + KafkaProducer kafkaProducer = new KafkaProducer(); + kafkaProducer.produce(); + kafkaProducer.close(); + } + + private void produce() { + for (int i = 0; i < Constant.ALL_DATA.length; i++) { + String key = Integer.toString(i); + producer.send(new KeyedMessage<>(Constant.TOPIC, key, Constant.ALL_DATA[i])); + LOGGER.info(Constant.ALL_DATA[i]); + } + } + + public void close(){ + producer.close(); + } +} diff --git a/example/pom.xml b/example/pom.xml new file mode 100644 index 0000000000000..70c9a909f1279 --- /dev/null +++ b/example/pom.xml @@ -0,0 +1,46 @@ + + + + 4.0.0 + + org.apache.iotdb + root + 0.8.0-SNAPSHOT + ../pom.xml + + pom + iotdb-examples + IoTDB Examples + + + + org.apache.iotdb + iotdb-jdbc + ${project.version} + + + + + kafka + rocketmq + + diff --git a/example/rocketmq/pom.xml b/example/rocketmq/pom.xml new file mode 100644 index 0000000000000..aa4d75ad18683 --- /dev/null +++ b/example/rocketmq/pom.xml @@ -0,0 +1,66 @@ + + + + 4.0.0 + + org.apache.iotdb + iotdb-examples + 0.8.0-SNAPSHOT + ../pom.xml + + rocketmq-example + IoTDB-RocketMQ Examples + jar + + + org.apache.iotdb + iotdb-jdbc + + + io.netty + netty-common + + + io.netty + netty-buffer + + + + + org.apache.rocketmq + rocketmq-client + 4.4.0 + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + 8 + 8 + + + + + diff --git a/example/rocketmq/readme.md b/example/rocketmq/readme.md new file mode 100644 index 0000000000000..0bf25a30efc8f --- /dev/null +++ b/example/rocketmq/readme.md @@ -0,0 +1,87 @@ + +# Rocketmq-IoTDB Demo +##Introduction +This demo shows how to store data into IoTDB via rocketmq +##Basic Concept +The following basic concepts are involved in IoTDB: + +* Device + +A devices is an installation equipped with sensors in real scenarios. In IoTDB, all sensors should have their corresponding devices. + +* Sensor + +A sensor is a detection equipment in an actual scene, which can sense the information to be measured, and can transform the sensed information into an electrical signal or other desired form of information output and send it to IoTDB. In IoTDB, all data and paths stored are organized in units of sensors. + +* Storage Group + +Storage groups are used to let users define how to organize and isolate different time series data on disk. Time series belonging to the same storage group will be continuously written to the same file in the corresponding folder. The file may be closed due to user commands or system policies, and hence the data coming next from these sensors will be stored in a new file in the same folder. Time series belonging to different storage groups are stored in different folders. +##Connector +> note:In this sample program, there are some update operations for historical data, so it is necessary to ensure the sequential transmission and consumption of data via RocketMQ. If there is no update operation in use, then there is no need to guarantee the order of data. IoTDB will process these data which may be disorderly. + +###Producer +Producers insert IoTDB insert statements into partitions according to devices, ensuring that the same device's data is inserted or updated in the same MessageQueue. +###Consumer +1. At startup, the consumer client first creates a JDBC connection and check whether the storage groups and timeseries are created in IoTDB. If not, create it. +2. Then consume client consume data from rocketmq using MessageListener Orderly to ensure orderly consumption, and insert the sql statement into IoTDB. + +## Usage +### Version usage +IoTDB: 0.8.0-SNAPSHOT +Kafka: 4.4.0 +### Dependencies with Maven + +``` + + + org.apache.iotdb + iotdb-jdbc + 0.8.0-SNAPSHOT + + + io.netty + netty-common + + + io.netty + netty-buffer + + + + + org.apache.rocketmq + rocketmq-client + 4.4.0 + + +``` +Note: The maven dependencies of io.netty in IoTDB are in conflicts with those dependencies in RocketMQ-Client. +###1. Install IoTDB +please refer to [https://iotdb.apache.org/#/Download](https://iotdb.apache.org/#/Download) +###2. Install RocketMQ +pleasr refer to [http://rocketmq.apache.org/docs/quick-start/](http://rocketmq.apache.org/docs/quick-start/) +###3. Startup IoTDB +please refer to [https://iotdb.apache.org/#/Documents](https://iotdb.apache.org/#/Documents) +###4. Startup RocketMQ +please refer to [http://rocketmq.apache.org/docs/quick-start/](http://rocketmq.apache.org/docs/quick-start/) +###5. Start the consumer client:RocketMQConsumer +###6. Start the producer client:RocketMQProducer diff --git a/example/rocketmq/src/main/java/org/apache/iotdb/example/Constant.java b/example/rocketmq/src/main/java/org/apache/iotdb/example/Constant.java new file mode 100644 index 0000000000000..1a290554a08b3 --- /dev/null +++ b/example/rocketmq/src/main/java/org/apache/iotdb/example/Constant.java @@ -0,0 +1,66 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iotdb.example; + +import org.apache.iotdb.jdbc.Config; + +/** + * @author lta + */ +public class Constant { + + public static final String TOPIC = "RocketMQ-Test"; + public static final String PRODUCER_GROUP = "IoTDBConsumer"; + public static final String CONSUMER_GROUP = "IoTDBProducer"; + public static final String SERVER_ADDRESS = "localhost:9876"; + public static final String IOTDB_CONNECTION_URL = Config.IOTDB_URL_PREFIX + "localhost:6667/"; + public static final String IOTDB_CONNECTION_USER = "root"; + public static final String IOTDB_CONNECTION_PASSWORD = "root"; + public static final String[] STORAGE_GROUP = {"root.vehicle", "root.test"}; + public static final String[] CREATE_TIMESERIES = { + "CREATE TIMESERIES root.vehicle.d0.s0 WITH DATATYPE=INT32, ENCODING=PLAIN", + "CREATE TIMESERIES root.vehicle.d0.s1 WITH DATATYPE=TEXT, ENCODING=PLAIN", + "CREATE TIMESERIES root.vehicle.d0.s2 WITH DATATYPE=FLOAT, ENCODING=PLAIN", + "CREATE TIMESERIES root.vehicle.d0.s3 WITH DATATYPE=BOOLEAN, ENCODING=PLAIN", + "CREATE TIMESERIES root.test.d0.s0 WITH DATATYPE=INT32, ENCODING=PLAIN", + "CREATE TIMESERIES root.test.d0.s1 WITH DATATYPE=TEXT, ENCODING=PLAIN", + "CREATE TIMESERIES root.test.d1.s0 WITH DATATYPE=INT32, ENCODING=PLAIN"}; + public static final String[] ALL_DATA = { + "insert into root.vehicle.d0(timestamp,s0) values(10,100)", + "insert into root.vehicle.d0(timestamp,s0,s1) values(12,101,'102')", + "insert into root.vehicle.d0(timestamp,s1) values(19,'103')", + "insert into root.vehicle.d1(timestamp,s2) values(11,104.0)", + "insert into root.vehicle.d1(timestamp,s2,s3) values(15,105.0,true)", + "insert into root.vehicle.d1(timestamp,s3) values(17,false)", + "insert into root.vehicle.d0(timestamp,s0) values(20,1000)", + "insert into root.vehicle.d0(timestamp,s0,s1) values(22,1001,'1002')", + "insert into root.vehicle.d0(timestamp,s1) values(29,'1003')", + "insert into root.vehicle.d1(timestamp,s2) values(21,1004.0)", + "insert into root.vehicle.d1(timestamp,s2,s3) values(25,1005.0,true)", + "insert into root.vehicle.d1(timestamp,s3) values(27,true)", + "insert into root.test.d0(timestamp,s0) values(10,106)", + "insert into root.test.d0(timestamp,s0,s1) values(14,107,'108')", + "insert into root.test.d0(timestamp,s1) values(16,'109')", + "insert into root.test.d1(timestamp,s0) values(1,110)", + "insert into root.test.d0(timestamp,s0) values(30,1006)", + "insert into root.test.d0(timestamp,s0,s1) values(34,1007,'1008')", + "insert into root.test.d0(timestamp,s1) values(36,'1090')", + "insert into root.test.d1(timestamp,s0) values(10,1100)" + }; +} diff --git a/example/rocketmq/src/main/java/org/apache/iotdb/example/RocketMQConsumer.java b/example/rocketmq/src/main/java/org/apache/iotdb/example/RocketMQConsumer.java new file mode 100644 index 0000000000000..255a4189542d0 --- /dev/null +++ b/example/rocketmq/src/main/java/org/apache/iotdb/example/RocketMQConsumer.java @@ -0,0 +1,138 @@ +/** + * 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.example; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.Statement; +import org.apache.rocketmq.client.consumer.DefaultMQPushConsumer; +import org.apache.rocketmq.client.consumer.listener.ConsumeOrderlyStatus; +import org.apache.rocketmq.client.consumer.listener.MessageListenerOrderly; +import org.apache.rocketmq.client.exception.MQClientException; +import org.apache.rocketmq.common.consumer.ConsumeFromWhere; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * @author lta + */ +public class RocketMQConsumer { + + private DefaultMQPushConsumer consumer; + private String producerGroup; + private String serverAddresses; + private Connection connection; + private Statement statement; + private String createStorageGroupSqlTemplate = "SET STORAGE GROUP TO %s"; + private static final Logger LOGGER = LoggerFactory.getLogger(RocketMQConsumer.class); + + public RocketMQConsumer(String producerGroup, String serverAddresses, String connectionUrl, + String user, String password) throws ClassNotFoundException, SQLException { + this.producerGroup = producerGroup; + this.serverAddresses = serverAddresses; + this.consumer = new DefaultMQPushConsumer(producerGroup); + this.consumer.setNamesrvAddr(serverAddresses); + initIoTDB(connectionUrl, user, password); + } + + private void initIoTDB(String connectionUrl, String user, String password) + throws ClassNotFoundException, SQLException { + Class.forName("org.apache.iotdb.jdbc.IoTDBDriver"); + user = (user == null ? "root" : user); + password = (password == null ? "root" : password); + connection = DriverManager.getConnection(connectionUrl, user, password); + statement = connection.createStatement(); + for (String storageGroup : Constant.STORAGE_GROUP) { + statement.addBatch(String.format(createStorageGroupSqlTemplate, storageGroup)); + } + for (String sql : Constant.CREATE_TIMESERIES) { + statement.addBatch(sql); + } + try { + statement.executeBatch(); + } catch (SQLException e) { + } + statement.clearBatch(); + } + + public void start() throws MQClientException { + consumer.start(); + } + + /** + * Subscribe topic and add regiser Listener + * @throws MQClientException + */ + public void prepareConsume() throws MQClientException { + /** + * Subscribe one more more topics to consume. + */ + consumer.subscribe(Constant.TOPIC, "*"); + /** + * Setting Consumer to start first from the head of the queue or from the tail of the queue + * If not for the first time, then continue to consume according to the position of last consumption. + */ + consumer.setConsumeFromWhere(ConsumeFromWhere.CONSUME_FROM_FIRST_OFFSET); + /** + * Register callback to execute on arrival of messages fetched from brokers. + */ + consumer.registerMessageListener((MessageListenerOrderly) (msgs, context) -> { + LOGGER.info(String.format("%s Receive New Messages: %s %n", Thread.currentThread().getName(), + new String(msgs.get(0).getBody()))); + try { + statement.execute(new String(msgs.get(0).getBody())); + } catch (SQLException e) { + LOGGER.error(e.getMessage()); + } + return ConsumeOrderlyStatus.SUCCESS; + }); + } + + public void shutdown() { + consumer.shutdown(); + } + + public String getProducerGroup() { + return producerGroup; + } + + public void setProducerGroup(String producerGroup) { + this.producerGroup = producerGroup; + } + + public String getServerAddresses() { + return serverAddresses; + } + + public void setServerAddresses(String serverAddresses) { + this.serverAddresses = serverAddresses; + } + + public static void main(String[] args) + throws MQClientException, SQLException, ClassNotFoundException { + /** + *Instantiate with specified consumer group name and specify name server addresses. + */ + RocketMQConsumer consumer = new RocketMQConsumer(Constant.CONSUMER_GROUP, Constant.SERVER_ADDRESS, Constant.IOTDB_CONNECTION_URL, Constant.IOTDB_CONNECTION_USER, + Constant.IOTDB_CONNECTION_PASSWORD); + consumer.prepareConsume(); + consumer.start(); + } +} diff --git a/example/rocketmq/src/main/java/org/apache/iotdb/example/RocketMQProducer.java b/example/rocketmq/src/main/java/org/apache/iotdb/example/RocketMQProducer.java new file mode 100644 index 0000000000000..f4c5a07cb3e0b --- /dev/null +++ b/example/rocketmq/src/main/java/org/apache/iotdb/example/RocketMQProducer.java @@ -0,0 +1,104 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iotdb.example; + +import org.apache.rocketmq.client.exception.MQBrokerException; +import org.apache.rocketmq.client.exception.MQClientException; +import org.apache.rocketmq.client.producer.DefaultMQProducer; +import org.apache.rocketmq.client.producer.SendResult; +import org.apache.rocketmq.common.message.Message; +import org.apache.rocketmq.remoting.common.RemotingHelper; +import org.apache.rocketmq.remoting.exception.RemotingException; + +import java.io.UnsupportedEncodingException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * @author lta + */ +public class RocketMQProducer { + + private DefaultMQProducer producer; + private String producerGroup; + private String serverAddresses; + private static final Logger LOGGER = LoggerFactory.getLogger(RocketMQProducer.class); + + public RocketMQProducer(String producerGroup, String serverAddresses) { + this.producerGroup = producerGroup; + this.serverAddresses = serverAddresses; + this.producer = new DefaultMQProducer(producerGroup); + this.producer.setNamesrvAddr(serverAddresses); + } + + public void start() throws MQClientException { + producer.start(); + } + + public void sendMessage() + throws UnsupportedEncodingException, InterruptedException, RemotingException, MQClientException, MQBrokerException { + for (String sql : Constant.ALL_DATA) { + /** + * Create a message instance, specifying topic, tag and message body. + */ + Message msg = new Message(Constant.TOPIC, null, null, + (sql).getBytes(RemotingHelper.DEFAULT_CHARSET)); + SendResult sendResult = producer.send(msg, (mqs, msg1, arg) -> { + Integer id = (Integer) arg; + int index = id % mqs.size(); + return mqs.get(index); + }, Utils + .ConvertStringToInteger(Utils.getTimeSeries(sql))); + LOGGER.info(sendResult.toString()); + } + } + + public void shutdown() { + producer.shutdown(); + } + + public String getProducerGroup() { + return producerGroup; + } + + public void setProducerGroup(String producerGroup) { + this.producerGroup = producerGroup; + } + + public String getServerAddresses() { + return serverAddresses; + } + + public void setServerAddresses(String serverAddresses) { + this.serverAddresses = serverAddresses; + } + + public static void main(String[] args) throws Exception { + /** + * Instantiate with a producer group name and specify name server addresses. + */ + RocketMQProducer producer = new RocketMQProducer(Constant.PRODUCER_GROUP, Constant.SERVER_ADDRESS); + /** + * Launch the instance + */ + producer.start(); + producer.sendMessage(); + producer.shutdown(); + } +} diff --git a/example/rocketmq/src/main/java/org/apache/iotdb/example/Utils.java b/example/rocketmq/src/main/java/org/apache/iotdb/example/Utils.java new file mode 100644 index 0000000000000..c0a9eb1c24e3a --- /dev/null +++ b/example/rocketmq/src/main/java/org/apache/iotdb/example/Utils.java @@ -0,0 +1,42 @@ +/** + * 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.example; + +/** + * @author lta + */ +public class Utils { + + private Utils() { + throw new IllegalStateException("Utility class"); + } + + public static int ConvertStringToInteger(String device) { + int sum = 0; + for (char c : device.toCharArray()) { + sum += c; + } + return sum; + } + + public static String getTimeSeries(String sql){ + return sql.substring(sql.indexOf("into") + 4, sql.indexOf('(')).trim(); + } + +} diff --git a/iotdb-cli/src/main/java/org/apache/iotdb/cli/client/AbstractClient.java b/iotdb-cli/src/main/java/org/apache/iotdb/cli/client/AbstractClient.java index 267355bf794c7..5f9d72f940c7b 100644 --- a/iotdb-cli/src/main/java/org/apache/iotdb/cli/client/AbstractClient.java +++ b/iotdb-cli/src/main/java/org/apache/iotdb/cli/client/AbstractClient.java @@ -42,6 +42,7 @@ import org.apache.iotdb.jdbc.IoTDBConnection; import org.apache.iotdb.jdbc.IoTDBDatabaseMetadata; import org.apache.iotdb.jdbc.IoTDBMetadataResultSet; +import org.apache.iotdb.jdbc.IoTDBSQLException; import org.apache.iotdb.service.rpc.thrift.ServerProperties; import org.apache.thrift.TException; @@ -82,7 +83,6 @@ public abstract class AbstractClient { protected static final String TIMESTAMP_STR = "Time"; protected static final int ISO_DATETIME_LEN = 26; protected static final String IMPORT_CMD = "import"; - protected static final String EXPORT_CMD = "export"; private static final String NEED_NOT_TO_PRINT_TIMESTAMP = "AGGREGATION"; private static final String DEFAULT_TIME_FORMAT = "default"; protected static String timeFormat = DEFAULT_TIME_FORMAT; @@ -90,6 +90,7 @@ public abstract class AbstractClient { protected static int fetchSize = 10000; protected static int maxTimeLength = ISO_DATETIME_LEN; protected static int maxValueLength = 15; + protected static boolean isQuit = false; /** * control the width of columns for 'show timeseries path' and 'show storage group'. *

@@ -493,15 +494,15 @@ protected static void displayLogo(String version) { protected static OperationResult handleInputCmd(String cmd, IoTDBConnection connection) { String specialCmd = cmd.toLowerCase().trim(); - if (specialCmd.equals(QUIT_COMMAND) || specialCmd.equals(EXIT_COMMAND)) { - println(specialCmd + " normally"); - return OperationResult.RETURN_OPER; + if (QUIT_COMMAND.equals(specialCmd) || EXIT_COMMAND.equals(specialCmd)) { + isQuit = true; + return OperationResult.STOP_OPER; } - if (specialCmd.equals(HELP)) { + if (HELP.equals(specialCmd)) { showHelp(); return OperationResult.CONTINUE_OPER; } - if (specialCmd.equals(SHOW_METADATA_COMMAND)) { + if (SHOW_METADATA_COMMAND.equals(specialCmd)) { showMetaData(connection); return OperationResult.CONTINUE_OPER; } @@ -526,7 +527,7 @@ protected static OperationResult handleInputCmd(String cmd, IoTDBConnection conn } if (specialCmd.startsWith(SHOW_TIMEZONE)) { - + showTimeZone(connection); return OperationResult.CONTINUE_OPER; } if (specialCmd.startsWith(SHOW_TIMESTAMP_DISPLAY)) { @@ -683,7 +684,7 @@ protected static void executeQuery(IoTDBConnection connection, String cmd) { if (hasResultSet) { ResultSet resultSet = statement.getResultSet(); output(resultSet, printToConsole, zoneId); - + closeResultSet(resultSet); } } catch (Exception e) { println("Msg: " + e.getMessage()); @@ -714,30 +715,30 @@ protected static void closeResultSet(ResultSet resultSet) { } enum OperationResult { - RETURN_OPER, CONTINUE_OPER, NO_OPER + STOP_OPER, CONTINUE_OPER, NO_OPER } - private static void printf(String format, Object ... args) { + protected static void printf(String format, Object ... args) { SCREEN_PRINTER.printf(format, args); } - private static void print(String msg) { + protected static void print(String msg) { SCREEN_PRINTER.println(msg); } - private static void println() { + protected static void println() { SCREEN_PRINTER.println(); } - private static void println(String msg) { + protected static void println(String msg) { SCREEN_PRINTER.println(msg); } - private static void println(Object obj) { + protected static void println(Object obj) { SCREEN_PRINTER.println(obj); } - private static void handleException(Exception e) { + protected static void handleException(Exception e) { if (showException) { e.printStackTrace(SCREEN_PRINTER); } diff --git a/iotdb-cli/src/main/java/org/apache/iotdb/cli/client/Client.java b/iotdb-cli/src/main/java/org/apache/iotdb/cli/client/Client.java index c9d1d6563487b..8115e8e7e4eec 100644 --- a/iotdb-cli/src/main/java/org/apache/iotdb/cli/client/Client.java +++ b/iotdb-cli/src/main/java/org/apache/iotdb/cli/client/Client.java @@ -18,15 +18,10 @@ */ package org.apache.iotdb.cli.client; +import java.io.IOException; import java.sql.DriverManager; import java.sql.SQLException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; import jline.console.ConsoleReader; -import jline.console.completer.ArgumentCompleter; -import jline.console.completer.Completer; -import jline.console.completer.StringsCompleter; import org.apache.commons.cli.CommandLine; import org.apache.commons.cli.CommandLineParser; import org.apache.commons.cli.DefaultParser; @@ -36,38 +31,11 @@ import org.apache.iotdb.cli.exception.ArgsErrorException; import org.apache.iotdb.jdbc.Config; import org.apache.iotdb.jdbc.IoTDBConnection; +import org.apache.thrift.TException; public class Client extends AbstractClient { - private static final String[] FUNCTION_NAME_LIST = new String[]{"count", "max_time", "min_time", - "max_value", - "min_value", "subsequence_matching", // with parameter(s) - "now()" // without parameter - }; - - private static final String[] KEY_WORD_LIST = new String[]{"SHOW", "SELECT", "DROP", "UPDATE", - "DELETE", "CREATE", - "INSERT", "INDEX", "TIMESERIES", "TIME", "TIMESTAMP", "VALUES", "FROM", "WHERE", "TO", "ON", - "WITH", - "USING", "AND", "OR", "USER", "ROLE", "EXIT", "QUIT", "IMPORT"}; - - private static final String[] CONF_NAME_LIST = new String[]{ // set = - "time_display_type", "time_zone", // specific format - "fetch_size", "max_display_num", // integer - "storage group to" // special - }; - - private static final String[] PARAM_NAME_LIST = new String[]{"datatype", "encoding", // list - "window_length", // integer - }; - - private static final String[] DATA_TYPE_LIST = new String[]{"BOOLEAN", "INT32", "INT64", "INT96", - "FLOAT", - "DOUBLE", "TEXT", "FIXED_LEN_BYTE_ARRAY", "ENUMS", "BIGDECIMAL"}; - - private static final String[] ENCODING_LIST = new String[]{"PLAIN", "PLAIN_DICTIONARY", "RLE", - "DIFF", "TS_2DIFF", - "BITMAP", "GORILLA"}; + private static CommandLine commandLine; /** * IoTDB CLI main function. @@ -80,50 +48,63 @@ public static void main(String[] args) throws ClassNotFoundException { Options options = createOptions(); HelpFormatter hf = new HelpFormatter(); hf.setWidth(MAX_HELP_CONSOLE_WIDTH); - CommandLine commandLine = null; - CommandLineParser parser = new DefaultParser(); - String[] newArgs = null; + commandLine = null; + + String[] newArgs; if (args == null || args.length == 0) { - System.out.println( + println( "Require more params input, eg. ./start-client.sh(start-client.bat if Windows) " + "-h xxx.xxx.xxx.xxx -p xxxx -u xxx."); - System.out.println("For more information, please check the following hint."); + println("For more information, please check the following hint."); hf.printHelp(SCRIPT_HINT, options, true); return; } init(); newArgs = removePasswordArgs(args); + boolean continues = parseCommandLine(options, newArgs, hf); + if (!continues) { + return; + } + + serve(); + } + + private static boolean parseCommandLine(Options options, String[] newArgs, HelpFormatter hf) { try { + CommandLineParser parser = new DefaultParser(); commandLine = parser.parse(options, newArgs); if (commandLine.hasOption(HELP_ARGS)) { hf.printHelp(SCRIPT_HINT, options, true); - return; + return false; } if (commandLine.hasOption(ISO8601_ARGS)) { setTimeFormat("long"); } if (commandLine.hasOption(MAX_PRINT_ROW_COUNT_ARGS)) { - try { - setMaxDisplayNumber(commandLine.getOptionValue(MAX_PRINT_ROW_COUNT_ARGS)); - } catch (NumberFormatException e) { - System.out.println( - IOTDB_CLI_PREFIX + "> error format of max print row count, it should be number"); - return; - } + setMaxDisplayNumber(commandLine.getOptionValue(MAX_PRINT_ROW_COUNT_ARGS)); } } catch (ParseException e) { - System.out.println( + println( "Require more params input, eg. ./start-client.sh(start-client.bat if Windows) " + "-h xxx.xxx.xxx.xxx -p xxxx -u xxx."); - System.out.println("For more information, please check the following hint."); + println("For more information, please check the following hint."); hf.printHelp(IOTDB_CLI_PREFIX, options, true); - return; + handleException(e); + return false; + } catch (NumberFormatException e) { + println( + IOTDB_CLI_PREFIX + "> error format of max print row count, it should be number"); + handleException(e); + return false; } + return true; + } + private static void serve() { try (ConsoleReader reader = new ConsoleReader()) { reader.setExpandEvents(false); - String s; + host = checkRequiredArg(HOST_ARGS, HOST_NAME, commandLine, false, host); port = checkRequiredArg(PORT_ARGS, PORT_NAME, commandLine, false, port); @@ -133,185 +114,58 @@ public static void main(String[] args) throws ClassNotFoundException { if (password == null) { password = reader.readLine("please input your password:", '\0'); } - try (IoTDBConnection connection = (IoTDBConnection) DriverManager - .getConnection(Config.IOTDB_URL_PREFIX + host + ":" + port + "/", username, password)) { - properties = connection.getServerProperties(); - AGGREGRATE_TIME_LIST.addAll(properties.getSupportedTimeAggregationOperations()); - displayLogo(properties.getVersion()); - System.out.println(IOTDB_CLI_PREFIX + "> login successfully"); - while (true) { - s = reader.readLine(IOTDB_CLI_PREFIX + "> ", null); - if (s != null) { - String[] cmds = s.trim().split(";"); - for (int i = 0; i < cmds.length; i++) { - String cmd = cmds[i]; - if (cmd != null && !"".equals(cmd.trim())) { - OperationResult result = handleInputCmd(cmd, connection); - switch (result) { - case RETURN_OPER: - return; - case CONTINUE_OPER: - continue; - default: - break; - } - } - } - } - } - } catch (SQLException e) { - System.out.println(String - .format("%s> %s Host is %s, port is %s.", IOTDB_CLI_PREFIX, e.getMessage(), host, - port)); - } + receiveCommands(reader); } catch (ArgsErrorException e) { - System.out.println(IOTDB_CLI_PREFIX + "> input params error because" + e.getMessage()); + println(IOTDB_CLI_PREFIX + "> input params error because" + e.getMessage()); + handleException(e); } catch (Exception e) { - System.out.println(IOTDB_CLI_PREFIX + "> exit client with error " + e.getMessage()); + println(IOTDB_CLI_PREFIX + "> exit client with error " + e.getMessage()); + handleException(e); } } - /** - * @deprecated this method has been deprecated. - */ - @Deprecated - private static Completer[] getCommandCompleter() { - List candidateStrings = new ArrayList<>(); - for (String s : FUNCTION_NAME_LIST) { - if (!s.endsWith("()")) { - candidateStrings.add(s + "("); - } else { // for functions with no parameter, such as now(). - candidateStrings.add(s + " "); - } - } - for (String s : KEY_WORD_LIST) { - candidateStrings.add(s + " "); - candidateStrings.add(s.toLowerCase() + " "); - } - StringsCompleter strCompleter = new StringsCompleter(candidateStrings); - ArgumentCompleter.ArgumentDelimiter delim = new ArgumentCompleter.AbstractArgumentDelimiter() { - @Override - public boolean isDelimiterChar(CharSequence buffer, int pos) { - char c = buffer.charAt(pos); - return Character.isWhitespace(c) || c == '(' || c == ')' || c == ','; - } - }; - final ArgumentCompleter argCompleter = new ArgumentCompleter(delim, strCompleter); - argCompleter.setStrict(false); - - StringsCompleter confCompleter = new StringsCompleter(Arrays.asList(CONF_NAME_LIST)) { - @Override - public int complete(final String buffer, final int cursor, - final List candidates) { - int result = super.complete(buffer, cursor, candidates); - if (candidates.isEmpty() && cursor > 1 && buffer.charAt(cursor - 1) == '=') { - String confName = buffer.substring(0, cursor - 1); - switch (confName) { // TODO: give config suggestion - default: - break; - } - return cursor; - } - return result; - } - }; - StringsCompleter setCompleter = new StringsCompleter(Arrays.asList("set", "show")) { - @Override - public int complete(String buffer, int cursor, List candidates) { - return buffer != null && ("set".equals(buffer) || "show".equals(buffer)) - ? super.complete(buffer, cursor, candidates) : -1; - } - }; - ArgumentCompleter confPropCompleter = new ArgumentCompleter(setCompleter, confCompleter) { - @Override - public int complete(String buffer, int offset, List completions) { - int ret = super.complete(buffer, offset, completions); - if (completions.size() == 1) { - completions.set(0, ((String) completions.get(0)).trim()); - } - return ret; - } - }; - - StringsCompleter insertConfCompleter = new StringsCompleter(Arrays.asList("into")) { - @Override - public int complete(final String buffer, final int cursor, - final List candidates) { - int result = super.complete(buffer, cursor, candidates); - if (candidates.isEmpty() && cursor > 1 && buffer.charAt(cursor - 1) == '=') { - String confName = buffer.substring(0, cursor - 1); - switch (confName) { // TODO: give config suggestion - default: - break; - } - return cursor; - } - return result; - } - }; - StringsCompleter insertCompleter = new StringsCompleter(Arrays.asList("insert")) { - @Override - public int complete(String buffer, int cursor, List candidates) { - return buffer != null && ("insert".equals(buffer)) ? super - .complete(buffer, cursor, candidates) : -1; - } - }; - ArgumentCompleter insertPropCompleter = new ArgumentCompleter(insertCompleter, - insertConfCompleter) { - @Override - public int complete(String buffer, int offset, List completions) { - int ret = super.complete(buffer, offset, completions); - if (completions.size() == 1) { - completions.set(0, ((String) completions.get(0)).trim()); + private static void receiveCommands(ConsoleReader reader) throws TException, IOException { + try (IoTDBConnection connection = (IoTDBConnection) DriverManager + .getConnection(Config.IOTDB_URL_PREFIX + host + ":" + port + "/", username, password)) { + String s; + properties = connection.getServerProperties(); + AGGREGRATE_TIME_LIST.addAll(properties.getSupportedTimeAggregationOperations()); + displayLogo(properties.getVersion()); + println(IOTDB_CLI_PREFIX + "> login successfully"); + while (true) { + s = reader.readLine(IOTDB_CLI_PREFIX + "> ", null); + boolean continues = processCmd(s, connection); + if (!continues) { + break; } - return ret; } - }; + } catch (SQLException e) { + println(String + .format("%s> %s Host is %s, port is %s.", IOTDB_CLI_PREFIX, e.getMessage(), host, + port)); + handleException(e); + } + } - StringsCompleter withParamCompleter = new StringsCompleter(Arrays.asList(PARAM_NAME_LIST)) { - @Override - public int complete(final String buffer, final int cursor, - final List candidates) { - int result = super.complete(buffer, cursor, candidates); - if (candidates.isEmpty() && cursor > 1) { - int equalsIdx = buffer.indexOf('='); - if (equalsIdx != -1) { - String confName = buffer.substring(0, equalsIdx); - String value = buffer.substring(equalsIdx + 1).toUpperCase(); - if (confName.startsWith("encoding")) { - for (String str : ENCODING_LIST) { - if (str.startsWith(value) && !str.equals(value)) { - candidates.add(str); - } - } - return equalsIdx + 1; - } else if (confName.startsWith("datatype")) { - for (String str : DATA_TYPE_LIST) { - if (str.startsWith(value) && !str.equals(value)) { - candidates.add(str); - } - } - return equalsIdx + 1; - } - return cursor; - } - } - return result; - } - }; - ArgumentCompleter withParamPropCompleter = new ArgumentCompleter(delim, withParamCompleter) { - @Override - public int complete(String buffer, int offset, List completions) { - int ret = super.complete(buffer, offset, completions); - if (completions.size() == 1) { - completions.set(0, ((String) completions.get(0)).trim()); + private static boolean processCmd(String s, IoTDBConnection connection) { + if (s == null) { + return true; + } + String[] cmds = s.trim().split(";"); + for (int i = 0; i < cmds.length; i++) { + String cmd = cmds[i]; + if (cmd != null && !"".equals(cmd.trim())) { + OperationResult result = handleInputCmd(cmd, connection); + switch (result) { + case STOP_OPER: + return false; + case CONTINUE_OPER: + continue; + default: + break; } - return ret; } - }; - withParamPropCompleter.setStrict(false); - - return new Completer[]{confPropCompleter, insertPropCompleter, withParamPropCompleter, - argCompleter}; + } + return true; } } diff --git a/iotdb-cli/src/main/java/org/apache/iotdb/cli/client/WinClient.java b/iotdb-cli/src/main/java/org/apache/iotdb/cli/client/WinClient.java index 4e9e2e59e8356..57885dc981636 100644 --- a/iotdb-cli/src/main/java/org/apache/iotdb/cli/client/WinClient.java +++ b/iotdb-cli/src/main/java/org/apache/iotdb/cli/client/WinClient.java @@ -31,9 +31,12 @@ import org.apache.iotdb.cli.exception.ArgsErrorException; import org.apache.iotdb.jdbc.Config; import org.apache.iotdb.jdbc.IoTDBConnection; +import org.apache.thrift.TException; public class WinClient extends AbstractClient { + private static CommandLine commandLine; + /** * main function. * @@ -44,12 +47,12 @@ public static void main(String[] args) throws ClassNotFoundException, SQLExcepti Options options = createOptions(); HelpFormatter hf = new HelpFormatter(); hf.setWidth(MAX_HELP_CONSOLE_WIDTH); - CommandLine commandLine = null; - CommandLineParser parser = new DefaultParser(); - String[] newArgs = null; + commandLine = null; + + String[] newArgs; if (args == null || args.length == 0) { - System.out.println("Require more params input, please check the following hint."); + println("Require more params input, please check the following hint."); hf.printHelp(IOTDB_CLI_PREFIX, options, true); return; } @@ -58,33 +61,57 @@ public static void main(String[] args) throws ClassNotFoundException, SQLExcepti newArgs = removePasswordArgs(args); + boolean continues = parseCommandLine(options, newArgs, hf); + if (!continues) { + return; + } + + serve(); + } + + private static String readPassword() { + Console c = System.console(); + if (c == null) { // IN ECLIPSE IDE + print(IOTDB_CLI_PREFIX + "> please input password: "); + Scanner scanner = new Scanner(System.in); + return scanner.nextLine(); + } else { // Outside Eclipse IDE + return new String(c.readPassword(IOTDB_CLI_PREFIX + "> please input password: ")); + } + } + + private static boolean parseCommandLine(Options options, String[] newArgs, HelpFormatter hf) { try { + CommandLineParser parser = new DefaultParser(); commandLine = parser.parse(options, newArgs); if (commandLine.hasOption(HELP_ARGS)) { hf.printHelp(IOTDB_CLI_PREFIX, options, true); - return; + return false; } if (commandLine.hasOption(ISO8601_ARGS)) { setTimeFormat("long"); } if (commandLine.hasOption(MAX_PRINT_ROW_COUNT_ARGS)) { - try { - maxPrintRowCount = Integer.valueOf(commandLine.getOptionValue(MAX_PRINT_ROW_COUNT_ARGS)); - if (maxPrintRowCount < 0) { - maxPrintRowCount = Integer.MAX_VALUE; - } - } catch (NumberFormatException e) { - System.out.println( - IOTDB_CLI_PREFIX + "> error format of max print row count, it should be number"); - return; + maxPrintRowCount = Integer.valueOf(commandLine.getOptionValue(MAX_PRINT_ROW_COUNT_ARGS)); + if (maxPrintRowCount < 0) { + maxPrintRowCount = Integer.MAX_VALUE; } } } catch (ParseException e) { - System.out.println("Require more params input, please check the following hint."); + println("Require more params input, please check the following hint."); hf.printHelp(IOTDB_CLI_PREFIX, options, true); - return; + handleException(e); + return false; + } catch (NumberFormatException e) { + println( + IOTDB_CLI_PREFIX + "> error format of max print row count, it should be number"); + handleException(e); + return false; } + return true; + } + private static void serve() { try (Scanner scanner = new Scanner(System.in)) { host = checkRequiredArg(HOST_ARGS, HOST_NAME, commandLine, false, host); port = checkRequiredArg(PORT_ARGS, PORT_NAME, commandLine, false, port); @@ -93,53 +120,58 @@ public static void main(String[] args) throws ClassNotFoundException, SQLExcepti if (password == null) { password = readPassword(); } - try (IoTDBConnection connection = (IoTDBConnection) DriverManager - .getConnection(Config.IOTDB_URL_PREFIX + host + ":" + port + "/", username, password)) { - properties = connection.getServerProperties(); - AGGREGRATE_TIME_LIST.addAll(properties.getSupportedTimeAggregationOperations()); - displayLogo(properties.getVersion()); - System.out.println(IOTDB_CLI_PREFIX + "> login successfully"); - while (true) { - System.out.print(IOTDB_CLI_PREFIX + "> "); - String s = scanner.nextLine(); - if (s != null) { - String[] cmds = s.trim().split(";"); - for (int i = 0; i < cmds.length; i++) { - String cmd = cmds[i]; - if (cmd != null && !"".equals(cmd.trim())) { - OperationResult result = handleInputCmd(cmd, connection); - switch (result) { - case RETURN_OPER: - return; - case CONTINUE_OPER: - continue; - default: - break; - } - } - } - } - } - } catch (SQLException e) { - System.out.println(String - .format("%s> %s Host is %s, port is %s.", IOTDB_CLI_PREFIX, e.getMessage(), host, - port)); - } + receiveCommands(scanner); } catch (ArgsErrorException e) { - System.out.println(IOTDB_CLI_PREFIX + "> input params error because" + e.getMessage()); + println(IOTDB_CLI_PREFIX + "> input params error because" + e.getMessage()); + handleException(e); } catch (Exception e) { - System.out.println(IOTDB_CLI_PREFIX + "> exit client with error " + e.getMessage()); + println(IOTDB_CLI_PREFIX + "> exit client with error " + e.getMessage()); + handleException(e); } } - private static String readPassword() { - Console c = System.console(); - if (c == null) { // IN ECLIPSE IDE - System.out.print(IOTDB_CLI_PREFIX + "> please input password: "); - Scanner scanner = new Scanner(System.in); - return scanner.nextLine(); - } else { // Outside Eclipse IDE - return new String(c.readPassword(IOTDB_CLI_PREFIX + "> please input password: ")); + private static void receiveCommands(Scanner scanner) throws TException { + try (IoTDBConnection connection = (IoTDBConnection) DriverManager + .getConnection(Config.IOTDB_URL_PREFIX + host + ":" + port + "/", username, password)) { + properties = connection.getServerProperties(); + AGGREGRATE_TIME_LIST.addAll(properties.getSupportedTimeAggregationOperations()); + displayLogo(properties.getVersion()); + println(IOTDB_CLI_PREFIX + "> login successfully"); + while (true) { + print(IOTDB_CLI_PREFIX + "> "); + String s = scanner.nextLine(); + boolean continues = processCommand(s, connection); + if (!continues) { + break; + } + } + } catch (SQLException e) { + println(String + .format("%s> %s Host is %s, port is %s.", IOTDB_CLI_PREFIX, e.getMessage(), host, + port)); + handleException(e); + } + } + + private static boolean processCommand(String s, IoTDBConnection connection) { + if (s == null) { + return true; + } + String[] cmds = s.trim().split(";"); + for (int i = 0; i < cmds.length; i++) { + String cmd = cmds[i]; + if (cmd != null && !"".equals(cmd.trim())) { + OperationResult result = handleInputCmd(cmd, connection); + switch (result) { + case STOP_OPER: + return false; + case CONTINUE_OPER: + continue; + default: + break; + } + } } + return true; } } diff --git a/iotdb-cli/src/main/java/org/apache/iotdb/cli/tool/CsvTestDataGen.java b/iotdb-cli/src/main/java/org/apache/iotdb/cli/tool/CsvTestDataGen.java deleted file mode 100644 index b8b497c2092d6..0000000000000 --- a/iotdb-cli/src/main/java/org/apache/iotdb/cli/tool/CsvTestDataGen.java +++ /dev/null @@ -1,125 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iotdb.cli.tool; - -import java.io.BufferedWriter; -import java.io.File; -import java.io.FileNotFoundException; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.OutputStreamWriter; - -@Deprecated -public class CsvTestDataGen { - - private CsvTestDataGen() { - - } - - private static final String PATHS = "Time,root.fit.p.s1,root.fit.d1.s1,root.fit.d1.s2,root.fit.d2." - + "s1,root.fit.d2.s3"; - private static String[] iso = { - PATHS, - "1970-01-01T08:00:00.001+08:00,,1,pass,1,1", "1970-01-01T08:00:00.002+08:00,,2,pass,,", - "1970-01-01T08:00:00.003+08:00,,3,pass,,", "1970-01-01T08:00:00.004+08:00,4,,,4,4"}; - private static String[] defaultLong = { - PATHS, - "1,,1,pass,1,1", - "2,,2,pass,,", "1970-01-01T08:00:00.003+08:00,,3,pass,,", "3,4,,,4,4"}; - private static String[] userSelfDefine = { - PATHS, - "1971,,1,pass,1,1", - "1972,,2,pass,,", "1973-01-01T08:00:00.003+08:00,,3,pass,,", "1974,4,,,4,4"}; - private static FileOutputStream fos = null; - private static OutputStreamWriter osw = null; - private static BufferedWriter bw = null; - private static final String USER_DIR = "user.dir"; - - /** - * generate iso.csv data. - * - * @return path - */ - public static String isoDataGen() { - String path = System.getProperties().getProperty(USER_DIR) + "/src/test/resources/iso.csv"; - File file = new File(path); - writeDataFrom(file, iso); - return path; - } - - /** - * generate default long data file: defaultLong.csv . - * - * @return path - */ - public static String defaultLongDataGen() { - String path = - System.getProperties().getProperty(USER_DIR) + "/src/test/resources/defaultLong.csv"; - File file = new File(path); - writeDataFrom(file, defaultLong); - return path; - } - - /** - * generate user defined data: userSelfDefine.csv . - * - * @return path - */ - public static String userSelfDataGen() { - String path = - System.getProperties().getProperty(USER_DIR) + "/src/test/resources/userSelfDefine.csv"; - File file = new File(path); - writeDataFrom(file, userSelfDefine); - return path; - } - - private static void writeDataFrom(File file, String[] info) { - try { - if (!file.exists()) { - file.createNewFile(); - } - fos = new FileOutputStream(file); - osw = new OutputStreamWriter(fos); - bw = new BufferedWriter(osw); - for (String str : info) { - bw.write(str + "\n"); - } - bw.flush(); - } catch (FileNotFoundException e) { - e.printStackTrace(); - } catch (IOException e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } finally { - try { - bw.close(); - osw.close(); - fos.close(); - } catch (IOException e) { - // TODO Auto-generated catch block - e.printStackTrace(); - } - } - } - - public static void main(String[] args) { - System.out.println(defaultLongDataGen()); - } - -} diff --git a/iotdb-cli/src/main/java/org/apache/iotdb/cli/tool/ExportCsv.java b/iotdb-cli/src/main/java/org/apache/iotdb/cli/tool/ExportCsv.java index 427aa8a22dd82..cae4527bd3757 100644 --- a/iotdb-cli/src/main/java/org/apache/iotdb/cli/tool/ExportCsv.java +++ b/iotdb-cli/src/main/java/org/apache/iotdb/cli/tool/ExportCsv.java @@ -121,23 +121,22 @@ public static void main(String[] args) throws IOException, SQLException { for (int i = 0; i < values.length; i++) { dumpResult(values[i], i); } - return; } else { dumpFromSqlFile(sqlFile); } } catch (ClassNotFoundException e) { LOGGER.error( "Failed to dump data because cannot find TsFile JDBC Driver, " - + "please check whether you have imported driver or not"); + + "please check whether you have imported driver or not", e); } catch (SQLException e) { - LOGGER.error("Encounter an error when dumping data, error is {}", e.getMessage()); + LOGGER.error("Encounter an error when dumping data, error is ", e); } catch (IOException e) { - LOGGER.error("Failed to operate on file, because {}", e.getMessage()); + LOGGER.error("Failed to operate on file, because ", e); } catch (TException e) { - LOGGER.error("Encounter an error when connecting to server, because {}", - e.getMessage()); + LOGGER.error("Encounter an error when connecting to server, because ", + e); } catch (ArgsErrorException e) { - e.printStackTrace(); + LOGGER.error("Invalid args.", e); } finally { reader.close(); if (connection != null) { @@ -222,7 +221,7 @@ private static void dumpFromSqlFile(String filePath) throws IOException { try { dumpResult(sql, index); } catch (SQLException e) { - LOGGER.error("Cannot dump data for statment {}, because {}", sql, e.getMessage()); + LOGGER.error("Cannot dump data for statement {}, because ", sql, e); } index++; } @@ -238,95 +237,99 @@ private static void dumpFromSqlFile(String filePath) throws IOException { */ private static void dumpResult(String sql, int index) throws SQLException { - FileWriter fw = null; - BufferedWriter bw = null; + final String path = targetDirectory + DUMP_FILE_NAME + index + ".csv"; + File tf = new File(path); try { - File tf = new File(path); if (!tf.exists() && !tf.createNewFile()) { LOGGER.error("Could not create target file for sql statement: {}", sql); return; } - fw = new FileWriter(tf); - bw = new BufferedWriter(fw); } catch (IOException e) { LOGGER.error(e.getMessage()); return; } - Statement statement = connection.createStatement(); - ResultSet rs = statement.executeQuery(sql); - ResultSetMetaData metadata = rs.getMetaData(); - long startTime = System.currentTimeMillis(); - try { + try (Statement statement = connection.createStatement(); + ResultSet rs = statement.executeQuery(sql); + BufferedWriter bw = new BufferedWriter(new FileWriter(tf))) { + ResultSetMetaData metadata = rs.getMetaData(); + long startTime = System.currentTimeMillis(); + int count = metadata.getColumnCount(); // write data in csv file - for (int i = 1; i <= count; i++) { - if (i < count) { - bw.write(metadata.getColumnLabel(i) + ","); - } else { - bw.write(metadata.getColumnLabel(i) + "\n"); - } + writeMetadata(bw, count, metadata); + + writeResultSet(rs, bw, count); + LOGGER.info("Statement [{}] has dumped to file {} successfully! It costs {}ms.", + sql, path, System.currentTimeMillis() - startTime); + } catch (IOException e) { + LOGGER.error("Cannot dump result because", e); + } + } + + private static void writeMetadata(BufferedWriter bw, int count, ResultSetMetaData metadata) + throws SQLException, IOException { + for (int i = 1; i <= count; i++) { + if (i < count) { + bw.write(metadata.getColumnLabel(i) + ","); + } else { + bw.write(metadata.getColumnLabel(i) + "\n"); + } + } + } + + private static void writeResultSet(ResultSet rs, BufferedWriter bw, int count) + throws SQLException, IOException { + while (rs.next()) { + if (rs.getString(1) == null || + "null".equalsIgnoreCase(rs.getString(1))) { + bw.write(","); + } else { + writeTime(rs, bw); + writeValue(rs, count, bw); } - while (rs.next()) { - if (rs.getString(1) == null || "null".equalsIgnoreCase(rs.getString(1))) { + } + } + + private static void writeTime(ResultSet rs, BufferedWriter bw) throws SQLException, IOException { + ZonedDateTime dateTime; + switch (timeFormat) { + case DEFAULT_TIME_FORMAT: + case "default": + dateTime = ZonedDateTime.ofInstant(Instant.ofEpochMilli(rs.getLong(1)), + zoneId); + bw.write(dateTime.toString() + ","); + break; + case "timestamp": + case "long": + case "nubmer": + bw.write(rs.getLong(1) + ","); + break; + default: + dateTime = ZonedDateTime.ofInstant(Instant.ofEpochMilli(rs.getLong(1)), + zoneId); + bw.write(dateTime.format(DateTimeFormatter.ofPattern(timeFormat)) + ","); + break; + } + } + + private static void writeValue(ResultSet rs, int count, BufferedWriter bw) + throws SQLException, IOException { + for (int j = 2; j <= count; j++) { + if (j < count) { + if ("null".equals(rs.getString(j))) { bw.write(","); } else { - ZonedDateTime dateTime; - switch (timeFormat) { - case DEFAULT_TIME_FORMAT: - case "default": - dateTime = ZonedDateTime.ofInstant(Instant.ofEpochMilli(rs.getLong(1)), zoneId); - bw.write(dateTime.toString() + ","); - break; - case "timestamp": - case "long": - case "nubmer": - bw.write(rs.getLong(1) + ","); - break; - default: - dateTime = ZonedDateTime.ofInstant(Instant.ofEpochMilli(rs.getLong(1)), zoneId); - bw.write(dateTime.format(DateTimeFormatter.ofPattern(timeFormat)) + ","); - break; - } - - for (int j = 2; j <= count; j++) { - if (j < count) { - if ("null".equals(rs.getString(j))) { - bw.write(","); - } else { - bw.write(rs.getString(j) + ","); - } - } else { - if ("null".equals(rs.getString(j))) { - bw.write("\n"); - } else { - bw.write(rs.getString(j) + "\n"); - } - } - } + bw.write(rs.getString(j) + ","); } - } - LOGGER.info("Statement [{}] has dumped to file {} successfully! It costs {}ms.", - sql, path, System.currentTimeMillis() - startTime); - } catch (IOException e) { - LOGGER.error(e.getMessage()); - } finally { - try { - if (rs != null) { - rs.close(); - } - if (bw != null) { - bw.close(); - } - if (fw != null) { - fw.close(); + } else { + if ("null".equals(rs.getString(j))) { + bw.write("\n"); + } else { + bw.write(rs.getString(j) + "\n"); } - } catch (IOException e) { - LOGGER.error(e.getMessage()); } - statement.close(); } } - } diff --git a/iotdb-cli/src/main/java/org/apache/iotdb/cli/tool/ImportCsv.java b/iotdb-cli/src/main/java/org/apache/iotdb/cli/tool/ImportCsv.java index 21b85d5e7abb5..4ad30fe42300b 100644 --- a/iotdb-cli/src/main/java/org/apache/iotdb/cli/tool/ImportCsv.java +++ b/iotdb-cli/src/main/java/org/apache/iotdb/cli/tool/ImportCsv.java @@ -70,8 +70,11 @@ public class ImportCsv extends AbstractCsvTool { private static final String STRING_DATA_TYPE = "TEXT"; private static final int BATCH_EXECUTE_COUNT = 10; - private static String filename; private static String errorInsertInfo = ""; + private static boolean errorFlag; + + private static int count; + private static Statement statement; /** * create the commandline options. @@ -120,21 +123,21 @@ private static Options createOptions() { * Data from csv To tsfile. */ private static void loadDataFromCSV(File file, int index) { - Statement statement = null; - FileReader fr = null; - BufferedReader br = null; - FileWriter fw = null; - BufferedWriter bw = null; + statement = null; + File errorFile = new File(errorInsertInfo + index); - boolean errorFlag = true; - try { - fr = new FileReader(file); - br = new BufferedReader(fr); - if (!errorFile.exists()) { + if (!errorFile.exists()) { + try { errorFile.createNewFile(); + } catch (IOException e) { + LOGGER.error("Cannot create a errorFile because, ", e); + return; } - fw = new FileWriter(errorFile); - bw = new BufferedWriter(fw); + } + + errorFlag = true; + try(BufferedReader br = new BufferedReader(new FileReader(file)); + BufferedWriter bw = new BufferedWriter(new FileWriter(errorFile))) { String header = br.readLine(); @@ -160,112 +163,34 @@ private static void loadDataFromCSV(File file, int index) { long startTime = System.currentTimeMillis(); Map timeseriesDataType = new HashMap<>(); - DatabaseMetaData databaseMetaData = connection.getMetaData(); - - for (int i = 1; i < strHeadInfo.length; i++) { - ResultSet resultSet = databaseMetaData.getColumns(null, - null, strHeadInfo[i], null); - if (resultSet.next()) { - timeseriesDataType.put(resultSet.getString(1), - resultSet.getString(2)); - } else { - String errorInfo = String.format("Database cannot find %s in %s, stop import!", - strHeadInfo[i], file.getAbsolutePath()); - LOGGER.error("Database cannot find {} in {}, stop import!", - strHeadInfo[i], file.getAbsolutePath()); - bw.write(errorInfo); - errorFlag = false; - return; - } - headInfo.add(strHeadInfo[i]); - String deviceInfo = strHeadInfo[i].substring(0, strHeadInfo[i].lastIndexOf('.')); - if (!deviceToColumn.containsKey(deviceInfo)) { - deviceToColumn.put(deviceInfo, new ArrayList<>()); - } - // storage every device's sensor index info - deviceToColumn.get(deviceInfo).add(i - 1); - colInfo.add(strHeadInfo[i].substring(strHeadInfo[i].lastIndexOf('.') + 1)); + boolean success = queryDatabaseMeta(strHeadInfo, file, bw, timeseriesDataType, headInfo, + deviceToColumn, colInfo); + if (!success) { + errorFlag = false; + return; } - String line; statement = connection.createStatement(); - int count = 0; + + List tmp = new ArrayList<>(); - while ((line = br.readLine()) != null) { - List sqls = new ArrayList<>(); - try { - sqls = createInsertSQL(line, timeseriesDataType, deviceToColumn, colInfo, headInfo); - } catch (Exception e) { - bw.write(String.format("error input line, maybe it is not complete: %s", line)); - bw.newLine(); - errorFlag = false; - } - for (String str : sqls) { - try { - count++; - statement.addBatch(str); - tmp.add(str); - if (count == BATCH_EXECUTE_COUNT) { - int[] result = statement.executeBatch(); - for (int i = 0; i < result.length; i++) { - if (result[i] != Statement.SUCCESS_NO_INFO && i < tmp.size()) { - bw.write(tmp.get(i)); - bw.newLine(); - errorFlag = false; - } - } - statement.clearBatch(); - count = 0; - tmp.clear(); - } - } catch (SQLException e) { - bw.write(e.getMessage()); - bw.newLine(); - errorFlag = false; - } - } - } - try { - int[] result = statement.executeBatch(); - for (int i = 0; i < result.length; i++) { - if (result[i] != Statement.SUCCESS_NO_INFO && i < tmp.size()) { - bw.write(tmp.get(i)); - bw.newLine(); - errorFlag = false; - } - } - statement.clearBatch(); - count = 0; - tmp.clear(); - LOGGER.info("Load data from {} successfully, it takes {}ms", file.getName(), - System.currentTimeMillis() - startTime); - } catch (SQLException e) { - bw.write(e.getMessage()); - bw.newLine(); - errorFlag = false; + success = readAndGenSqls(br, timeseriesDataType, deviceToColumn, colInfo, headInfo, + bw, tmp); + if (!success) { + return; } + executeSqls(bw, tmp, startTime, file); + } catch (FileNotFoundException e) { - LOGGER.error("Cannot find {}", file.getName()); + LOGGER.error("Cannot find {}", file.getName(), e); } catch (IOException e) { - LOGGER.error("CSV file read exception! {}", e.getMessage()); + LOGGER.error("CSV file read exception! ", e); } catch (SQLException e) { - LOGGER.error("Database connection exception! {}", e.getMessage()); + LOGGER.error("Database connection exception!", e); } finally { try { - if (fr != null) { - fr.close(); - } - if (br != null) { - br.close(); - } - if (fw != null) { - fw.close(); - } - if (bw != null) { - bw.close(); - } if (statement != null) { statement.close(); } @@ -276,13 +201,132 @@ private static void loadDataFromCSV(File file, int index) { + "information", file.getAbsolutePath(), errorFile.getAbsolutePath()); } } catch (SQLException e) { - System.out.println("[ERROR] Sql statement can not be closed ! " + e.getMessage()); + LOGGER.error("Sql statement can not be closed ! ", e); } catch (IOException e) { - System.out.println("[ERROR] Close file error ! " + e.getMessage()); + LOGGER.error("Close file error ! ", e); + } + } + } + + private static void executeSqls(BufferedWriter bw, List tmp, long startTime, File file) + throws IOException { + try { + int[] result = statement.executeBatch(); + for (int i = 0; i < result.length; i++) { + if (result[i] != Statement.SUCCESS_NO_INFO && i < tmp.size()) { + bw.write(tmp.get(i)); + bw.newLine(); + errorFlag = false; + } + } + statement.clearBatch(); + tmp.clear(); + LOGGER.info("Load data from {} successfully, it takes {}ms", file.getName(), + System.currentTimeMillis() - startTime); + } catch (SQLException e) { + bw.write(e.getMessage()); + bw.newLine(); + errorFlag = false; + LOGGER.error("Cannot execute sql because ", e); + } + } + + private static boolean readAndGenSqls(BufferedReader br, Map timeseriesDataType, + Map> deviceToColumn, List colInfo, + List headInfo, BufferedWriter bw, List tmp) throws IOException { + String line; + count = 0; + while ((line = br.readLine()) != null) { + List sqls; + try { + sqls = createInsertSQL(line, timeseriesDataType, deviceToColumn, colInfo, headInfo); + } catch (Exception e) { + bw.write(String.format("error input line, maybe it is not complete: %s", line)); + bw.newLine(); + LOGGER.error("Cannot create sql for {} because ", line, e); + errorFlag = false; + return false; + } + boolean success = addSqlsToBatch(sqls, tmp, bw); + if (!success) { + return false; + } + } + return true; + } + + private static boolean addSqlsToBatch(List sqls, List tmp, BufferedWriter bw) + throws IOException { + for (String str : sqls) { + try { + count++; + statement.addBatch(str); + tmp.add(str); + checkBatchSize(bw, tmp); + + } catch (SQLException e) { + bw.write(e.getMessage()); + bw.newLine(); + errorFlag = false; + LOGGER.error("Cannot execute sql because ", e); + return false; + } + } + return true; + } + + + private static void checkBatchSize(BufferedWriter bw, List tmp) + throws SQLException, IOException { + if (count == BATCH_EXECUTE_COUNT) { + int[] result = statement.executeBatch(); + for (int i = 0; i < result.length; i++) { + if (result[i] != Statement.SUCCESS_NO_INFO && i < tmp.size()) { + bw.write(tmp.get(i)); + bw.newLine(); + errorFlag = false; + } } + statement.clearBatch(); + count = 0; + tmp.clear(); } } + private static boolean queryDatabaseMeta(String[] strHeadInfo, File file, BufferedWriter bw, + Map timeseriesDataType, List headInfo, + Map> deviceToColumn, + List colInfo) + throws SQLException, IOException { + DatabaseMetaData databaseMetaData = connection.getMetaData(); + + for (int i = 1; i < strHeadInfo.length; i++) { + ResultSet resultSet = databaseMetaData.getColumns(null, + null, strHeadInfo[i], null); + if (resultSet.next()) { + timeseriesDataType.put(resultSet.getString(1), + resultSet.getString(2)); + } else { + String errorInfo = String.format("Database cannot find %s in %s, stop import!", + strHeadInfo[i], file.getAbsolutePath()); + LOGGER.error("Database cannot find {} in {}, stop import!", + strHeadInfo[i], file.getAbsolutePath()); + bw.write(errorInfo); + return false; + } + headInfo.add(strHeadInfo[i]); + String deviceInfo = strHeadInfo[i].substring(0, strHeadInfo[i].lastIndexOf('.')); + + if (!deviceToColumn.containsKey(deviceInfo)) { + deviceToColumn.put(deviceInfo, new ArrayList<>()); + } + // storage every device's sensor index info + deviceToColumn.get(deviceInfo).add(i - 1); + colInfo.add(strHeadInfo[i].substring(strHeadInfo[i].lastIndexOf('.') + 1)); + } + return true; + } + private static List createInsertSQL(String line, Map timeseriesToType, Map> deviceToColumn, List colInfo, List headInfo) { @@ -291,42 +335,50 @@ private static List createInsertSQL(String line, Map tim Iterator>> it = deviceToColumn.entrySet().iterator(); while (it.hasNext()) { Map.Entry> entry = it.next(); - StringBuilder sbd = new StringBuilder(); - ArrayList colIndex = entry.getValue(); - sbd.append("insert into " + entry.getKey() + "(timestamp"); - int skipCount = 0; - for (int j = 0; j < colIndex.size(); ++j) { - if ("".equals(data[entry.getValue().get(j) + 1])) { - skipCount++; - continue; - } - sbd.append(", " + colInfo.get(colIndex.get(j))); + String sql = createOneSql(entry, data, colInfo, timeseriesToType, headInfo); + if (sql != null) { + sqls.add(sql); } - // define every device null value' number, if the number equal the - // sensor number, the insert operation stop - if (skipCount == entry.getValue().size()) { + } + return sqls; + } + + private static String createOneSql(Map.Entry> entry, String[] data, + List colInfo, Map timeseriesToType, List headInfo) { + StringBuilder sbd = new StringBuilder(); + ArrayList colIndex = entry.getValue(); + sbd.append("insert into ").append(entry.getKey()).append("(timestamp"); + int skipCount = 0; + for (int j = 0; j < colIndex.size(); ++j) { + if ("".equals(data[entry.getValue().get(j) + 1])) { + skipCount++; continue; } + sbd.append(", ").append(colInfo.get(colIndex.get(j))); + } + // define every device null value' number, if the number equal the + // sensor number, the insert operation stop + if (skipCount == entry.getValue().size()) { + return null; + } - // TODO when timestampsStr is empty - String timestampsStr = data[0]; - sbd.append(") values(").append(timestampsStr.trim().isEmpty() - ? "NO TIMESTAMP" : timestampsStr); + // TODO when timestampsStr is empty + String timestampsStr = data[0]; + sbd.append(") values(").append(timestampsStr.trim().isEmpty() + ? "NO TIMESTAMP" : timestampsStr); - for (int j = 0; j < colIndex.size(); ++j) { - if ("".equals(data[entry.getValue().get(j) + 1])) { - continue; - } - if (timeseriesToType.get(headInfo.get(colIndex.get(j))).equals(STRING_DATA_TYPE)) { - sbd.append(", \'" + data[colIndex.get(j) + 1] + "\'"); - } else { - sbd.append("," + data[colIndex.get(j) + 1]); - } + for (int j = 0; j < colIndex.size(); ++j) { + if ("".equals(data[entry.getValue().get(j) + 1])) { + continue; + } + if (timeseriesToType.get(headInfo.get(colIndex.get(j))).equals(STRING_DATA_TYPE)) { + sbd.append(", \'").append(data[colIndex.get(j) + 1]).append("\'"); + } else { + sbd.append(",").append(data[colIndex.get(j) + 1]); } - sbd.append(")"); - sqls.add(sbd.toString()); } - return sqls; + sbd.append(")"); + return sbd.toString(); } public static void main(String[] args) throws IOException, SQLException { @@ -334,7 +386,7 @@ public static void main(String[] args) throws IOException, SQLException { HelpFormatter hf = new HelpFormatter(); hf.setOptionComparator(null); hf.setWidth(MAX_HELP_CONSOLE_WIDTH); - CommandLine commandLine = null; + CommandLine commandLine; CommandLineParser parser = new DefaultParser(); if (args == null || args.length == 0) { @@ -345,7 +397,7 @@ public static void main(String[] args) throws IOException, SQLException { try { commandLine = parser.parse(options, args); } catch (ParseException e) { - LOGGER.error(e.getMessage()); + LOGGER.error("Parse error ", e); hf.printHelp(TSFILEDB_CLI_PREFIX, options, true); return; } @@ -358,7 +410,7 @@ public static void main(String[] args) throws IOException, SQLException { reader.setExpandEvents(false); try { parseBasicParams(commandLine, reader); - filename = commandLine.getOptionValue(FILE_ARGS); + String filename = commandLine.getOptionValue(FILE_ARGS); if (filename == null) { hf.printHelp(TSFILEDB_CLI_PREFIX, options, true); return; @@ -366,9 +418,9 @@ public static void main(String[] args) throws IOException, SQLException { parseSpecialParams(commandLine); importCsvFromFile(host, port, username, password, filename, timeZoneID); } catch (ArgsErrorException e) { - // ignored + LOGGER.error("Args error", e); } catch (Exception e) { - LOGGER.error("Encounter an error, because {}", e.getMessage()); + LOGGER.error("Encounter an error, because ", e); } finally { reader.close(); } @@ -405,12 +457,12 @@ public static void importCsvFromFile(String ip, String port, String username, } catch (ClassNotFoundException e) { LOGGER.error( "Failed to dump data because cannot find TsFile JDBC Driver, " - + "please check whether you have imported driver or not"); + + "please check whether you have imported driver or not", e); } catch (TException e) { - LOGGER.error("Encounter an error when connecting to server, because {}", - e.getMessage()); + LOGGER.error("Encounter an error when connecting to server, because ", + e); } catch (Exception e) { - LOGGER.error("Encounter an error, because {}", e.getMessage()); + LOGGER.error("Encounter an error, because ", e); } finally { if (connection != null) { connection.close(); @@ -428,7 +480,12 @@ private static void importFromSingleFile(File file) { private static void importFromDirectory(File file) { int i = 1; - for (File subFile : file.listFiles()) { + File[] files = file.listFiles(); + if (files == null) { + return; + } + + for (File subFile : files) { if (subFile.isFile()) { if (subFile.getName().endsWith(FILE_SUFFIX)) { loadDataFromCSV(subFile, i); diff --git a/iotdb-cli/src/test/java/org/apache/iotdb/cli/client/AbstractClientIT.java b/iotdb-cli/src/test/java/org/apache/iotdb/cli/client/AbstractClientIT.java index 275bb9b1393e6..2fe6ee80593c4 100644 --- a/iotdb-cli/src/test/java/org/apache/iotdb/cli/client/AbstractClientIT.java +++ b/iotdb-cli/src/test/java/org/apache/iotdb/cli/client/AbstractClientIT.java @@ -141,8 +141,8 @@ private void isTwoStringArrayEqual(String[] expected, String[] actual) { @Test public void testHandleInputInputCmd() { - assertEquals(OperationResult.RETURN_OPER, AbstractClient.handleInputCmd(AbstractClient.EXIT_COMMAND, connection)); - assertEquals(OperationResult.RETURN_OPER, AbstractClient.handleInputCmd(AbstractClient.QUIT_COMMAND, connection)); + assertEquals(OperationResult.STOP_OPER, AbstractClient.handleInputCmd(AbstractClient.EXIT_COMMAND, connection)); + assertEquals(OperationResult.STOP_OPER, AbstractClient.handleInputCmd(AbstractClient.QUIT_COMMAND, connection)); assertEquals(OperationResult.CONTINUE_OPER, AbstractClient.handleInputCmd(AbstractClient.SHOW_METADATA_COMMAND, connection)); diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/FileNodeManager.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/FileNodeManager.java index f4e4864ebd443..a88f076edb900 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/FileNodeManager.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/FileNodeManager.java @@ -30,7 +30,6 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; - import org.apache.commons.io.FileUtils; import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBConstant; @@ -75,6 +74,10 @@ public class FileNodeManager implements IStatistic, IService { private static final Logger LOGGER = LoggerFactory.getLogger(FileNodeManager.class); private static final IoTDBConfig TsFileDBConf = IoTDBDescriptor.getInstance().getConfig(); private static final Directories directories = Directories.getInstance(); + /** + * a folder that persist FileNodeProcessorStore classes. Each stroage group will have a subfolder. + * by default, it is system/info + */ private final String baseDir; /** @@ -92,6 +95,7 @@ public class FileNodeManager implements IStatistic, IService { private FileNodeManager(String baseDir) { processorMap = new ConcurrentHashMap<>(); statParamsHashMap = new HashMap<>(); + //label: A for (MonitorConstants.FileNodeManagerStatConstants fileNodeManagerStatConstant : MonitorConstants.FileNodeManagerStatConstants.values()) { statParamsHashMap.put(fileNodeManagerStatConstant.name(), new AtomicLong(0)); @@ -106,7 +110,7 @@ private FileNodeManager(String baseDir) { if (dir.mkdirs()) { LOGGER.info("{} dir home doesn't exist, create it", dir.getPath()); } - + //TODO merge this with label A if (TsFileDBConf.enableStatMonitor) { StatMonitor statMonitor = StatMonitor.getInstance(); registStatMetadata(); @@ -182,6 +186,12 @@ public synchronized void resetFileNodeManager() { processorMap.clear(); } + /** + * + * @param filenodeName storage name, e.g., root.a.b + * @return + * @throws FileNodeManagerException + */ private FileNodeProcessor constructNewProcessor(String filenodeName) throws FileNodeManagerException { try { @@ -196,6 +206,7 @@ private FileNodeProcessor getProcessor(String path, boolean isWriteLock) throws FileNodeManagerException { String filenodeName; try { + // return the stroage name filenodeName = MManager.getInstance().getFileNameByPath(path); } catch (PathErrorException e) { LOGGER.error("MManager get filenode name error, seriesPath is {}", path); @@ -214,8 +225,8 @@ private FileNodeProcessor getProcessor(String path, boolean isWriteLock) processor.lock(isWriteLock); } else { // calculate the value with the key monitor - LOGGER.debug("Calcuate the processor, the filenode is {}, Thread is {}", filenodeName, - Thread.currentThread().getId()); + LOGGER.debug("construct a processor instance, the filenode is {}, Thread is {}", + filenodeName, Thread.currentThread().getId()); processor = constructNewProcessor(filenodeName); processor.lock(isWriteLock); processorMap.put(filenodeName, processor); @@ -229,22 +240,30 @@ private FileNodeProcessor getProcessor(String path, boolean isWriteLock) * recovery the filenode processor. */ public void recovery() { - + List filenodeNames = null; try { - List filenodeNames = MManager.getInstance().getAllFileNames(); - for (String filenodeName : filenodeNames) { - FileNodeProcessor fileNodeProcessor = getProcessor(filenodeName, true); + filenodeNames = MManager.getInstance().getAllFileNames(); + } catch (PathErrorException e) { + LOGGER.error("Restoring all FileNodes failed.", e); + return; + } + for (String filenodeName : filenodeNames) { + FileNodeProcessor fileNodeProcessor = null; + try { + fileNodeProcessor = getProcessor(filenodeName, true); if (fileNodeProcessor.shouldRecovery()) { LOGGER.info("Recovery the filenode processor, the filenode is {}, the status is {}", filenodeName, fileNodeProcessor.getFileNodeProcessorStatus()); fileNodeProcessor.fileNodeRecovery(); - } else { + } + } catch (FileNodeManagerException | FileNodeProcessorException e) { + LOGGER.error("Restoring fileNode {} failed.", filenodeName, e); + } finally { + if (fileNodeProcessor != null) { fileNodeProcessor.writeUnlock(); } - // add index check sum } - } catch (PathErrorException | FileNodeManagerException | FileNodeProcessorException e) { - LOGGER.error("Restoring all FileNodes failed, the reason is ", e); + // add index check sum } } @@ -607,6 +626,8 @@ private void delete(String processorName, /** * begin query. + * @param deviceId queried deviceId + * @return a query token for the device. */ public int beginQuery(String deviceId) throws FileNodeManagerException { FileNodeProcessor fileNodeProcessor = getProcessor(deviceId, true); @@ -827,7 +848,6 @@ public void deleteOneFileNode(String processorName) throws FileNodeManagerExcept fileNodePath = standardizeDir(fileNodePath) + processorName; FileUtils.deleteDirectory(new File(fileNodePath)); - cleanBufferWrite(processorName); cleanBufferWrite(processorName); MultiFileLogNodeManager.getInstance() @@ -1150,6 +1170,8 @@ public void recoverFileNode(String filenodeName) fileNodeProcessor.fileNodeRecovery(); } catch (FileNodeProcessorException e) { throw new FileNodeManagerException(e); + } finally { + fileNodeProcessor.writeUnlock(); } } diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/FileNodeProcessor.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/FileNodeProcessor.java index a7c4e777e8674..7cdeecb95cdf4 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/FileNodeProcessor.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/FileNodeProcessor.java @@ -462,8 +462,6 @@ public void fileNodeRecovery() throws FileNodeProcessorException { bufferWriteProcessor = new BufferWriteProcessor(baseDir, getProcessorName(), fileNames[fileNames.length - 1], parameters, versionController, fileSchema); } catch (BufferWriteProcessorException e) { - // unlock - writeUnlock(); LOGGER.error( "The filenode processor {} failed to recovery the bufferwrite processor, " + "the last bufferwrite file is {}.", @@ -480,7 +478,6 @@ public void fileNodeRecovery() throws FileNodeProcessorException { overflowProcessor = new OverflowProcessor(getProcessorName(), parameters, fileSchema, versionController); } catch (IOException e) { - writeUnlock(); LOGGER.error("The filenode processor {} failed to recovery the overflow processor.", getProcessorName()); throw new FileNodeProcessorException(e); @@ -498,10 +495,10 @@ public void fileNodeRecovery() throws FileNodeProcessorException { // unlock LOGGER.info("The filenode processor {} is recovering, the filenode status is {}.", getProcessorName(), isMerging); - writeUnlock(); + //writeUnlock(); switchWaitingToWorking(); } else { - writeUnlock(); + //writeUnlock(); } // add file into index of file addAllFileIntoIndex(newFileNodes); diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/TimeIntervalTsFile.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/TimeIntervalTsFile.java deleted file mode 100644 index 8dea600e657cc..0000000000000 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/TimeIntervalTsFile.java +++ /dev/null @@ -1,287 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iotdb.db.engine.filenode; - -import java.io.File; -import java.io.Serializable; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; -import org.apache.iotdb.db.conf.directories.Directories; - -/** - * This class is used to store the TsFile status.
- */ -public class TimeIntervalTsFile implements Serializable { - - private static final long serialVersionUID = -4309683416067212549L; - public OverflowChangeType overflowChangeType; - private int baseDirIndex; - private String relativePath; - private Map startTimeMap; - private Map endTimeMap; - private Set mergeChanged = new HashSet<>(); - - /** - * construct function for TimeIntervalTsFile. - */ - public TimeIntervalTsFile(Map startTimeMap, Map endTimeMap, - OverflowChangeType type, - int baseDirIndex, String relativePath) { - - this.overflowChangeType = type; - this.baseDirIndex = baseDirIndex; - this.relativePath = relativePath; - - this.startTimeMap = startTimeMap; - this.endTimeMap = endTimeMap; - - } - - /** - * This is just used to construct a new TsFile. - */ - public TimeIntervalTsFile(OverflowChangeType type, String relativePath) { - - this.overflowChangeType = type; - this.relativePath = relativePath; - - startTimeMap = new HashMap<>(); - endTimeMap = new HashMap<>(); - } - - public void setStartTime(String deviceId, long startTime) { - - startTimeMap.put(deviceId, startTime); - } - - /** - * get start time. - * - * @param deviceId -Map key - * @return -start time - */ - public long getStartTime(String deviceId) { - - if (startTimeMap.containsKey(deviceId)) { - return startTimeMap.get(deviceId); - } else { - return -1; - } - } - - public Map getStartTimeMap() { - - return startTimeMap; - } - - public void setStartTimeMap(Map startTimeMap) { - - this.startTimeMap = startTimeMap; - } - - public void setEndTime(String deviceId, long timestamp) { - - this.endTimeMap.put(deviceId, timestamp); - } - - /** - * get end time for given device. - * - * @param deviceId -id of device - * @return -end time of the device - */ - public long getEndTime(String deviceId) { - - if (endTimeMap.get(deviceId) == null) { - return -1; - } - return endTimeMap.get(deviceId); - } - - public Map getEndTimeMap() { - - return endTimeMap; - } - - public void setEndTimeMap(Map endTimeMap) { - - this.endTimeMap = endTimeMap; - } - - /** - * remove given device'startTime start time and end time. - * - * @param deviceId -id of the device - */ - public void removeTime(String deviceId) { - - startTimeMap.remove(deviceId); - endTimeMap.remove(deviceId); - } - - /** - * get file path. - */ - public String getFilePath() { - - if (relativePath == null) { - return relativePath; - } - return new File(Directories.getInstance().getTsFileFolder(baseDirIndex), relativePath) - .getPath(); - } - - public String getRelativePath() { - - return relativePath; - } - - public void setRelativePath(String relativePath) { - - this.relativePath = relativePath; - } - - public boolean checkEmpty() { - - return startTimeMap.isEmpty() && endTimeMap.isEmpty(); - } - - /** - * clear the member variable of the given object. - */ - public void clear() { - - startTimeMap.clear(); - endTimeMap.clear(); - mergeChanged.clear(); - overflowChangeType = OverflowChangeType.NO_CHANGE; - relativePath = null; - } - - /** - * change file type corresponding to the given param. - */ - public void changeTypeToChanged(FileNodeProcessorStatus fileNodeProcessorState) { - - if (fileNodeProcessorState == FileNodeProcessorStatus.MERGING_WRITE) { - overflowChangeType = OverflowChangeType.MERGING_CHANGE; - } else { - overflowChangeType = OverflowChangeType.CHANGED; - } - } - - public void addMergeChanged(String deviceId) { - - mergeChanged.add(deviceId); - } - - public Set getMergeChanged() { - - return mergeChanged; - } - - public void clearMergeChanged() { - - mergeChanged.clear(); - } - - /** - * judge whether the time interval is closed. - */ - public boolean isClosed() { - - return !endTimeMap.isEmpty(); - - } - - /** - * back up the time interval of tsfile. - */ - public TimeIntervalTsFile backUp() { - - Map startTimeMap = new HashMap<>(this.startTimeMap); - Map endTimeMap = new HashMap<>(this.endTimeMap); - return new TimeIntervalTsFile(startTimeMap, endTimeMap, overflowChangeType, baseDirIndex, - relativePath); - } - - @Override - public int hashCode() { - - final int prime = 31; - int result = 1; - result = prime * result + ((endTimeMap == null) ? 0 : endTimeMap.hashCode()); - result = prime * result + ((relativePath == null) ? 0 : relativePath.hashCode()); - result = prime * result + ((overflowChangeType == null) ? 0 : overflowChangeType.hashCode()); - result = prime * result + ((startTimeMap == null) ? 0 : startTimeMap.hashCode()); - return result; - } - - @Override - public boolean equals(Object obj) { - - if (this == obj) { - return true; - } - if (obj == null) { - return false; - } - if (getClass() != obj.getClass()) { - return false; - } - TimeIntervalTsFile other = (TimeIntervalTsFile) obj; - if (endTimeMap == null) { - if (other.endTimeMap != null) { - return false; - } - } else if (!endTimeMap.equals(other.endTimeMap)) { - return false; - } - if (relativePath == null) { - if (other.relativePath != null) { - return false; - } - } else if (!relativePath.equals(other.relativePath)) { - return false; - } - if (overflowChangeType != other.overflowChangeType) { - return false; - } - if (startTimeMap == null) { - if (other.startTimeMap != null) { - return false; - } - } else if (!startTimeMap.equals(other.startTimeMap)) { - return false; - } - return true; - } - - @Override - public String toString() { - return "TimeIntervalTsFile [relativePath=" + relativePath + ", overflowChangeType=" - + overflowChangeType - + ", startTimeMap=" + startTimeMap + ", endTimeMap=" + endTimeMap + ", mergeChanged=" - + mergeChanged - + "]"; - } - -} diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/io/LocalTextModificationAccessor.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/io/LocalTextModificationAccessor.java index 249b18a84f56f..5d2d8fd757c8b 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/io/LocalTextModificationAccessor.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/io/LocalTextModificationAccessor.java @@ -87,6 +87,7 @@ public Collection read() throws IOException { public void close() throws IOException { if (writer != null) { writer.close(); + writer = null; } } diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/control/FileReaderManager.java b/iotdb/src/main/java/org/apache/iotdb/db/query/control/FileReaderManager.java index 6e3c8bdee9f4d..ade802994c82b 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/control/FileReaderManager.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/control/FileReaderManager.java @@ -47,22 +47,34 @@ public class FileReaderManager implements IService { private static final int MAX_CACHED_FILE_SIZE = 30000; /** - * the key of fileReaderMap is the file path and the value of fileReaderMap is the corresponding - * reader. + * the key of closedFileReaderMap is the file path and the value of closedFileReaderMap + * is the corresponding reader. */ - private ConcurrentHashMap fileReaderMap; + private ConcurrentHashMap closedFileReaderMap; + /** + * the key of unclosedFileReaderMap is the file path and the value of unclosedFileReaderMap + * is the corresponding reader. + */ + private ConcurrentHashMap unclosedFileReaderMap; /** - * the key of fileReaderMap is the file path and the value of fileReaderMap is the file's - * reference count. + * the key of closedFileReaderMap is the file path and the value of closedFileReaderMap + * is the file's reference count. + */ + private ConcurrentHashMap closedReferenceMap; + /** + * the key of unclosedFileReaderMap is the file path and the value of unclosedFileReaderMap + * is the file's reference count. */ - private ConcurrentHashMap referenceMap; + private ConcurrentHashMap unclosedReferenceMap; private ScheduledExecutorService executorService; private FileReaderManager() { - fileReaderMap = new ConcurrentHashMap<>(); - referenceMap = new ConcurrentHashMap<>(); + closedFileReaderMap = new ConcurrentHashMap<>(); + unclosedFileReaderMap = new ConcurrentHashMap<>(); + closedReferenceMap = new ConcurrentHashMap<>(); + unclosedReferenceMap = new ConcurrentHashMap<>(); executorService = IoTDBThreadPoolFactory.newScheduledThreadPool(1, "opended-files-manager"); @@ -79,76 +91,99 @@ private void clearUnUsedFilesInFixTime() { executorService.scheduleAtFixedRate(() -> { synchronized (this) { - for (Map.Entry entry : fileReaderMap.entrySet()) { - TsFileSequenceReader reader = entry.getValue(); - int referenceNum = referenceMap.get(entry.getKey()).get(); - - if (referenceNum == 0) { - try { - reader.close(); - } catch (IOException e) { - LOGGER.error("Can not close TsFileSequenceReader {} !", reader.getFileName(), e); - } - fileReaderMap.remove(entry.getKey()); - referenceMap.remove(entry.getKey()); - } - } + clearMap(unclosedFileReaderMap, unclosedReferenceMap); + clearMap(closedFileReaderMap, closedReferenceMap); } }, 0, examinePeriod, TimeUnit.MILLISECONDS); } + private void clearMap(Map readerMap, + Map refMap) { + for (Map.Entry entry : readerMap.entrySet()) { + TsFileSequenceReader reader = entry.getValue(); + int referenceNum = refMap.get(entry.getKey()).get(); + + if (referenceNum == 0) { + try { + reader.close(); + } catch (IOException e) { + LOGGER.error("Can not close TsFileSequenceReader {} !", reader.getFileName(), e); + } + readerMap.remove(entry.getKey()); + refMap.remove(entry.getKey()); + } + } + } + /** * Get the reader of the file(tsfile or unseq tsfile) indicated by filePath. If the reader already - * exists, just get it from fileReaderMap. Otherwise a new reader will be created. + * exists, just get it from closedFileReaderMap or unclosedFileReaderMap depending on isClosed . + * Otherwise a new reader will be created and cached. * * @param filePath the path of the file, of which the reader is desired. - * @param isUnClosed whether the corresponding file still receives insertions or not. + * @param isClosed whether the corresponding file still receives insertions or not. * @return the reader of the file specified by filePath. * @throws IOException when reader cannot be created. */ - public synchronized TsFileSequenceReader get(String filePath, boolean isUnClosed) + public synchronized TsFileSequenceReader get(String filePath, boolean isClosed) throws IOException { - if (!fileReaderMap.containsKey(filePath)) { + Map readerMap = !isClosed ? unclosedFileReaderMap + : closedFileReaderMap; + if (!readerMap.containsKey(filePath)) { - if (fileReaderMap.size() >= MAX_CACHED_FILE_SIZE) { - LOGGER.warn("Query has opened {} files !", fileReaderMap.size()); + if (readerMap.size() >= MAX_CACHED_FILE_SIZE) { + LOGGER.warn("Query has opened {} files !", readerMap.size()); } - TsFileSequenceReader tsFileReader = isUnClosed ? new UnClosedTsFileReader(filePath) + TsFileSequenceReader tsFileReader = !isClosed ? new UnClosedTsFileReader(filePath) : new TsFileSequenceReader(filePath); - fileReaderMap.put(filePath, tsFileReader); + readerMap.put(filePath, tsFileReader); return tsFileReader; } - return fileReaderMap.get(filePath); + return readerMap.get(filePath); } /** * Increase the reference count of the reader specified by filePath. Only when the reference count * of a reader equals zero, the reader can be closed and removed. */ - public synchronized void increaseFileReaderReference(String filePath) { - referenceMap.computeIfAbsent(filePath, k -> new AtomicInteger()).getAndIncrement(); + public synchronized void increaseFileReaderReference(String filePath, boolean isClosed) { + if (!isClosed) { + unclosedReferenceMap.computeIfAbsent(filePath, k -> new AtomicInteger()).getAndIncrement(); + } else { + closedReferenceMap.computeIfAbsent(filePath, k -> new AtomicInteger()).getAndIncrement(); + } } /** * Decrease the reference count of the reader specified by filePath. This method is latch-free. * Only when the reference count of a reader equals zero, the reader can be closed and removed. */ - public synchronized void decreaseFileReaderReference(String filePath) { - referenceMap.get(filePath).getAndDecrement(); + public synchronized void decreaseFileReaderReference(String filePath, boolean isClosed) { + if (!isClosed && unclosedReferenceMap.containsKey(filePath)) { + unclosedReferenceMap.get(filePath).getAndDecrement(); + } else if (closedReferenceMap.containsKey(filePath)){ + closedReferenceMap.get(filePath).getAndDecrement(); + } } /** * This method is used when the given file path is deleted. */ - public synchronized void closeFileAndRemoveReader(String filePath) throws IOException { - if (fileReaderMap.containsKey(filePath)) { - referenceMap.remove(filePath); - fileReaderMap.get(filePath).close(); - fileReaderMap.remove(filePath); + public synchronized void closeFileAndRemoveReader(String filePath) + throws IOException { + if (unclosedFileReaderMap.containsKey(filePath)) { + unclosedReferenceMap.remove(filePath); + unclosedFileReaderMap.get(filePath).close(); + unclosedFileReaderMap.remove(filePath); + } + if (closedFileReaderMap.containsKey(filePath)) { + closedReferenceMap.remove(filePath); + closedFileReaderMap.get(filePath).close(); + closedFileReaderMap.remove(filePath); } } @@ -157,18 +192,24 @@ public synchronized void closeFileAndRemoveReader(String filePath) throws IOExce * integration tests will not conflict with each other. */ public synchronized void closeAndRemoveAllOpenedReaders() throws IOException { - for (Map.Entry entry : fileReaderMap.entrySet()) { + for (Map.Entry entry : closedFileReaderMap.entrySet()) { + entry.getValue().close(); + closedReferenceMap.remove(entry.getKey()); + closedFileReaderMap.remove(entry.getKey()); + } + for (Map.Entry entry : unclosedFileReaderMap.entrySet()) { entry.getValue().close(); - referenceMap.remove(entry.getKey()); - fileReaderMap.remove(entry.getKey()); + unclosedReferenceMap.remove(entry.getKey()); + unclosedFileReaderMap.remove(entry.getKey()); } } /** * This method is only for unit tests. */ - public synchronized boolean contains(String filePath) { - return fileReaderMap.containsKey(filePath); + public synchronized boolean contains(String filePath, boolean isClosed) { + return (isClosed && closedFileReaderMap.containsKey(filePath)) + || (!isClosed && unclosedFileReaderMap.containsKey(filePath)); } @Override diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/control/OpenedFilePathsManager.java b/iotdb/src/main/java/org/apache/iotdb/db/query/control/OpenedFilePathsManager.java index c68ad105fd4b5..b2c1452b19615 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/control/OpenedFilePathsManager.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/control/OpenedFilePathsManager.java @@ -40,11 +40,13 @@ public class OpenedFilePathsManager { /** * Map> */ - private ConcurrentHashMap> filePathsMap; + private ConcurrentHashMap> closedFilePathsMap; + private ConcurrentHashMap> unclosedFilePathsMap; private OpenedFilePathsManager() { jobIdContainer = new ThreadLocal<>(); - filePathsMap = new ConcurrentHashMap<>(); + closedFilePathsMap = new ConcurrentHashMap<>(); + unclosedFilePathsMap = new ConcurrentHashMap<>(); } public static OpenedFilePathsManager getInstance() { @@ -56,27 +58,29 @@ public static OpenedFilePathsManager getInstance() { */ public void setJobIdForCurrentRequestThread(long jobId) { jobIdContainer.set(jobId); - filePathsMap.put(jobId, new HashSet<>()); + closedFilePathsMap.put(jobId, new HashSet<>()); + unclosedFilePathsMap.put(jobId, new HashSet<>()); } /** - * Add the unique file paths to filePathsMap. + * 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(); - addFilePathToMap(jobId, sealedFilePath); + addFilePathToMap(jobId, sealedFilePath, true); } if (dataSource.getSeqDataSource().hasUnsealedTsFile()) { String unSealedFilePath = dataSource.getSeqDataSource().getUnsealedTsFile().getFilePath(); - addFilePathToMap(jobId, unSealedFilePath); + addFilePathToMap(jobId, unSealedFilePath, false); } for (OverflowInsertFile overflowInsertFile : dataSource.getOverflowSeriesDataSource() .getOverflowInsertFileList()) { String overflowFilePath = overflowInsertFile.getFilePath(); - addFilePathToMap(jobId, overflowFilePath); + // overflow is unclosed by default + addFilePathToMap(jobId, overflowFilePath, false); } } @@ -89,22 +93,29 @@ public void removeUsedFilesForCurrentRequestThread() { long jobId = jobIdContainer.get(); jobIdContainer.remove(); - for (String filePath : filePathsMap.get(jobId)) { - FileReaderManager.getInstance().decreaseFileReaderReference(filePath); + for (String filePath : closedFilePathsMap.get(jobId)) { + FileReaderManager.getInstance().decreaseFileReaderReference(filePath, false); } - filePathsMap.remove(jobId); + closedFilePathsMap.remove(jobId); + for (String filePath : unclosedFilePathsMap.get(jobId)) { + FileReaderManager.getInstance().decreaseFileReaderReference(filePath, true); + } + unclosedFilePathsMap.remove(jobId); } } /** * Increase the usage reference of filePath of job id. Before the invoking of this method, - * this.setJobIdForCurrentRequestThread has been invoked, so filePathsMap.get(jobId) must - * not return null. + * this.setJobIdForCurrentRequestThread has been invoked, + * so closedFilePathsMap.get(jobId) or unclosedFilePathsMap.get(jobId) + * must not return null. */ - public void addFilePathToMap(long jobId, String filePath) { - if (!filePathsMap.get(jobId).contains(filePath)) { - filePathsMap.get(jobId).add(filePath); - FileReaderManager.getInstance().increaseFileReaderReference(filePath); + public void addFilePathToMap(long jobId, String filePath, boolean isClosed) { + ConcurrentHashMap> pathMap = !isClosed ? unclosedFilePathsMap : + closedFilePathsMap; + if (!pathMap.get(jobId).contains(filePath)) { + pathMap.get(jobId).add(filePath); + FileReaderManager.getInstance().increaseFileReaderReference(filePath, isClosed); } } diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/factory/SeriesReaderFactory.java b/iotdb/src/main/java/org/apache/iotdb/db/query/factory/SeriesReaderFactory.java index f9e43b3b12b4f..d4a80756ee8fe 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/factory/SeriesReaderFactory.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/factory/SeriesReaderFactory.java @@ -81,7 +81,7 @@ public PriorityMergeReader createUnSeqMergeReader( // store only one opened file stream into manager, to avoid too many opened files TsFileSequenceReader unClosedTsFileReader = FileReaderManager.getInstance() - .get(overflowInsertFile.getFilePath(), true); + .get(overflowInsertFile.getFilePath(), false); ChunkLoaderImpl chunkLoader = new ChunkLoaderImpl(unClosedTsFileReader); @@ -142,7 +142,7 @@ public PriorityMergeReaderByTimestamp createUnSeqMergeReaderByTimestamp( // store only one opened file stream into manager, to avoid too many opened files TsFileSequenceReader unClosedTsFileReader = FileReaderManager.getInstance() - .get(overflowInsertFile.getFilePath(), true); + .get(overflowInsertFile.getFilePath(), false); ChunkLoaderImpl chunkLoader = new ChunkLoaderImpl(unClosedTsFileReader); @@ -204,7 +204,7 @@ private IReader createSealedTsFileReaderForMerge(IntervalFileNode fileNode, QueryContext context) throws IOException { TsFileSequenceReader tsFileSequenceReader = FileReaderManager.getInstance() - .get(fileNode.getFilePath(), false); + .get(fileNode.getFilePath(), true); ChunkLoaderImpl chunkLoader = new ChunkLoaderImpl(tsFileSequenceReader); MetadataQuerier metadataQuerier = new MetadataQuerierByFileImpl(tsFileSequenceReader); List metaDataList = metadataQuerier diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReader.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReader.java index 4db8c9d6ca89c..a8465bf66ed4a 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReader.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReader.java @@ -171,7 +171,7 @@ private void initSingleTsFileReader(IntervalFileNode fileNode, QueryContext cont // to avoid too many opened files TsFileSequenceReader tsFileReader = FileReaderManager.getInstance() - .get(fileNode.getFilePath(), false); + .get(fileNode.getFilePath(), true); MetadataQuerierByFileImpl metadataQuerier = new MetadataQuerierByFileImpl(tsFileReader); List metaDataList = metadataQuerier.getChunkMetaDataList(seriesPath); diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReaderByTimestamp.java index f1e75765d7f75..8faf8e62f7b6e 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReaderByTimestamp.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReaderByTimestamp.java @@ -118,7 +118,7 @@ private void initSingleTsFileReader(IntervalFileNode fileNode, QueryContext cont // to avoid too many opened files TsFileSequenceReader tsFileReader = FileReaderManager.getInstance() - .get(fileNode.getFilePath(), false); + .get(fileNode.getFilePath(), true); MetadataQuerierByFileImpl metadataQuerier = new MetadataQuerierByFileImpl(tsFileReader); List metaDataList = metadataQuerier.getChunkMetaDataList(seriesPath); diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFileReader.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFileReader.java index 0dda8c5a6d413..3c02930f034ea 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFileReader.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFileReader.java @@ -50,7 +50,7 @@ public UnSealedTsFileReader(UnsealedTsFile unsealedTsFile, Filter filter) throws TsFileSequenceReader unClosedTsFileReader = FileReaderManager.getInstance() .get(unsealedTsFile.getFilePath(), - true); + false); ChunkLoader chunkLoader = new ChunkLoaderImpl(unClosedTsFileReader); if (filter == null) { diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFilesReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFilesReaderByTimestamp.java index 251dc799dcd57..bf73bee0085fb 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFilesReaderByTimestamp.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFilesReaderByTimestamp.java @@ -39,7 +39,7 @@ public UnSealedTsFilesReaderByTimestamp(UnsealedTsFile unsealedTsFile) throws IO TsFileSequenceReader unClosedTsFileReader = FileReaderManager.getInstance() .get(unsealedTsFile.getFilePath(), - true); + false); ChunkLoader chunkLoader = new ChunkLoaderImpl(unClosedTsFileReader); unSealedReader = new SeriesReaderByTimestamp(chunkLoader, unsealedTsFile.getChunkMetaDataList()); diff --git a/iotdb/src/main/java/org/apache/iotdb/db/service/JDBCService.java b/iotdb/src/main/java/org/apache/iotdb/db/service/JDBCService.java index f75b62fd7d52e..cb2831298493d 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/service/JDBCService.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/service/JDBCService.java @@ -70,14 +70,14 @@ public static final JDBCService getInstance() { public String getJDBCServiceStatus() { // TODO debug log, will be deleted in production env if(startLatch == null) { - LOGGER.debug("Start latch is null when getting status"); + LOGGER.info("Start latch is null when getting status"); } else { - LOGGER.debug("Start latch is {} when getting status", startLatch.getCount()); + LOGGER.info("Start latch is {} when getting status", startLatch.getCount()); } if(stopLatch == null) { - LOGGER.debug("Stop latch is null when getting status"); + LOGGER.info("Stop latch is null when getting status"); } else { - LOGGER.debug("Stop latch is {} when getting status", stopLatch.getCount()); + LOGGER.info("Stop latch is {} when getting status", stopLatch.getCount()); } // debug log, will be deleted in production env @@ -218,9 +218,9 @@ public void run() { close(); // TODO debug log, will be deleted in production env if(threadStopLatch == null) { - LOGGER.debug("Stop Count Down latch is null"); + LOGGER.info("Stop Count Down latch is null"); } else { - LOGGER.debug("Stop Count Down latch is {}", threadStopLatch.getCount()); + LOGGER.info("Stop Count Down latch is {}", threadStopLatch.getCount()); } // debug log, will be deleted in production env diff --git a/iotdb/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java b/iotdb/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java index d3e0a71c4f504..e1f4ff5f423aa 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/service/TSServiceImpl.java @@ -567,7 +567,7 @@ public TSExecuteStatementResp executeQueryStatement(TSExecuteStatementReq req) t recordANewQuery(statement, plan); return resp; } catch (Exception e) { - LOGGER.error("{}: Internal server error: {}", IoTDBConstant.GLOBAL_DB_NAME, e.getMessage()); + LOGGER.error("{}: Internal server error: ", IoTDBConstant.GLOBAL_DB_NAME, e); return getTSExecuteStatementResp(TS_StatusCode.ERROR_STATUS, e.getMessage()); } } @@ -605,7 +605,7 @@ public TSFetchResultsResp fetchResults(TSFetchResultsReq req) throws TException resp.setQueryDataSet(result); return resp; } catch (Exception e) { - LOGGER.error("{}: Internal server error: {}", IoTDBConstant.GLOBAL_DB_NAME, e.getMessage()); + LOGGER.error("{}: Internal server error: ", IoTDBConstant.GLOBAL_DB_NAME, e); return getTSFetchResultsResp(TS_StatusCode.ERROR_STATUS, e.getMessage()); } } @@ -623,7 +623,7 @@ public TSExecuteStatementResp executeUpdateStatement(TSExecuteStatementReq req) LOGGER.error("meet error while executing update statement.", e); return getTSExecuteStatementResp(TS_StatusCode.ERROR_STATUS, e.getMessage()); } catch (Exception e) { - LOGGER.error("{}: server Internal Error: {}", IoTDBConstant.GLOBAL_DB_NAME, e.getMessage()); + LOGGER.error("{}: server Internal Error: ", IoTDBConstant.GLOBAL_DB_NAME, e); return getTSExecuteStatementResp(TS_StatusCode.ERROR_STATUS, e.getMessage()); } } diff --git a/iotdb/src/main/java/org/apache/iotdb/db/utils/OpenFileNumUtil.java b/iotdb/src/main/java/org/apache/iotdb/db/utils/OpenFileNumUtil.java index cf0c24b555f6e..6c071325b6494 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/utils/OpenFileNumUtil.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/utils/OpenFileNumUtil.java @@ -26,6 +26,7 @@ import java.util.EnumMap; import java.util.List; import org.apache.iotdb.db.conf.IoTDBConfig; +import org.apache.iotdb.db.conf.IoTDBConstant; import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.conf.directories.Directories; import org.slf4j.Logger; @@ -38,7 +39,7 @@ public class OpenFileNumUtil { private static final int PID_ERROR_CODE = -1; private static final int UNSUPPORTED_OS_ERROR_CODE = -2; private static final int UNKNOWN_STATISTICS_ERROR_CODE = -3; - private static final String IOTDB_PROCESS_KEY_WORD = "iotdb.IoTDB"; + private static final String IOTDB_PROCESS_KEY_WORD = IoTDBConstant.GLOBAL_DB_NAME; private static final String LINUX_OS_NAME = "linux"; private static final String MAC_OS_NAME = "mac"; private static final String SEARCH_PID_LINUX = "ps -aux | grep -i %s | grep -v grep"; @@ -56,7 +57,7 @@ public class OpenFileNumUtil { * constructor, process key word is defined by IOTDB_PROCESS_KEY_WORD. */ private OpenFileNumUtil() { - pid = getPid(); + pid = getIotdbPid(); } /** @@ -89,11 +90,11 @@ private static boolean isNumeric(String str) { } /** - * get process ID by executing command. + * get IoTDB server process ID by executing command. * - * @return pid + * @return pid of IoTDB server process */ - private static int getPid() { + private static int getIotdbPid() { int iotdbPid = -1; Process pro1; Runtime r = Runtime.getRuntime(); @@ -104,11 +105,10 @@ private static int getPid() { if (osName.startsWith(LINUX_OS_NAME) || osName.startsWith(MAC_OS_NAME)) { try { String command; - String processName = IOTDB_PROCESS_KEY_WORD; if (osName.startsWith(LINUX_OS_NAME)) { - command = String.format(SEARCH_PID_LINUX, processName); + command = String.format(SEARCH_PID_LINUX, IOTDB_PROCESS_KEY_WORD); } else { - command = String.format(SEARCH_PID_MAC, processName); + command = String.format(SEARCH_PID_MAC, IOTDB_PROCESS_KEY_WORD); } COMMAND_TEMPLATE[2] = command; pro1 = r.exec(COMMAND_TEMPLATE); @@ -125,10 +125,10 @@ private static int getPid() { in1.close(); pro1.destroy(); } catch (IOException e) { - LOGGER.error("Cannot get pid of IoTDB process because {}", e.getMessage()); + LOGGER.error("Cannot get PID of IoTDB process because ", e); } } else { - LOGGER.warn("Unsupported OS {} for OpenFileNumUtil getting Pid.", os); + LOGGER.warn("Unsupported OS {} for OpenFileNumUtil to get the PID of IoTDB.", os); } return iotdbPid; } @@ -204,7 +204,7 @@ private static EnumMap getOpenFile(int pid) { in.close(); pro.destroy(); } catch (Exception e) { - LOGGER.error("Cannot get open file number of IoTDB process because {}", e.getMessage()); + LOGGER.error("Cannot get open file number of IoTDB process because ", e); } return resultMap; } diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionQueryTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionQueryTest.java index ded02f8a3d73b..e7890ab087a25 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionQueryTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionQueryTest.java @@ -271,8 +271,6 @@ public void testSuccessiveDeletion() FileNodeManager.getInstance().delete(processorName, measurements[5], 30); FileNodeManager.getInstance().delete(processorName, measurements[5], 50); - FileNodeManager.getInstance().forceFlush(UsageLevel.DANGEROUS); - Thread.sleep(3000); FileNodeManager.getInstance().closeAll(); List pathList = new ArrayList<>(); diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBDeletionIT.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBDeletionIT.java index f23bae80b62c9..e193f0ad5eada 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBDeletionIT.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBDeletionIT.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.integration; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import java.sql.Connection; import java.sql.DriverManager; @@ -147,6 +148,31 @@ public void testMerge() throws SQLException, InterruptedException { cleanData(); } + @Test + public void testDelAfterFlush() throws SQLException { + Connection connection = DriverManager + .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", + "root"); + Statement statement = connection.createStatement(); + + statement.execute("SET STORAGE GROUP TO root.ln.wf01.wt01"); + statement.execute("CREATE TIMESERIES root.ln.wf01.wt01.status WITH DATATYPE=BOOLEAN," + + " ENCODING=PLAIN"); + statement.execute("INSERT INTO root.ln.wf01.wt01(timestamp,status) " + + "values(1509465600000,true)"); + statement.execute("INSERT INTO root.ln.wf01.wt01(timestamp,status) VALUES(NOW(), false)"); + + statement.execute("delete from root.ln.wf01.wt01.status where time < NOW()"); + statement.execute("flush"); + statement.execute("delete from root.ln.wf01.wt01.status where time < NOW()"); + ResultSet resultSet = statement.executeQuery("select status from root.ln.wf01.wt01"); + assertFalse(resultSet.next()); + + statement.close(); + connection.close(); + } + + private static void prepareSeries() throws SQLException { Connection connection = null; try { diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBSeriesReaderIT.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBSeriesReaderIT.java index eb8da77f11013..387310b52d970 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBSeriesReaderIT.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBSeriesReaderIT.java @@ -19,11 +19,13 @@ package org.apache.iotdb.db.integration; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.fail; import java.io.IOException; import java.sql.Connection; import java.sql.DriverManager; +import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Statement; import org.apache.iotdb.db.exception.FileNodeManagerException; @@ -343,4 +345,13 @@ public void crossSeriesReadUpdateTest() throws IOException, FileNodeManagerExcep QueryTokenManager.getInstance().endQueryForCurrentRequestThread(); } + + @Test + public void queryEmptySeriesTest() throws SQLException { + Statement statement = connection.createStatement(); + statement.execute("CREATE TIMESERIES root.vehicle.d_empty.s1 WITH DATATYPE=INT64, ENCODING=RLE"); + ResultSet resultSet = statement.executeQuery("select * from root.vehicle.d_empty"); + assertFalse (resultSet.next()); + resultSet.close(); + } } diff --git a/iotdb/src/test/java/org/apache/iotdb/db/query/control/FileReaderManagerTest.java b/iotdb/src/test/java/org/apache/iotdb/db/query/control/FileReaderManagerTest.java index 5675fc66092ea..cff2664f00fb8 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/query/control/FileReaderManagerTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/query/control/FileReaderManagerTest.java @@ -65,9 +65,10 @@ public void test() throws IOException, InterruptedException { OpenedFilePathsManager.getInstance().setJobIdForCurrentRequestThread(1L); for (int i = 1; i <= 6; i++) { - OpenedFilePathsManager.getInstance().addFilePathToMap(1L, filePath + i); - manager.get(filePath + i, true); - Assert.assertTrue(manager.contains(filePath + i)); + OpenedFilePathsManager.getInstance().addFilePathToMap(1L, filePath + i, + false); + manager.get(filePath + i, false); + Assert.assertTrue(manager.contains(filePath + i, false)); } } catch (IOException e) { @@ -82,9 +83,10 @@ public void test() throws IOException, InterruptedException { OpenedFilePathsManager.getInstance().setJobIdForCurrentRequestThread(2L); for (int i = 4; i <= MAX_FILE_SIZE; i++) { - OpenedFilePathsManager.getInstance().addFilePathToMap(2L, filePath + i); - manager.get(filePath + i, true); - Assert.assertTrue(manager.contains(filePath + i)); + OpenedFilePathsManager.getInstance().addFilePathToMap(2L, filePath + i, + false); + manager.get(filePath + i, false); + Assert.assertTrue(manager.contains(filePath + i, false)); } } catch (IOException e) { @@ -98,11 +100,11 @@ public void test() throws IOException, InterruptedException { t2.join(); for (int i = 1; i <= MAX_FILE_SIZE; i++) { - Assert.assertTrue(manager.contains(filePath + i)); + Assert.assertTrue(manager.contains(filePath + i, false)); } for (int i = 1; i <= MAX_FILE_SIZE; i++) { - manager.decreaseFileReaderReference(filePath + i); + manager.decreaseFileReaderReference(filePath + i, true); } // the code below is not valid because the cacheFileReaderClearPeriod config in this class is not valid diff --git a/iotdb/src/test/java/org/apache/iotdb/db/utils/OpenFileNumUtilTest.java b/iotdb/src/test/java/org/apache/iotdb/db/utils/OpenFileNumUtilTest.java index b2fbec388348a..7a616f15707f2 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/utils/OpenFileNumUtilTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/utils/OpenFileNumUtilTest.java @@ -152,7 +152,7 @@ public void testDataOpenFileNumWhenCreateFileWriter() { LOGGER.debug("Create a file {} successfully", file); fileWriterList.add(new FileWriter(file)); } else { - LOGGER.error("create test file {} failed.", file.getPath()); + LOGGER.error("create test file {} failed when creating file writer.", file.getPath()); } } catch (IOException e) { LOGGER.error(e.getMessage()); diff --git a/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBConnection.java b/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBConnection.java index fbb1aa21d5d6f..871a6653df312 100644 --- a/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBConnection.java +++ b/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBConnection.java @@ -131,7 +131,7 @@ public void close() throws SQLException { try { client.closeSession(req); } catch (TException e) { - throw new SQLException("Error occurs when closing session at server", e); + throw new SQLException("Error occurs when closing session at server. Maybe server is down.", e); } finally { isClosed = true; if (transport != null) { diff --git a/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBConnectionParams.java b/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBConnectionParams.java index 07878121adc8a..3decc2fc9866c 100644 --- a/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBConnectionParams.java +++ b/jdbc/src/main/java/org/apache/iotdb/jdbc/IoTDBConnectionParams.java @@ -20,7 +20,7 @@ public class IoTDBConnectionParams { - private String host = Config.IOTDB_URL_PREFIX; + private String host = Config.IOTDB_DEFAULT_HOST; private int port = Config.IOTDB_DEFAULT_PORT; private String jdbcUriString; private String seriesName = Config.DEFAULT_SERIES_NAME; diff --git a/pom.xml b/pom.xml index 5aa950317cacb..6f9215815e5bb 100644 --- a/pom.xml +++ b/pom.xml @@ -134,6 +134,7 @@ grafana service-rpc iotdb-cli + example @@ -200,6 +201,9 @@ .checkstyle local-snapshots-dir/** + + iotdb/data/** + iotdb/logs/** diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/MetadataQuerier.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/MetadataQuerier.java index e288e58863c85..673789623931b 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/MetadataQuerier.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/MetadataQuerier.java @@ -20,8 +20,10 @@ import java.io.IOException; import java.util.List; +import org.apache.iotdb.tsfile.exception.write.NoMeasurementException; import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData; import org.apache.iotdb.tsfile.file.metadata.TsFileMetaData; +import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; import org.apache.iotdb.tsfile.read.common.Path; public interface MetadataQuerier { @@ -38,4 +40,11 @@ public interface MetadataQuerier { */ void loadChunkMetaDatas(List paths) throws IOException; + /** + * + * @param measurement + * @return the corresponding data type. + * @throws NoMeasurementException if the measurement not exists. + */ + TSDataType getDataType(String measurement) throws NoMeasurementException; } diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/MetadataQuerierByFileImpl.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/MetadataQuerierByFileImpl.java index b1e16c9c207ec..acc8fc4ccc708 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/MetadataQuerierByFileImpl.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/MetadataQuerierByFileImpl.java @@ -28,13 +28,16 @@ import java.util.TreeMap; import org.apache.iotdb.tsfile.common.cache.LRUCache; import org.apache.iotdb.tsfile.common.constant.QueryConstant; +import org.apache.iotdb.tsfile.exception.write.NoMeasurementException; import org.apache.iotdb.tsfile.file.metadata.ChunkGroupMetaData; import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData; import org.apache.iotdb.tsfile.file.metadata.TsDeviceMetadata; import org.apache.iotdb.tsfile.file.metadata.TsDeviceMetadataIndex; import org.apache.iotdb.tsfile.file.metadata.TsFileMetaData; +import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; import org.apache.iotdb.tsfile.read.TsFileSequenceReader; import org.apache.iotdb.tsfile.read.common.Path; +import org.apache.iotdb.tsfile.write.schema.MeasurementSchema; public class MetadataQuerierByFileImpl implements MetadataQuerier { @@ -178,6 +181,15 @@ public void loadChunkMetaDatas(List paths) throws IOException { } + @Override + public TSDataType getDataType(String measurement) throws NoMeasurementException { + MeasurementSchema measurementSchema = fileMetaData.getMeasurementSchema().get(measurement); + if(measurementSchema != null) { + return measurementSchema.getType(); + } + throw new NoMeasurementException(String.format("%s not found.", measurement)); + } + private List loadChunkMetadata(Path path) throws IOException { if (!fileMetaData.containsDevice(path.getDevice())) { diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/executor/TsFileExecutor.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/executor/TsFileExecutor.java index 69d364f5159b9..39883afb16615 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/executor/TsFileExecutor.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/query/executor/TsFileExecutor.java @@ -22,6 +22,7 @@ import java.util.ArrayList; import java.util.List; import org.apache.iotdb.tsfile.exception.filter.QueryFilterOptimizationException; +import org.apache.iotdb.tsfile.exception.write.NoMeasurementException; import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData; import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; import org.apache.iotdb.tsfile.read.common.Path; @@ -33,6 +34,7 @@ import org.apache.iotdb.tsfile.read.expression.util.ExpressionOptimizer; import org.apache.iotdb.tsfile.read.query.dataset.DataSetWithoutTimeGenerator; import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet; +import org.apache.iotdb.tsfile.read.reader.series.EmptyFileSeriesReader; import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReader; import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReaderWithFilter; import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReaderWithoutFilter; @@ -66,11 +68,15 @@ public QueryDataSet execute(QueryExpression queryExpression) throws IOException return new ExecutorWithTimeGenerator(metadataQuerier, chunkLoader) .execute(queryExpression); } - } catch (QueryFilterOptimizationException e) { + } catch (QueryFilterOptimizationException | NoMeasurementException e) { throw new IOException(e); } } else { - return execute(queryExpression.getSelectedSeries()); + try { + return execute(queryExpression.getSelectedSeries()); + } catch (NoMeasurementException e) { + throw new IOException(e); + } } } @@ -80,18 +86,9 @@ public QueryDataSet execute(QueryExpression queryExpression) throws IOException * @param selectedPathList all selected paths * @return DataSet without TimeGenerator */ - private QueryDataSet execute(List selectedPathList) throws IOException { - List readersOfSelectedSeries = new ArrayList<>(); - List dataTypes = new ArrayList<>(); - - for (Path path : selectedPathList) { - List chunkMetaDataList = metadataQuerier.getChunkMetaDataList(path); - FileSeriesReader seriesReader = new FileSeriesReaderWithoutFilter(chunkLoader, - chunkMetaDataList); - readersOfSelectedSeries.add(seriesReader); - dataTypes.add(chunkMetaDataList.get(0).getTsDataType()); - } - return new DataSetWithoutTimeGenerator(selectedPathList, dataTypes, readersOfSelectedSeries); + private QueryDataSet execute(List selectedPathList) + throws IOException, NoMeasurementException { + return executeMayAttachTimeFiler(selectedPathList, null); } /** @@ -102,18 +99,40 @@ private QueryDataSet execute(List selectedPathList) throws IOException { * @return DataSet without TimeGenerator */ private QueryDataSet execute(List selectedPathList, GlobalTimeExpression timeFilter) - throws IOException { + throws IOException, NoMeasurementException { + return executeMayAttachTimeFiler(selectedPathList, timeFilter); + } + + /** + * + * @param selectedPathList completed path + * @param timeFilter a GlobalTimeExpression or null + * @return DataSetWithoutTimeGenerator + * @throws IOException + * @throws NoMeasurementException + */ + private QueryDataSet executeMayAttachTimeFiler(List selectedPathList, GlobalTimeExpression timeFilter) + throws IOException, NoMeasurementException { List readersOfSelectedSeries = new ArrayList<>(); List dataTypes = new ArrayList<>(); for (Path path : selectedPathList) { List chunkMetaDataList = metadataQuerier.getChunkMetaDataList(path); - FileSeriesReader seriesReader = new FileSeriesReaderWithFilter(chunkLoader, chunkMetaDataList, - timeFilter.getFilter()); + FileSeriesReader seriesReader; + if (chunkMetaDataList.isEmpty()) { + seriesReader = new EmptyFileSeriesReader(); + dataTypes.add(metadataQuerier.getDataType(path.getMeasurement())); + } else { + if (timeFilter == null) { + seriesReader = new FileSeriesReaderWithoutFilter(chunkLoader, chunkMetaDataList); + } else { + seriesReader = new FileSeriesReaderWithFilter(chunkLoader, chunkMetaDataList, + timeFilter.getFilter()); + } + dataTypes.add(chunkMetaDataList.get(0).getTsDataType()); + } readersOfSelectedSeries.add(seriesReader); - dataTypes.add(chunkMetaDataList.get(0).getTsDataType()); } - return new DataSetWithoutTimeGenerator(selectedPathList, dataTypes, readersOfSelectedSeries); } diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/EmptyFileSeriesReader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/EmptyFileSeriesReader.java new file mode 100644 index 0000000000000..26a509f70f93c --- /dev/null +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/EmptyFileSeriesReader.java @@ -0,0 +1,64 @@ +/** + * 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.tsfile.read.reader.series; + +import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData; +import org.apache.iotdb.tsfile.read.common.BatchData; + +/** + * this is for those series which has no data points + */ +public class EmptyFileSeriesReader extends FileSeriesReader { + BatchData data = new BatchData(); + + public EmptyFileSeriesReader() { + super(null, null); + } + + @Override + protected void initChunkReader(ChunkMetaData chunkMetaData) { + //do nothing + } + + @Override + protected boolean chunkSatisfied(ChunkMetaData chunkMetaData) { + return false; + } + + @Override + public boolean hasNextBatch() { + return false; + } + + @Override + public BatchData nextBatch() { + return data; + } + + @Override + public BatchData currentBatch() { + return data; + } + + @Override + public void close() { + data = null; + } +} diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/write/TsFileReadWriteTest.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/write/TsFileReadWriteTest.java index 53d2d02a434b7..3cb2a84f3e269 100644 --- a/tsfile/src/test/java/org/apache/iotdb/tsfile/write/TsFileReadWriteTest.java +++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/write/TsFileReadWriteTest.java @@ -19,6 +19,8 @@ package org.apache.iotdb.tsfile.write; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; import java.io.File; import java.io.IOException; @@ -216,4 +218,38 @@ public void doubleTest() throws IOException, WriteProcessException { reader.close(); } + @Test + public void readEmptyMeasurementTest() throws IOException, WriteProcessException { + String path = "test.tsfile"; + File f = new File(path); + if (f.exists()) { + assertTrue(f.delete()); + } + TsFileWriter tsFileWriter = new TsFileWriter(f); + + // add measurements into file schema + tsFileWriter + .addMeasurement(new MeasurementSchema("sensor_1", TSDataType.FLOAT, TSEncoding.RLE)); + tsFileWriter + .addMeasurement(new MeasurementSchema("sensor_2", TSDataType.INT32, TSEncoding.TS_2DIFF)); + // construct TSRecord + TSRecord tsRecord = new TSRecord(1, "device_1"); + DataPoint dPoint1 = new FloatDataPoint("sensor_1", 1.2f); + tsRecord.addTuple(dPoint1); + // write a TSRecord to TsFile + tsFileWriter.write(tsRecord); + // close TsFile + tsFileWriter.close(); + + // read example : no filter + TsFileSequenceReader reader = new TsFileSequenceReader(path); + ReadOnlyTsFile readTsFile = new ReadOnlyTsFile(reader); + ArrayList paths = new ArrayList<>(); + paths.add(new Path("device_1.sensor_2")); + QueryExpression queryExpression = QueryExpression.create(paths, null); + QueryDataSet queryDataSet = readTsFile.query(queryExpression); + assertFalse(queryDataSet.hasNext()); + reader.close(); + assertTrue(f.delete()); + } } From a990d0bbdab466f15b6c7021c674d22729fa3b94 Mon Sep 17 00:00:00 2001 From: suyue <2335813255@qq.com> Date: Mon, 4 Mar 2019 23:06:36 +0800 Subject: [PATCH 5/8] fix based on PR review opinion --- .../db/query/factory/SeriesReaderFactory.java | 22 ++++++++++------- .../merge/PriorityMergeReaderByTimestamp.java | 20 ++++------------ .../SealedTsFilesReaderByTimestamp.java | 6 ++++- .../EngineChunkReaderByTimestamp.java | 24 +++++++++++-------- .../PriorityMergeReaderByTimestampTest.java | 23 +++++++++++------- tsfile/pom.xml | 1 - .../iotdb/tsfile/read/common/BatchData.java | 1 - .../series/SeriesReaderByTimestamp.java | 20 ++++++++-------- 8 files changed, 60 insertions(+), 57 deletions(-) diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/factory/SeriesReaderFactory.java b/iotdb/src/main/java/org/apache/iotdb/db/query/factory/SeriesReaderFactory.java index d4a80756ee8fe..f07c97cfbd568 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/factory/SeriesReaderFactory.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/factory/SeriesReaderFactory.java @@ -1,15 +1,19 @@ /** - * 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 + * 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 + * 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 + * 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. */ diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestamp.java index e18cc1d7cadd4..f6ed7c7d28d68 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestamp.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestamp.java @@ -36,24 +36,12 @@ public class PriorityMergeReaderByTimestamp implements EngineReaderByTimeStamp { private List priorityList = new ArrayList<>(); /** - * The bigger the priority value is, the higher the priority of this reader is. + * This function doesn't sort reader by priority. So you have to call this function in order of + * reader priority from small to large. */ public void addReaderWithPriority(EngineReaderByTimeStamp reader, int priority) { - int size = readerList.size(); - boolean hasAdded = false; - - //sort by priority, elements with low priority are at front of list. - for (int i = 0; i < size; i++) { - if (priority < priorityList.get(i)) { - readerList.add(i, reader); - priorityList.add(i, priority); - hasAdded = true; - } - } - if (!hasAdded) { - readerList.add(reader); - priorityList.add(priority); - } + readerList.add(reader); + priorityList.add(priority); } @Override diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReaderByTimestamp.java index 8faf8e62f7b6e..9c4376d12ff41 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReaderByTimestamp.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SealedTsFilesReaderByTimestamp.java @@ -95,7 +95,11 @@ public void close() throws IOException { // file streams are managed uniformly. } - // construct reader from the file that might overlap this timestamp + /** + * construct reader with the file that might overlap this timestamp. + * @param timestamp + * @throws IOException + */ private void constructReader(long timestamp) throws IOException { while (nextIntervalFileIndex < sealedTsFiles.size()) { if (singleTsFileSatisfied(sealedTsFiles.get(nextIntervalFileIndex), timestamp)) { diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/unsequence/EngineChunkReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/unsequence/EngineChunkReaderByTimestamp.java index 286da26d7dcaa..dbdf31fcfad8c 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/unsequence/EngineChunkReaderByTimestamp.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/unsequence/EngineChunkReaderByTimestamp.java @@ -1,15 +1,19 @@ /** - * 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 + * 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 + * 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 + * 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. */ @@ -66,7 +70,7 @@ public Object getValueInTimestamp(long timestamp) throws IOException { @Override public boolean hasNext() throws IOException { - if (data != null & data.hasNext()) { + if (data != null && data.hasNext()) { return true; } if (chunkReader != null && chunkReader.hasNextBatch()) { diff --git a/iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestampTest.java b/iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestampTest.java index f74ae85a733f1..9593f315f3e6d 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestampTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestampTest.java @@ -1,17 +1,22 @@ /** - * Licensed to the Apache Software Foundation (ASF) under one or more contributor license - * agreements. See the NOTICE file distributed with this work for additional information regarding - * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with the License. You may obtain - * a copy of the License at + * 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 + * 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 + * 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.query.reader.merge; import java.io.IOException; diff --git a/tsfile/pom.xml b/tsfile/pom.xml index 53ef892ded422..5749bc5e6d9d4 100644 --- a/tsfile/pom.xml +++ b/tsfile/pom.xml @@ -113,7 +113,6 @@ - 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 ed82aed09920b..c2160700db823 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 @@ -20,7 +20,6 @@ package org.apache.iotdb.tsfile.read.common; import java.util.ArrayList; -import java.util.Arrays; 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; diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/SeriesReaderByTimestamp.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/SeriesReaderByTimestamp.java index 1674dd8a3da52..735206f9f5a9b 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/SeriesReaderByTimestamp.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/SeriesReaderByTimestamp.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.iotdb.tsfile.read.reader.series; import java.io.IOException; @@ -27,7 +28,6 @@ import org.apache.iotdb.tsfile.read.controller.ChunkLoader; import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReader; import org.apache.iotdb.tsfile.read.reader.chunk.ChunkReaderByTimestamp; -import org.apache.iotdb.tsfile.utils.Pair; /** *

@@ -35,7 +35,7 @@ * a series with given timestamps. *

*/ -public class SeriesReaderByTimestamp{ +public class SeriesReaderByTimestamp { protected ChunkLoader chunkLoader; protected List chunkMetaDataList; @@ -107,27 +107,27 @@ public Object getValueInTimestamp(long timestamp) throws IOException { /** * Judge if the series reader has next time-value pair. + * * @return true if has next, false if not. - * @throws IOException */ public boolean hasNext() throws IOException { - if(chunkReader != null){ - if(data != null && data.hasNext()){ + if (chunkReader != null) { + if (data != null && data.hasNext()) { return true; } - while (chunkReader.hasNextBatch()){ + while (chunkReader.hasNextBatch()) { data = chunkReader.nextBatch(); - if(data != null && data.hasNext()){ + if (data != null && data.hasNext()) { return true; } } } - while(constructNextSatisfiedChunkReader()){ - while (chunkReader.hasNextBatch()){ + while (constructNextSatisfiedChunkReader()) { + while (chunkReader.hasNextBatch()) { data = chunkReader.nextBatch(); - if(data != null && data.hasNext()){ + if (data != null && data.hasNext()) { return true; } } From ff2acca5590a3f93965e074c533a59dc5ee5f1c4 Mon Sep 17 00:00:00 2001 From: suyue <2335813255@qq.com> Date: Mon, 4 Mar 2019 23:29:36 +0800 Subject: [PATCH 6/8] add test for SeriesReaderByTimestamp --- .../iotdb/tsfile/read/reader/ReaderTest.java | 25 +++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/read/reader/ReaderTest.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/read/reader/ReaderTest.java index 6bc2e8c226d24..ff25688ab85b7 100644 --- a/tsfile/src/test/java/org/apache/iotdb/tsfile/read/reader/ReaderTest.java +++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/read/reader/ReaderTest.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.util.List; +import java.util.Random; import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor; import org.apache.iotdb.tsfile.exception.write.WriteProcessException; import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData; @@ -36,6 +37,7 @@ import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReader; import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReaderWithFilter; import org.apache.iotdb.tsfile.read.reader.series.FileSeriesReaderWithoutFilter; +import org.apache.iotdb.tsfile.read.reader.series.SeriesReaderByTimestamp; import org.apache.iotdb.tsfile.utils.TsFileGeneratorForTest; import org.junit.After; import org.junit.Assert; @@ -126,4 +128,27 @@ public void readWithFilterTest() throws IOException { } } } + + @Test + public void readByTimestampTest() throws IOException { + ChunkLoaderImpl seriesChunkLoader = new ChunkLoaderImpl(fileReader); + List chunkMetaDataList = metadataQuerierByFile + .getChunkMetaDataList(new Path("d1.s1")); + SeriesReaderByTimestamp seriesReader = new SeriesReaderByTimestamp(seriesChunkLoader, + chunkMetaDataList); + + long startTime = TsFileGeneratorForTest.START_TIMESTAMP; + long endTime = TsFileGeneratorForTest.START_TIMESTAMP + rowCount; + Random random = new Random(); + for (long time = startTime - 500; time < endTime + 500; ) { + time += random.nextInt(10) + 1; + Object value = seriesReader.getValueInTimestamp(time); + if (time < startTime || time >= endTime) { + Assert.assertNull(value); + } else { + int actualData = (int) ((time - startTime) * 10 + 1); + Assert.assertEquals(actualData, value); + } + } + } } From a16bbe23f98caa1787b7f15d6be11a8d29fb1666 Mon Sep 17 00:00:00 2001 From: suyue <2335813255@qq.com> Date: Tue, 5 Mar 2019 19:01:10 +0800 Subject: [PATCH 7/8] add test --- .../SequenceDataReaderByTimestamp.java | 32 +++++-- .../UnSealedTsFilesReaderByTimestamp.java | 22 +++-- .../reader/FakedSeriesReaderByTimestamp.java | 83 ++++++++++++++++ .../PriorityMergeReaderByTimestampTest.java | 4 + .../SequenceDataReaderByTimestampTest.java | 94 +++++++++++++++++++ 5 files changed, 217 insertions(+), 18 deletions(-) create mode 100644 iotdb/src/test/java/org/apache/iotdb/db/query/reader/FakedSeriesReaderByTimestamp.java create mode 100644 iotdb/src/test/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReaderByTimestampTest.java diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReaderByTimestamp.java index beaecb6e70b51..cf634265aa230 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReaderByTimestamp.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReaderByTimestamp.java @@ -1,15 +1,19 @@ /** - * 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 + * 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 + * 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 + * 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. */ @@ -57,6 +61,16 @@ public SequenceDataReaderByTimestamp(GlobalSortedSeriesDataSource sources, Query } + /** + * This method is used only in unit test. + * @param seriesReaders + */ + public SequenceDataReaderByTimestamp(List seriesReaders){ + this.seriesReaders = seriesReaders; + nextSeriesReaderIndex = 0; + } + + @Override public Object getValueInTimestamp(long timestamp) throws IOException { Object value = null; diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFilesReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFilesReaderByTimestamp.java index bf73bee0085fb..4cd0dfb012257 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFilesReaderByTimestamp.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/sequence/UnSealedTsFilesReaderByTimestamp.java @@ -1,15 +1,19 @@ /** - * 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 + * 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 + * 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 + * 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. */ diff --git a/iotdb/src/test/java/org/apache/iotdb/db/query/reader/FakedSeriesReaderByTimestamp.java b/iotdb/src/test/java/org/apache/iotdb/db/query/reader/FakedSeriesReaderByTimestamp.java new file mode 100644 index 0000000000000..ae7f496657942 --- /dev/null +++ b/iotdb/src/test/java/org/apache/iotdb/db/query/reader/FakedSeriesReaderByTimestamp.java @@ -0,0 +1,83 @@ +/** + * 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.query.reader; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp; +import org.apache.iotdb.db.utils.TimeValuePair; +import org.apache.iotdb.db.utils.TsPrimitiveType; +import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; + +public class FakedSeriesReaderByTimestamp implements EngineReaderByTimeStamp { + private Iterator iterator; + private boolean hasCachedTimeValuePair = false; + private TimeValuePair cachedTimeValuePair; + + public FakedSeriesReaderByTimestamp(long startTime, int size, int interval, int modValue) { + long time = startTime; + List list = new ArrayList<>(); + for (int i = 0; i < size; i++) { + list.add( + new TimeValuePair(time, TsPrimitiveType.getByType(TSDataType.INT64, time % modValue))); + time += interval; + } + iterator = list.iterator(); + } + + @Override + public Object getValueInTimestamp(long timestamp) throws IOException { + if(hasCachedTimeValuePair){ + if(timestamp == cachedTimeValuePair.getTimestamp()){ + hasCachedTimeValuePair = false; + return cachedTimeValuePair.getValue().getValue(); + } + else if(timestamp > cachedTimeValuePair.getTimestamp()){ + hasCachedTimeValuePair = false; + } + else { + return null; + } + } + while(iterator.hasNext()){ + cachedTimeValuePair = iterator.next(); + if(timestamp == cachedTimeValuePair.getTimestamp()){ + return cachedTimeValuePair.getValue().getValue(); + } + else if(timestamp < cachedTimeValuePair.getTimestamp()){ + hasCachedTimeValuePair = true; + break; + } + } + return null; + } + + @Override + public boolean hasNext() throws IOException { + return hasCachedTimeValuePair || iterator.hasNext(); + } + + @Override + public void close() throws IOException { + + } +} diff --git a/iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestampTest.java b/iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestampTest.java index 9593f315f3e6d..2a5104cf3184f 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestampTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/query/reader/merge/PriorityMergeReaderByTimestampTest.java @@ -53,6 +53,10 @@ public void test() throws IOException { Random random = new Random(); for (long time = 4; time < 1080 + 200 * 13 + 600; ) { Long value = (Long) priorityReader.getValueInTimestamp(time); + if(time < 1080 + 199 * 13){ + Assert.assertTrue(priorityReader.hasNext()); + } + //System.out.println("time = " + time + " value = " + value); if (time < 100) { // null diff --git a/iotdb/src/test/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReaderByTimestampTest.java b/iotdb/src/test/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReaderByTimestampTest.java new file mode 100644 index 0000000000000..6706653c457a2 --- /dev/null +++ b/iotdb/src/test/java/org/apache/iotdb/db/query/reader/sequence/SequenceDataReaderByTimestampTest.java @@ -0,0 +1,94 @@ +/** + * 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.query.reader.sequence; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Random; +import org.apache.iotdb.db.query.reader.FakedSeriesReaderByTimestamp; +import org.apache.iotdb.db.query.reader.merge.EngineReaderByTimeStamp; +import org.junit.Assert; +import org.junit.Test; + +public class SequenceDataReaderByTimestampTest { + + /** + * Test hasNext() and getValueInTimestamp(long timestamp) method in SequenceDataReaderByTimestamp. + */ + @Test + public void text() throws IOException { + List readers = new ArrayList<>(); + FakedSeriesReaderByTimestamp sealedTsFile = new FakedSeriesReaderByTimestamp(100,1000, 7, 11); + FakedSeriesReaderByTimestamp unsealedTsFile = new FakedSeriesReaderByTimestamp(100 + 1005 *7,100, 17, 3); + FakedSeriesReaderByTimestamp dataInMemory = new FakedSeriesReaderByTimestamp(100 + 1005 *7 + 100 * 17,60, 19, 23); + readers.add(sealedTsFile); + readers.add(unsealedTsFile); + readers.add(dataInMemory); + SequenceDataReaderByTimestamp sequenceReader = new SequenceDataReaderByTimestamp(readers); + + long startTime = 100l; + long endTime = 100 + 1005 *7 + 100 * 17 + 59 * 19; + Random random = new Random(); + for(long time = startTime - 50; time < endTime + 50; time++){ + if(time < endTime){ + Assert.assertTrue(sequenceReader.hasNext()); + } + time += 1 + random.nextInt(10); + Object value = sequenceReader.getValueInTimestamp(time); + if(time < 100){ + Assert.assertNull(value); + } + //sealed tsfile + else if(time < 100 + 1005 *7){ + if((time - 100) % 7 != 0 || time > 100 + 999*7){ + Assert.assertNull(value); + } + else { + Assert.assertEquals(time % 11, value); + } + } + //unsealed tsfile + else if(time < 100 + 1005 *7 + 100 * 17){ + if((time - (100 + 1005 *7)) % 17 != 0 ){ + Assert.assertNull(value); + } + else { + Assert.assertEquals(time % 3, value); + } + } + //memory data + else if(time < 100 + 1005 *7 + 100 * 17 + 60 * 19){ + if((time - (100 + 1005 *7 + 100 * 17)) % 19 != 0 ){ + Assert.assertNull(value); + } + else { + Assert.assertEquals(time % 23, value); + } + } + else { + Assert.assertNull(value); + } + } + + + } + +} From a447205ac0bd6a758c1a3062f9386934dd7f1a61 Mon Sep 17 00:00:00 2001 From: CGF Date: Fri, 8 Mar 2019 14:20:10 +0800 Subject: [PATCH 8/8] fix some comment --- .../dataset/EngineDataSetWithTimeGenerator.java | 1 + .../query/reader/mem/MemChunkReaderByTimestamp.java | 13 +++++++------ .../reader/mem/MemChunkReaderWithoutFilter.java | 4 +++- 3 files changed, 11 insertions(+), 7 deletions(-) diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithTimeGenerator.java b/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithTimeGenerator.java index a658eb0777879..e820d79e788b8 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithTimeGenerator.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/dataset/EngineDataSetWithTimeGenerator.java @@ -35,6 +35,7 @@ public class EngineDataSetWithTimeGenerator extends QueryDataSet { private EngineTimeGenerator timeGenerator; private List readers; + //TODO add null filed /** * constructor of EngineDataSetWithTimeGenerator. diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderByTimestamp.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderByTimestamp.java index 2770b324c870c..82c4dcfb32d64 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderByTimestamp.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderByTimestamp.java @@ -66,15 +66,16 @@ public void close() { } // TODO consider change timeValuePairIterator to List structure, and use binary search instead of - // sequential search - @Override - public Object getValueInTimestamp(long timestamp) throws IOException { + /** + * sequential search + */ + @Override public Object getValueInTimestamp(long timestamp) throws IOException { while (hasNext()) { TimeValuePair timeValuePair = next(); - long time = timeValuePair.getTimestamp(); - if (time == timestamp) { + long currentMemTime = timeValuePair.getTimestamp(); + if (currentMemTime == timestamp) { return timeValuePair.getValue().getValue(); - } else if (time > timestamp) { + } else if (currentMemTime > timestamp) { hasCachedTimeValuePair = true; cachedTimeValuePair = timeValuePair; break; diff --git a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderWithoutFilter.java b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderWithoutFilter.java index bae74968dbb0a..a23260380892e 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderWithoutFilter.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/query/reader/mem/MemChunkReaderWithoutFilter.java @@ -24,7 +24,9 @@ import org.apache.iotdb.db.utils.TimeValuePair; import org.apache.iotdb.tsfile.read.common.BatchData; -// TODO merge MemChunkReaderWithoutFilter and MemChunkReaderWithFilter to one class +/** + * TODO merge MemChunkReaderWithoutFilter and MemChunkReaderWithFilter to one class + */ public class MemChunkReaderWithoutFilter implements IReader { private Iterator timeValuePairIterator;