From d072e9e40a3e4fce19c3ee001a30169f1cedc586 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=B1=9F=E5=A4=A9?= Date: Sun, 20 Jan 2019 10:30:53 +0800 Subject: [PATCH 01/24] move from old repo --- .../db/engine/modification/Deletion.java | 45 ++++++ .../db/engine/modification/Modification.java | 70 +++++++++ .../engine/modification/ModificationFile.java | 101 +++++++++++++ .../io/LocalTextModificationAccessor.java | 135 ++++++++++++++++++ .../modification/io/ModificationReader.java | 40 ++++++ .../modification/io/ModificationWriter.java | 39 +++++ .../db/engine/modification/package-info.java | 22 +++ 7 files changed, 452 insertions(+) create mode 100644 iotdb/src/main/java/org/apache/iotdb/db/engine/modification/Deletion.java create mode 100644 iotdb/src/main/java/org/apache/iotdb/db/engine/modification/Modification.java create mode 100644 iotdb/src/main/java/org/apache/iotdb/db/engine/modification/ModificationFile.java create mode 100644 iotdb/src/main/java/org/apache/iotdb/db/engine/modification/io/LocalTextModificationAccessor.java create mode 100644 iotdb/src/main/java/org/apache/iotdb/db/engine/modification/io/ModificationReader.java create mode 100644 iotdb/src/main/java/org/apache/iotdb/db/engine/modification/io/ModificationWriter.java create mode 100644 iotdb/src/main/java/org/apache/iotdb/db/engine/modification/package-info.java diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/Deletion.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/Deletion.java new file mode 100644 index 000000000000..c25d5e2b7410 --- /dev/null +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/Deletion.java @@ -0,0 +1,45 @@ +/** + * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) + *

+ * Licensed 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.modification; + +/** + * Deletion is a delete operation on a timeseries. + */ +public class Deletion extends Modification { + private long timestamp; + + public Deletion(String path, long versionNum, long timestamp) { + super(Type.DELETION, path, versionNum); + this.timestamp = timestamp; + } + + public long getTimestamp() { + return timestamp; + } + + public void setTimestamp(long timestamp) { + this.timestamp = timestamp; + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof Modification)) + return false; + Deletion del = (Deletion) obj; + return super.equals(obj) && del.timestamp == this.timestamp; + } +} diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/Modification.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/Modification.java new file mode 100644 index 000000000000..3504749ad3b3 --- /dev/null +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/Modification.java @@ -0,0 +1,70 @@ +/** + * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) + *

+ * Licensed 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.modification; + +/** + * Modification represents an UPDATE or DELETE operation on a certain timeseries. + */ +public abstract class Modification { + + protected Type type; + protected String path; + protected long versionNum; + + public Modification(Type type, String path, long versionNum) { + this.type = type; + this.path = path; + this.versionNum = versionNum; + } + + public String getPath() { + return path; + } + + public void setPath(String path) { + this.path = path; + } + + public long getVersionNum() { + return versionNum; + } + + public void setVersionNum(long versionNum) { + this.versionNum = versionNum; + } + + public Type getType() { + return type; + } + + public void setType(Type type) { + this.type = type; + } + + public enum Type { + DELETION + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof Modification)) + return false; + Modification mod = (Modification) obj; + return mod.type.equals(this.type) && mod.path.equals(this.path) + && mod.versionNum == this.versionNum; + } +} diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/ModificationFile.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/ModificationFile.java new file mode 100644 index 000000000000..c04ff1d12ec4 --- /dev/null +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/ModificationFile.java @@ -0,0 +1,101 @@ +/** + * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) + *

+ * Licensed 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.modification; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; + +import org.apache.iotdb.db.engine.modification.io.LocalTextModificationAccessor; +import org.apache.iotdb.db.engine.modification.io.ModificationReader; +import org.apache.iotdb.db.engine.modification.io.ModificationWriter; + +/** + * ModificationFile stores the Modifications of a TsFile or unseq file in another file in the same + * directory. + * Methods in this class are highly synchronized for concurrency safety. + */ +public class ModificationFile { + + private Collection modifications; + private String filePath; + private ModificationWriter writer; + private ModificationReader reader; + + /** + * Construct a ModificationFile using a file as its storage. + * @param filePath the path of the storage file. + * @throws IOException when IOException raised when + */ + public ModificationFile(String filePath) throws IOException { + this.filePath = filePath; + LocalTextModificationAccessor accessor = new LocalTextModificationAccessor(filePath); + this.writer = accessor; + this.reader = accessor; + } + + private void init() throws IOException { + synchronized (this) { + Collection mods = reader.read(); + if (mods == null) { + mods = new ArrayList<>(); + } + modifications = mods; + } + } + + private void checkInit() throws IOException { + if (modifications == null) { + init(); + } + } + + /** + * Release the resources such as streams and caches. + */ + public void close() throws IOException { + synchronized (this) { + writer.close(); + modifications = null; + } + } + + /** + * Write a modification in this file. The modification will first be written to the persistent + * store then the memory cache. + * @param mod the modification to be written. + * @throws IOException if IOException is thrown when writing the modification to the store. + */ + public void write(Modification mod) throws IOException { + synchronized (this) { + checkInit(); + writer.write(mod); + modifications.add(mod); + } + } + + /** + * Get all modifications stored in this file. + * @return + */ + public Collection getModifications() throws IOException { + synchronized (this) { + checkInit(); + return new ArrayList<>(modifications); + } + } +} 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 new file mode 100644 index 000000000000..9fd14414e839 --- /dev/null +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/io/LocalTextModificationAccessor.java @@ -0,0 +1,135 @@ +/** + * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) + *

+ * Licensed 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.modification.io; + +import java.io.BufferedReader; +import java.io.BufferedWriter; +import java.io.FileNotFoundException; +import java.io.FileReader; +import java.io.FileWriter; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +import org.apache.iotdb.db.engine.modification.Deletion; +import org.apache.iotdb.db.engine.modification.Modification; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * LocalTextModificationAccessor uses a file on local file system to store the modifications + * in text format, and writes modifications by appending to the tail of the file. + */ +public class LocalTextModificationAccessor implements ModificationReader, ModificationWriter { + + private static final Logger logger = LoggerFactory.getLogger(LocalTextModificationAccessor.class); + private static final String SEPARATOR = ","; + + private String filePath; + private BufferedWriter writer; + + /** + * Construct a LocalTextModificationAccessor using a file specified by filePath. Only a writer + * will be created because the reader will be created only if necessary(call of read()). + * + * @param filePath the path of the file that is used for storing modifications. + * @throws IOException if the writer cannot be created. + */ + public LocalTextModificationAccessor(String filePath) throws IOException { + this.filePath = filePath; + writer = new BufferedWriter(new FileWriter(filePath)); + } + + @Override + public Collection read() throws IOException { + BufferedReader reader; + try { + reader = new BufferedReader(new FileReader(filePath)); + } catch (FileNotFoundException e) { + return null; + } + String line; + + List modificationList = new ArrayList<>(); + try { + while ((line = reader.readLine()) != null) { + modificationList.add(decodeModification(line)); + } + } catch (IOException e) { + reader.close(); + logger.error("An error occurred when reading modifications, and the remaining modifications " + + "were ignored.", e); + } + return modificationList; + } + + @Override + public void close() throws IOException { + writer.close(); + } + + @Override + public void write(Modification mod) throws IOException { + writer.write(encodeModification(mod)); + writer.newLine(); + writer.flush(); + } + + private static String encodeModification(Modification mod) { + if (mod instanceof Deletion) + return encodeDeletion((Deletion) mod); + return null; + } + + private static Modification decodeModification(String src) throws IOException { + String[] fields = src.split(SEPARATOR); + if (Modification.Type.DELETION.name().equals(fields[0])) { + return decodeDeletion(fields); + } + throw new IOException("Unknown modification type: " + fields[0]); + } + + private static String encodeDeletion(Deletion del) { + StringBuilder stringBuilder = new StringBuilder(); + stringBuilder.append(del.getType().toString()).append(SEPARATOR).append(del.getPath()) + .append(SEPARATOR).append(del.getVersionNum()).append(SEPARATOR) + .append(del.getTimestamp()); + return stringBuilder.toString(); + } + + private static Deletion decodeDeletion(String[] fields) throws IOException { + if (fields.length != 4) { + throw new IOException("Incorrect deletion fields number: " + fields.length); + } + + String path = fields[1]; + long versionNum, timestamp; + try { + versionNum = Long.parseLong(fields[2]); + } catch (NumberFormatException e) { + throw new IOException("Invalide version number: " + fields[2]); + } + try { + timestamp = Long.parseLong(fields[3]); + } catch (NumberFormatException e) { + throw new IOException("Invalide timestamp: " + fields[3]); + } + + return new Deletion(path, versionNum, timestamp); + } +} diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/io/ModificationReader.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/io/ModificationReader.java new file mode 100644 index 000000000000..d770eccbc6b5 --- /dev/null +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/io/ModificationReader.java @@ -0,0 +1,40 @@ +/** + * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) + *

+ * Licensed 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.modification.io; + +import java.io.IOException; +import java.util.Collection; + +import org.apache.iotdb.db.engine.modification.Modification; + +/** + * ModificationReader reads all modifications from a persistent medium like file system. + */ +public interface ModificationReader { + + /** + * Read all modifications from a persistent medium. + * + * @return a list of modifications contained the medium. + */ + Collection read() throws IOException; + + /** + * Release resources like streams. + */ + void close() throws IOException; +} diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/io/ModificationWriter.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/io/ModificationWriter.java new file mode 100644 index 000000000000..5c3806b3e1b6 --- /dev/null +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/io/ModificationWriter.java @@ -0,0 +1,39 @@ +/** + * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) + *

+ * Licensed 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.modification.io; + +import java.io.IOException; + +import org.apache.iotdb.db.engine.modification.Modification; + +/** + * ModificationWriter provides methods for writing a modification to a persistent medium like file + * system. + */ +public interface ModificationWriter { + + /** + * Write a new modification to the persistent medium. + * @param mod the modification to be written. + */ + void write(Modification mod) throws IOException; + + /** + * Release resources like streams. + */ + void close() throws IOException; +} diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/package-info.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/package-info.java new file mode 100644 index 000000000000..49697e1696c7 --- /dev/null +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/package-info.java @@ -0,0 +1,22 @@ +/** + * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) + *

+ * Licensed 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. + *

+ * modification is the functional module responsible for processing UPDATE and DELETE. + */ + +/** + * modification is the functional module responsible for processing UPDATE and DELETE. + */ +package org.apache.iotdb.db.engine.modification; \ No newline at end of file From f1cabf8848ed8e68c97a106e3d8644267960c8a3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=B1=9F=E5=A4=A9?= Date: Sun, 20 Jan 2019 10:32:43 +0800 Subject: [PATCH 02/24] move tests from old repo --- .../modification/ModificationFileTest.java | 62 +++++++++++++++ .../io/LocalTextModificationAccessorTest.java | 79 +++++++++++++++++++ 2 files changed, 141 insertions(+) create mode 100644 iotdb/src/test/java/org/apache/iotdb/db/engine/modification/ModificationFileTest.java create mode 100644 iotdb/src/test/java/org/apache/iotdb/db/engine/modification/io/LocalTextModificationAccessorTest.java diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/ModificationFileTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/ModificationFileTest.java new file mode 100644 index 000000000000..6907a3d8d63d --- /dev/null +++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/ModificationFileTest.java @@ -0,0 +1,62 @@ +/** + * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) + *

+ * Licensed 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.modification; + +import java.io.File; +import java.io.IOException; +import java.util.List; + +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +public class ModificationFileTest { + @Test + public void readMyWrite() { + String tempFileName = "mod.temp"; + Modification[] modifications = new Modification[]{ + new Deletion("p1", 1, 1), + new Deletion("p2", 2, 2), + new Deletion("p3", 3, 3), + new Deletion("p4", 4, 4), + }; + try { + ModificationFile mFile = new ModificationFile(tempFileName); + for (int i = 0; i < 2; i++) { + mFile.write(modifications[i]); + } + List modificationList = (List) mFile.getModifications(); + for (int i = 0; i < 2; i++) { + assertEquals(modifications[i], modificationList.get(i)); + } + + for (int i = 2; i < 4; i++) { + mFile.write(modifications[i]); + } + modificationList = (List) mFile.getModifications(); + for (int i = 0; i < 4; i++) { + assertEquals(modifications[i], modificationList.get(i)); + } + mFile.close(); + } catch (IOException e) { + fail(e.getMessage()); + } finally { + new File(tempFileName).delete(); + } + } +} \ No newline at end of file diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/io/LocalTextModificationAccessorTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/io/LocalTextModificationAccessorTest.java new file mode 100644 index 000000000000..125d243e76fe --- /dev/null +++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/io/LocalTextModificationAccessorTest.java @@ -0,0 +1,79 @@ +/** + * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) + *

+ * Licensed 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.modification.io; + +import java.io.File; +import java.io.IOException; +import java.util.Collection; +import java.util.List; + +import org.apache.iotdb.db.engine.modification.Deletion; +import org.apache.iotdb.db.engine.modification.Modification; +import org.junit.Test; + +import static org.junit.Assert.*; + +public class LocalTextModificationAccessorTest { + + @Test + public void readMyWrite() { + String tempFileName = "mod.temp"; + Modification[] modifications = new Modification[]{ + new Deletion("p1", 1, 1), + new Deletion("p2", 2, 2), + new Deletion("p3", 3, 3), + new Deletion("p4", 4, 4), + }; + try { + LocalTextModificationAccessor accessor = new LocalTextModificationAccessor(tempFileName); + for (int i = 0; i < 2; i++) { + accessor.write(modifications[i]); + } + List modificationList = (List) accessor.read(); + for (int i = 0; i < 2; i++) { + assertEquals(modifications[i], modificationList.get(i)); + } + + for (int i = 2; i < 4; i++) { + accessor.write(modifications[i]); + } + modificationList = (List) accessor.read(); + for (int i = 0; i < 4; i++) { + assertEquals(modifications[i], modificationList.get(i)); + } + accessor.close(); + } catch (IOException e) { + fail(e.getMessage()); + } finally { + new File(tempFileName).delete(); + } + } + + @Test + public void readNull() throws IOException { + String tempFileName = "mod.temp"; + LocalTextModificationAccessor accessor = null; + try { + accessor = new LocalTextModificationAccessor(tempFileName); + } catch (IOException e) { + fail(e.getMessage()); + } + new File(tempFileName).delete(); + Collection modifications = accessor.read(); + assertNull(modifications); + } +} \ No newline at end of file From 4a38abeb228236b397c4d74b4845452a1249f2ea Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=B1=9F=E5=A4=A9?= Date: Sun, 20 Jan 2019 10:47:43 +0800 Subject: [PATCH 03/24] move srcs from the old repo --- .../bufferwrite/BufferWriteProcessor.java | 29 +++++- .../db/engine/filenode/FileNodeManager.java | 59 ++++++------ .../db/engine/filenode/FileNodeProcessor.java | 78 ++++++++++++--- .../db/engine/filenode/IntervalFileNode.java | 27 +++++- .../db/engine/memtable/AbstractMemTable.java | 65 ++++++++++++- .../iotdb/db/engine/memtable/IMemTable.java | 17 +++- .../db/engine/memtable/IWritableMemChunk.java | 3 + .../db/engine/memtable/MemTableFlushUtil.java | 2 +- .../db/engine/memtable/PrimitiveMemTable.java | 17 ++++ .../db/engine/memtable/WritableMemChunk.java | 6 ++ .../engine/modification/ModificationFile.java | 10 +- .../io/LocalTextModificationAccessor.java | 15 +-- .../overflow/ioV2/OverflowProcessor.java | 39 ++++---- .../overflow/ioV2/OverflowResource.java | 43 ++++++--- .../engine/overflow/ioV2/OverflowSupport.java | 19 ++-- .../version/SimpleFileVersionController.java | 95 +++++++++++++++++++ .../version/SysTimeVersionController.java | 39 ++++++++ .../db/engine/version/VersionController.java | 34 +++++++ 18 files changed, 490 insertions(+), 107 deletions(-) create mode 100644 iotdb/src/main/java/org/apache/iotdb/db/engine/version/SimpleFileVersionController.java create mode 100644 iotdb/src/main/java/org/apache/iotdb/db/engine/version/SysTimeVersionController.java create mode 100644 iotdb/src/main/java/org/apache/iotdb/db/engine/version/VersionController.java diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessor.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessor.java index 5f841be3ac6f..fd05cd200397 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessor.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessor.java @@ -41,6 +41,7 @@ import org.apache.iotdb.db.engine.pool.FlushManager; import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk; import org.apache.iotdb.db.engine.utils.FlushStatus; +import org.apache.iotdb.db.engine.version.VersionController; import org.apache.iotdb.db.exception.BufferWriteProcessorException; import org.apache.iotdb.db.utils.MemUtils; import org.apache.iotdb.db.writelog.manager.MultiFileLogNodeManager; @@ -81,6 +82,7 @@ public class BufferWriteProcessor extends Processor { private String bufferWriteRelativePath; private WriteLogNode logNode; + private VersionController versionController; /** * constructor of BufferWriteProcessor. @@ -133,6 +135,7 @@ public BufferWriteProcessor(String baseDir, String processorName, String fileNam throw new BufferWriteProcessorException(e); } } + this.versionController = versionController; } /** @@ -259,7 +262,7 @@ private void switchFlushToWork() { } } - private void flushOperation(String flushFunction) { + private void flushOperation(String flushFunction, long version) { long flushStartTime = System.currentTimeMillis(); LOGGER.info("The bufferwrite processor {} starts flushing {}.", getProcessorName(), flushFunction); @@ -268,7 +271,8 @@ private void flushOperation(String flushFunction) { long startPos = writer.getPos(); long startTime = System.currentTimeMillis(); // flush data - MemTableFlushUtil.flushMemTable(fileSchema, writer, flushMemTable); + MemTableFlushUtil.flushMemTable(fileSchema, writer, flushMemTable, + version); // write restore information writer.flush(); } @@ -348,13 +352,14 @@ private Future flush(boolean synchronization) throws IOException { valueCount = 0; flushStatus.setFlushing(); switchWorkToFlush(); + long version = versionController.nextVersion(); BasicMemController.getInstance().reportFree(this, memSize.get()); memSize.set(0); // switch if (synchronization) { - flushOperation("synchronously"); + flushOperation("synchronously", version); } else { - FlushManager.getInstance().submit(() -> flushOperation("asynchronously")); + FlushManager.getInstance().submit(() -> flushOperation("asynchronously", version)); } } // TODO return a meaningful Future @@ -502,4 +507,20 @@ public void setNewProcessor(boolean isNewProcessor) { public WriteLogNode getLogNode() { return logNode; } + + /** + * Delete data whose timestamp <= 'timestamp' and belonging to timeseries deviceId.measurementId. + * Delete data in both working MemTable and flushing MemTable. + * @param deviceId the deviceId of the timeseries to be deleted. + * @param measurementId the measurementId of the timeseries to be deleted. + * @param timestamp the upper-bound of deletion time. + */ + public void delete(String deviceId, String measurementId, long timestamp) { + workMemTable.delele(deviceId, measurementId, timestamp); + if (isFlush) { + // flushing MemTable cannot be directly modified since another thread is reading it + flushMemTable = flushMemTable.copy(); + flushMemTable.delele(deviceId, measurementId, timestamp); + } + } } 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 1c6c1c1fedca..292e830b504c 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 @@ -53,7 +53,6 @@ import org.apache.iotdb.db.monitor.IStatistic; import org.apache.iotdb.db.monitor.MonitorConstants; import org.apache.iotdb.db.monitor.StatMonitor; -import org.apache.iotdb.db.qp.physical.crud.DeletePlan; import org.apache.iotdb.db.qp.physical.crud.InsertPlan; import org.apache.iotdb.db.qp.physical.crud.UpdatePlan; import org.apache.iotdb.db.query.control.FileReaderManager; @@ -497,66 +496,66 @@ public void update(String deviceId, String measurementId, long startTime, long e /** * delete data. */ - public void delete(String deviceId, String measurementId, long timestamp, TSDataType type) - throws FileNodeManagerException { + public void delete(String deviceId, String measurementId, long timestamp) + throws FileNodeManagerException { FileNodeProcessor fileNodeProcessor = getProcessor(deviceId, true); try { long lastUpdateTime = fileNodeProcessor.getLastUpdateTime(deviceId); // no tsfile data, the delete operation is invalid if (lastUpdateTime == -1) { - LOGGER.warn( - "The last update time is -1, delete overflow is invalid, the filenode processor is {}", - fileNodeProcessor.getProcessorName()); + LOGGER.warn("The last update time is -1, delete overflow is invalid, " + + "the filenode processor is {}", + fileNodeProcessor.getProcessorName()); } else { - if (timestamp > lastUpdateTime) { - timestamp = lastUpdateTime; - } - String filenodeName = fileNodeProcessor.getProcessorName(); - // get overflow processor - OverflowProcessor overflowProcessor; try { - overflowProcessor = fileNodeProcessor.getOverflowProcessor(filenodeName); + fileNodeProcessor.delete(deviceId, measurementId, timestamp); } catch (IOException e) { - LOGGER.error("Get the overflow processor failed, the filenode is {}, delete time is {}.", - filenodeName, timestamp); throw new FileNodeManagerException(e); } - overflowProcessor.delete(deviceId, measurementId, timestamp, type); // change the type of tsfile to overflowed fileNodeProcessor.changeTypeToChangedForDelete(deviceId, timestamp); fileNodeProcessor.setOverflowed(true); - // if (shouldMerge) { - // LOGGER.info( - // "The overflow file or metadata reaches the threshold, - // merge the filenode processor {}", - // filenodeName); - // fileNodeProcessor.submitToMerge(); - // } - fileNodeProcessor.changeTypeToChangedForDelete(deviceId, timestamp); - fileNodeProcessor.setOverflowed(true); - // write wal + // TODO: support atomic deletion + /*// write wal + // get processors for wal + String filenodeName = fileNodeProcessor.getProcessorName(); + OverflowProcessor overflowProcessor; + BufferWriteProcessor bufferWriteProcessor; + try { + overflowProcessor = fileNodeProcessor.getOverflowProcessor(filenodeName); + bufferWriteProcessor = fileNodeProcessor.getBufferWriteProcessor(); + } catch (IOException | FileNodeProcessorException e) { + LOGGER.error("Getting the processor failed, the filenode is {}, delete time is {}.", + filenodeName, timestamp); + throw new FileNodeManagerException(e); + } try { if (IoTDBDescriptor.getInstance().getConfig().enableWal) { overflowProcessor.getLogNode() - .write(new DeletePlan(timestamp, new Path(deviceId + "." + measurementId))); + .write(new DeletePlan(timestamp, + new Path(deviceId + "." + measurementId))); + bufferWriteProcessor.getLogNode() + .write(new DeletePlan(timestamp, + new Path(deviceId + "." + measurementId))); } } catch (IOException e) { throw new FileNodeManagerException(e); - } + }*/ } } finally { fileNodeProcessor.writeUnlock(); } + } /** * try to delete the filenode processor. */ private void delete(String processorName, - Iterator> processorIterator) - throws FileNodeManagerException { + Iterator> processorIterator) + throws FileNodeManagerException { if (processorMap.containsKey(processorName)) { LOGGER.info("Try to delete the filenode processor {}.", processorName); FileNodeProcessor processor = processorMap.get(processorName); 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 b802e6e62ac6..6a4eb28fda28 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 @@ -38,13 +38,16 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; + 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.apache.iotdb.db.engine.Processor; import org.apache.iotdb.db.engine.bufferwrite.Action; import org.apache.iotdb.db.engine.bufferwrite.BufferWriteProcessor; import org.apache.iotdb.db.engine.bufferwrite.FileNodeConstants; +import org.apache.iotdb.db.engine.modification.Deletion; import org.apache.iotdb.db.engine.overflow.ioV2.OverflowProcessor; import org.apache.iotdb.db.engine.pool.MergeManager; import org.apache.iotdb.db.engine.querycontext.GlobalSortedSeriesDataSource; @@ -52,6 +55,8 @@ import org.apache.iotdb.db.engine.querycontext.QueryDataSource; import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk; import org.apache.iotdb.db.engine.querycontext.UnsealedTsFile; +import org.apache.iotdb.db.engine.version.SimpleFileVersionController; +import org.apache.iotdb.db.engine.version.VersionController; import org.apache.iotdb.db.exception.BufferWriteProcessorException; import org.apache.iotdb.db.exception.ErrorDebugException; import org.apache.iotdb.db.exception.FileNodeProcessorException; @@ -194,6 +199,7 @@ public void act() throws Exception { }; // Token for query which used to private int multiPassLockToken = 0; + private VersionController versionController; /** * constructor of FileNodeProcessor. @@ -260,6 +266,11 @@ public FileNodeProcessor(String fileNodeDirPath, String processorName) registStatMetadata(); statMonitor.registStatistics(statStorageDeltaName, this); } + try { + versionController = new SimpleFileVersionController(fileNodeDirPath); + } catch (IOException e) { + throw new FileNodeProcessorException(e); + } } public HashMap getStatParamsHashMap() { @@ -299,7 +310,7 @@ public HashMap getAllStatisticsValue() { HashMap hashMap = getStatParamsHashMap(); tsRecord.dataPointList = new ArrayList() { { - for (Map.Entry entry : hashMap.entrySet()) { + for (Entry entry : hashMap.entrySet()) { add(new LongDataPoint(entry.getKey(), entry.getValue().get())); } } @@ -432,7 +443,7 @@ public void fileNodeRecovery() throws FileNodeProcessorException { getProcessorName(), fileNames[fileNames.length - 1]); try { bufferWriteProcessor = new BufferWriteProcessor(baseDir, getProcessorName(), - fileNames[fileNames.length - 1], parameters, fileSchema); + fileNames[fileNames.length - 1], parameters, versionController, fileSchema); } catch (BufferWriteProcessorException e) { // unlock writeUnlock(); @@ -449,7 +460,8 @@ public void fileNodeRecovery() throws FileNodeProcessorException { parameters.put(FileNodeConstants.OVERFLOW_FLUSH_ACTION, overflowFlushAction); parameters.put(FileNodeConstants.FILENODE_PROCESSOR_FLUSH_ACTION, flushFileNodeProcessorAction); try { - overflowProcessor = new OverflowProcessor(getProcessorName(), parameters, fileSchema); + overflowProcessor = new OverflowProcessor(getProcessorName(), parameters, fileSchema, + versionController); } catch (IOException e) { writeUnlock(); LOGGER.error("The filenode processor {} failed to recovery the overflow processor.", @@ -497,7 +509,7 @@ public BufferWriteProcessor getBufferWriteProcessor(String processorName, long i try { bufferWriteProcessor = new BufferWriteProcessor(baseDir, processorName, insertTime + FileNodeConstants.BUFFERWRITE_FILE_SEPARATOR + System.currentTimeMillis(), - parameters, fileSchema); + parameters, versionController, fileSchema); } catch (BufferWriteProcessorException e) { LOGGER.error("The filenode processor {} failed to get the bufferwrite processor.", processorName, e); @@ -528,7 +540,8 @@ public OverflowProcessor getOverflowProcessor(String processorName) throws IOExc parameters.put(FileNodeConstants.OVERFLOW_FLUSH_ACTION, overflowFlushAction); parameters .put(FileNodeConstants.FILENODE_PROCESSOR_FLUSH_ACTION, flushFileNodeProcessorAction); - overflowProcessor = new OverflowProcessor(processorName, parameters, fileSchema); + overflowProcessor = new OverflowProcessor(processorName, parameters, fileSchema, + versionController); } return overflowProcessor; } @@ -1452,7 +1465,7 @@ private void switchWaitingToWorkingv2(List backupIntervalFiles } private TSRecord constructTsRecord(TimeValuePair timeValuePair, String deviceId, - String measurementId) { + String measurementId) { TSRecord record = new TSRecord(timeValuePair.getTimestamp(), deviceId); record.addTuple(DataPoint.getDataPoint(timeValuePair.getValue().getDataType(), measurementId, timeValuePair.getValue().getValue().toString())); @@ -1546,7 +1559,7 @@ private String queryAndWriteDataForMerge(IntervalFileNode backupIntervalFile) // end the new rowGroupMetadata long size = fileIoWriter.getPos() - startPos; footer = new ChunkGroupFooter(deviceId, size, numOfChunk); - fileIoWriter.endChunkGroup(footer); + fileIoWriter.endChunkGroup(footer, versionController.nextVersion()); } } if (fileIoWriter != null) { @@ -1561,9 +1574,9 @@ private String queryAndWriteDataForMerge(IntervalFileNode backupIntervalFile) } private int writeOneSeries(String deviceId, String measurement, ChunkWriterImpl seriesWriterImpl, - TSDataType dataType, IReader seriesReader, Map startTimeMap, - Map endTimeMap, - TimeValuePair timeValuePair) throws IOException { + TSDataType dataType, IReader seriesReader, Map startTimeMap, + Map endTimeMap, + TimeValuePair timeValuePair) throws IOException { int count = 0; long startTime = -1; long endTime = -1; @@ -1877,6 +1890,15 @@ public void closeOverflow() throws FileNodeProcessorException { public void close() throws FileNodeProcessorException { closeBufferWrite(); closeOverflow(); + for (IntervalFileNode fileNode : newFileNodes) { + if (fileNode.getModFile() != null) { + try { + fileNode.getModFile().close(); + } catch (IOException e) { + throw new FileNodeProcessorException(e); + } + } + } } /** @@ -1930,7 +1952,6 @@ private FileNodeProcessorStore readStoreFromDisk() throws FileNodeProcessorExcep new IntervalFileNode(OverflowChangeType.NO_CHANGE, null), new ArrayList(), FileNodeProcessorStatus.NONE, 0)); } catch (IOException e) { - e.printStackTrace(); throw new FileNodeProcessorException(e); } return fileNodeProcessorStore; @@ -1942,7 +1963,36 @@ private FileNodeProcessorStore readStoreFromDisk() throws FileNodeProcessorExcep * { mergeIndex(); switchMergeIndex(); } */ - public String getFileNodeRestoreFilePath() { - return fileNodeRestoreFilePath; - } + public String getFileNodeRestoreFilePath() { + return fileNodeRestoreFilePath; + } + + /** + * Delete data whose timestamp <= 'timestamp' and belong to timeseries deviceId.measurementId. + * @param deviceId the deviceId of the timeseries to be deleted. + * @param measurementId the measurementId of the timeseries to be deleted. + * @param timestamp the delete range is (0, timestamp]. + */ + public void delete(String deviceId, String measurementId, long timestamp) throws IOException { + // TODO: how to avoid partial deletion? + long version = versionController.nextVersion(); + // delete data in memory + if (bufferWriteProcessor != null) { + bufferWriteProcessor.delete(deviceId, measurementId, timestamp); + } + OverflowProcessor overflowProcessor = getOverflowProcessor(getProcessorName()); + overflowProcessor.delete(deviceId, measurementId, timestamp, version); + + String fullPath = deviceId + + IoTDBConstant.PATH_SEPARATOR + measurementId; + Deletion deletion = new Deletion(fullPath, version, timestamp); + if (currentIntervalFileNode != null && currentIntervalFileNode.containsDevice(deviceId)) { + currentIntervalFileNode.getModFile().write(deletion); + } + for (IntervalFileNode fileNode : newFileNodes) { + if(fileNode != currentIntervalFileNode && fileNode.containsDevice(deviceId)) { + fileNode.getModFile().write(deletion); + } + } + } } \ No newline at end of file diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/IntervalFileNode.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/IntervalFileNode.java index 8575ee5c5473..a7a8841265cc 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/IntervalFileNode.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/filenode/IntervalFileNode.java @@ -25,7 +25,9 @@ import java.util.HashSet; import java.util.Map; import java.util.Set; + import org.apache.iotdb.db.conf.directories.Directories; +import org.apache.iotdb.db.engine.modification.ModificationFile; /** * This class is used to store one bufferwrite file status.
@@ -43,6 +45,7 @@ public class IntervalFileNode implements Serializable { private Map startTimeMap; private Map endTimeMap; private Set mergeChanged = new HashSet<>(); + private transient ModificationFile modFile; public IntervalFileNode(Map startTimeMap, Map endTimeMap, OverflowChangeType type, int baseDirIndex, String relativePath) { @@ -53,7 +56,9 @@ public IntervalFileNode(Map startTimeMap, Map endTim this.startTimeMap = startTimeMap; this.endTimeMap = endTimeMap; - + this.modFile = new ModificationFile( + Directories.getInstance().getTsFileFolder(baseDirIndex) + File.separator + + relativePath + ModificationFile.FILE_SUFFIX); } /** @@ -70,6 +75,9 @@ public IntervalFileNode(OverflowChangeType type, int baseDirIndex, String relati startTimeMap = new HashMap<>(); endTimeMap = new HashMap<>(); + this.modFile = new ModificationFile( + Directories.getInstance().getTsFileFolder(baseDirIndex) + File.separator + + relativePath + ModificationFile.FILE_SUFFIX); } public IntervalFileNode(OverflowChangeType type, String baseDir, String relativePath) { @@ -80,6 +88,9 @@ public IntervalFileNode(OverflowChangeType type, String baseDir, String relative startTimeMap = new HashMap<>(); endTimeMap = new HashMap<>(); + this.modFile = new ModificationFile( + Directories.getInstance().getTsFileFolder(baseDirIndex) + File.separator + + relativePath + ModificationFile.FILE_SUFFIX); } public IntervalFileNode(OverflowChangeType type, String relativePath) { @@ -279,4 +290,18 @@ public String toString() { + " endTimeMap=%s, mergeChanged=%s]", relativePath, overflowChangeType, startTimeMap, endTimeMap, mergeChanged); } + + public synchronized ModificationFile getModFile() { + if (modFile == null) { + modFile = new ModificationFile( + Directories.getInstance().getTsFileFolder(baseDirIndex) + File.separator + + relativePath + ModificationFile.FILE_SUFFIX); + } + return modFile; + } + + public boolean containsDevice(String deviceId) { + return endTimeMap.containsKey(deviceId); + } + } diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java index 48abbb68d9eb..db5bfcf7e5fb 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java @@ -20,7 +20,10 @@ package org.apache.iotdb.db.engine.memtable; import java.util.HashMap; +import java.util.List; import java.util.Map; + +import org.apache.iotdb.db.utils.TimeValuePair; import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; public abstract class AbstractMemTable implements IMemTable { @@ -31,6 +34,10 @@ public AbstractMemTable() { this.memTableMap = new HashMap<>(); } + public AbstractMemTable(Map> memTableMap) { + this.memTableMap = memTableMap; + } + @Override public Map> getMemTableMap() { return memTableMap; @@ -46,7 +53,7 @@ private boolean checkPath(String deviceId, String measurement) { } private IWritableMemChunk createIfNotExistAndGet(String deviceId, String measurement, - TSDataType dataType) { + TSDataType dataType) { if (!memTableMap.containsKey(deviceId)) { memTableMap.put(deviceId, new HashMap<>()); } @@ -61,7 +68,7 @@ private IWritableMemChunk createIfNotExistAndGet(String deviceId, String measure @Override public void write(String deviceId, String measurement, TSDataType dataType, long insertTime, - String insertValue) { + String insertValue) { IWritableMemChunk memSeries = createIfNotExistAndGet(deviceId, measurement, dataType); memSeries.write(insertTime, insertValue); } @@ -95,4 +102,58 @@ public TimeValuePairSorter query(String deviceId, String measurement, TSDataType return memTableMap.get(deviceId).get(measurement); } + @Override + public void delele(String deviceId, String measurementId, long timestamp) { + Map deviceMap = memTableMap.get(deviceId); + if (deviceMap != null) { + IWritableMemChunk chunk = deviceMap.get(measurementId); + IWritableMemChunk newChunk = filterChunk(chunk, timestamp); + if (newChunk != null) { + deviceMap.put(measurementId, newChunk); + } + } + } + + /** + * If chunk contains data with timestamp less than 'timestamp', create a copy and delete all + * those data. Otherwise return null. + * @param chunk the source chunk. + * @param timestamp the upper-bound of deletion time. + * @return A reduced copy of chunk if chunk contains data with timestamp less than 'timestamp', + * of null. + */ + private IWritableMemChunk filterChunk(IWritableMemChunk chunk, long timestamp) { + List timeValuePairs = chunk.getSortedTimeValuePairList(); + if (timeValuePairs.size() > 0 && timeValuePairs.get(0).getTimestamp() <= timestamp) { + TSDataType dataType = chunk.getType(); + IWritableMemChunk newChunk = genMemSeries(dataType); + for (TimeValuePair pair : timeValuePairs) { + if (pair.getTimestamp() > timestamp) { + switch (dataType) { + case BOOLEAN: + newChunk.putBoolean(pair.getTimestamp(), pair.getValue().getBoolean()); + break; + case DOUBLE: + newChunk.putDouble(pair.getTimestamp(), pair.getValue().getDouble()); + break; + case INT64: + newChunk.putLong(pair.getTimestamp(), pair.getValue().getLong()); + break; + case INT32: + newChunk.putInt(pair.getTimestamp(), pair.getValue().getInt()); + break; + case FLOAT: + newChunk.putFloat(pair.getTimestamp(), pair.getValue().getFloat()); + break; + case TEXT: + newChunk.putBinary(pair.getTimestamp(), pair.getValue().getBinary()); + break; + } + } + } + return newChunk; + } + return null; + } + } diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java index e5e5bad005cc..422a645fe541 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.engine.memtable; import java.util.Map; + import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; /** @@ -34,7 +35,7 @@ public interface IMemTable { Map> getMemTableMap(); void write(String deviceId, String measurement, TSDataType dataType, - long insertTime, String insertValue); + long insertTime, String insertValue); int size(); @@ -47,4 +48,18 @@ void write(String deviceId, String measurement, TSDataType dataType, boolean isEmpty(); + /** + * Delete data in it whose timestamp <= 'timestamp' and belonging to timeseries + * deviceId.measurementId. + * @param deviceId the deviceId of the timeseries to be deleted. + * @param measurementId the measurementId of the timeseries to be deleted. + * @param timestamp the upper-bound of deletion time. + */ + void delele(String deviceId, String measurementId, long timestamp); + + /** + * Make a copy of this MemTable. + * @return a MemTable with the same data as this one. + */ + IMemTable copy(); } diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/IWritableMemChunk.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/IWritableMemChunk.java index e20bc5e9f6d2..ee67b3a5edb4 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/IWritableMemChunk.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/IWritableMemChunk.java @@ -19,6 +19,7 @@ */ package org.apache.iotdb.db.engine.memtable; +import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; import org.apache.iotdb.tsfile.utils.Binary; public interface IWritableMemChunk extends TimeValuePairSorter { @@ -40,4 +41,6 @@ public interface IWritableMemChunk extends TimeValuePairSorter { void reset(); int count(); + + TSDataType getType(); } diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/MemTableFlushUtil.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/MemTableFlushUtil.java index 180329e9a9bc..80477e017672 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/MemTableFlushUtil.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/MemTableFlushUtil.java @@ -116,7 +116,7 @@ public static void flushMemTable(FileSchema fileSchema, TsFileIOWriter tsFileIoW } long memSize = tsFileIoWriter.getPos() - startPos; ChunkGroupFooter footer = new ChunkGroupFooter(deviceId, memSize, seriesNumber); - tsFileIoWriter.endChunkGroup(footer); + tsFileIoWriter.endChunkGroup(footer, version); } } } diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/PrimitiveMemTable.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/PrimitiveMemTable.java index 6961ee69e6c8..51229c67da21 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/PrimitiveMemTable.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/PrimitiveMemTable.java @@ -19,12 +19,29 @@ */ package org.apache.iotdb.db.engine.memtable; +import java.util.HashMap; +import java.util.Map; + import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; public class PrimitiveMemTable extends AbstractMemTable { + public PrimitiveMemTable() { + } + + public PrimitiveMemTable(Map> memTableMap) { + super(memTableMap); + } + @Override protected IWritableMemChunk genMemSeries(TSDataType dataType) { return new WritableMemChunk(dataType); } + + @Override + public IMemTable copy() { + Map> newMap = new HashMap<>(getMemTableMap()); + + return new PrimitiveMemTable(newMap); + } } diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/WritableMemChunk.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/WritableMemChunk.java index d3fcbb9c0e84..a279e74db861 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/WritableMemChunk.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/WritableMemChunk.java @@ -22,6 +22,7 @@ import java.util.ArrayList; import java.util.List; import java.util.TreeMap; + import org.apache.iotdb.db.utils.PrimitiveArrayList; import org.apache.iotdb.db.utils.PrimitiveArrayListFactory; import org.apache.iotdb.db.utils.TimeValuePair; @@ -121,4 +122,9 @@ public int count() { return list.size(); } + @Override + public TSDataType getType() { + return dataType; + } + } diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/ModificationFile.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/ModificationFile.java index c04ff1d12ec4..72be89f9fdb3 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/ModificationFile.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/ModificationFile.java @@ -30,19 +30,17 @@ * Methods in this class are highly synchronized for concurrency safety. */ public class ModificationFile { + public static final String FILE_SUFFIX = ".mods"; private Collection modifications; - private String filePath; private ModificationWriter writer; private ModificationReader reader; /** * Construct a ModificationFile using a file as its storage. * @param filePath the path of the storage file. - * @throws IOException when IOException raised when */ - public ModificationFile(String filePath) throws IOException { - this.filePath = filePath; + public ModificationFile(String filePath) { LocalTextModificationAccessor accessor = new LocalTextModificationAccessor(filePath); this.writer = accessor; this.reader = accessor; @@ -65,7 +63,7 @@ private void checkInit() throws IOException { } /** - * Release the resources such as streams and caches. + * Release resources such as streams and caches. */ public void close() throws IOException { synchronized (this) { @@ -90,7 +88,7 @@ public void write(Modification mod) throws IOException { /** * Get all modifications stored in this file. - * @return + * @return an ArrayList of modifications. */ public Collection getModifications() throws IOException { synchronized (this) { 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 9fd14414e839..9f11bae81e58 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 @@ -44,17 +44,13 @@ public class LocalTextModificationAccessor implements ModificationReader, Modifi private BufferedWriter writer; /** - * Construct a LocalTextModificationAccessor using a file specified by filePath. Only a writer - * will be created because the reader will be created only if necessary(call of read()). + * Construct a LocalTextModificationAccessor using a file specified by filePath. * * @param filePath the path of the file that is used for storing modifications. - * @throws IOException if the writer cannot be created. */ - public LocalTextModificationAccessor(String filePath) throws IOException { + public LocalTextModificationAccessor(String filePath) { this.filePath = filePath; - writer = new BufferedWriter(new FileWriter(filePath)); } - @Override public Collection read() throws IOException { BufferedReader reader; @@ -80,11 +76,16 @@ public Collection read() throws IOException { @Override public void close() throws IOException { - writer.close(); + if (writer != null) { + writer.close(); + } } @Override public void write(Modification mod) throws IOException { + if (writer == null) { + writer = new BufferedWriter(new FileWriter(filePath, true)); + } writer.write(encodeModification(mod)); writer.newLine(); writer.flush(); diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowProcessor.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowProcessor.java index fa1ab00edfe9..cb41970707b1 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowProcessor.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowProcessor.java @@ -46,6 +46,7 @@ import org.apache.iotdb.db.engine.querycontext.OverflowSeriesDataSource; import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk; import org.apache.iotdb.db.engine.utils.FlushStatus; +import org.apache.iotdb.db.engine.version.VersionController; import org.apache.iotdb.db.exception.OverflowProcessorException; import org.apache.iotdb.db.utils.MemUtils; import org.apache.iotdb.db.writelog.manager.MultiFileLogNodeManager; @@ -90,9 +91,10 @@ public class OverflowProcessor extends Processor { private AtomicLong memSize = new AtomicLong(); private WriteLogNode logNode; + private VersionController versionController; public OverflowProcessor(String processorName, Map parameters, - FileSchema fileSchema) + FileSchema fileSchema, VersionController versionController) throws IOException { super(processorName); this.fileSchema = fileSchema; @@ -119,18 +121,19 @@ public OverflowProcessor(String processorName, Map parameters, processorName + IoTDBConstant.OVERFLOW_LOG_NODE_SUFFIX, getOverflowRestoreFile(), FileNodeManager.getInstance().getRestoreFilePath(processorName)); } + this.versionController = versionController; } private void recovery(File parentFile) throws IOException { String[] subFilePaths = clearFile(parentFile.list()); if (subFilePaths.length == 0) { workResource = new OverflowResource(parentPath, - String.valueOf(dataPahtCount.getAndIncrement())); + String.valueOf(dataPahtCount.getAndIncrement()), versionController); return; } else if (subFilePaths.length == 1) { long count = Long.valueOf(subFilePaths[0]); dataPahtCount.addAndGet(count + 1); - workResource = new OverflowResource(parentPath, String.valueOf(count)); + workResource = new OverflowResource(parentPath, String.valueOf(count), versionController); LOGGER.info("The overflow processor {} recover from work status.", getProcessorName()); } else { long count1 = Long.valueOf(subFilePaths[0]); @@ -142,8 +145,8 @@ private void recovery(File parentFile) throws IOException { } dataPahtCount.addAndGet(count2 + 1); // work dir > merge dir - workResource = new OverflowResource(parentPath, String.valueOf(count2)); - mergeResource = new OverflowResource(parentPath, String.valueOf(count1)); + workResource = new OverflowResource(parentPath, String.valueOf(count2), versionController); + mergeResource = new OverflowResource(parentPath, String.valueOf(count1), versionController); LOGGER.info("The overflow processor {} recover from merge status.", getProcessorName()); } } @@ -231,17 +234,20 @@ private byte[] convertStringToBytes(TSDataType type, String o) { } /** - * delete one time-series data which time range is from 0 to time-stamp. + * Delete data of a timeseries whose time ranges from 0 to timestamp. * - * @param deviceId - * @param measurementId - * @param timestamp - * @param type + * @param deviceId the deviceId of the timeseries. + * @param measurementId the measurementId of the timeseries. + * @param timestamp the upper-bound of deletion time. + * @param version the version number of this deletion. */ - @Deprecated - public void delete(String deviceId, String measurementId, long timestamp, TSDataType type) { - workSupport.delete(deviceId, measurementId, timestamp, type); - valueCount++; + public void delete(String deviceId, String measurementId, long timestamp, long version) throws IOException { + workSupport.delete(deviceId, measurementId, timestamp, false); + workResource.delete(deviceId, measurementId, timestamp, version); + if (flushStatus.isFlushing()) { + flushSupport.delete(deviceId, measurementId, timestamp, true); + mergeResource.delete(deviceId, measurementId, timestamp, version); + } } /** @@ -420,7 +426,7 @@ public void switchWorkToMerge() throws IOException { mergeResource = workResource; // TODO: NEW ONE workResource workResource = new OverflowResource(parentPath, - String.valueOf(dataPahtCount.getAndIncrement())); + String.valueOf(dataPahtCount.getAndIncrement()), versionController); } isMerge = true; LOGGER.info("The overflow processor {} switch from WORK to MERGE", getProcessorName()); @@ -613,8 +619,7 @@ public long getMetaSize() { * @return The size of overflow file corresponding to this processor. */ public long getFileSize() { - return workResource.getInsertFile().length() + workResource.getUpdateDeleteFile().length() - + memoryUsage(); + return workResource.getInsertFile().length() + memoryUsage(); } /** diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowResource.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowResource.java index f2b71603ae8b..4844d3d4173d 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowResource.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowResource.java @@ -28,9 +28,14 @@ import java.util.HashMap; import java.util.List; import java.util.Map; + import org.apache.commons.io.FileUtils; +import org.apache.iotdb.db.conf.IoTDBConstant; import org.apache.iotdb.db.engine.memtable.IMemTable; import org.apache.iotdb.db.engine.memtable.MemTableFlushUtil; +import org.apache.iotdb.db.engine.modification.Deletion; +import org.apache.iotdb.db.engine.modification.ModificationFile; +import org.apache.iotdb.db.engine.version.VersionController; import org.apache.iotdb.db.utils.MemUtils; import org.apache.iotdb.tsfile.file.metadata.ChunkGroupMetaData; import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData; @@ -46,7 +51,6 @@ public class OverflowResource { private static final Logger LOGGER = LoggerFactory.getLogger(OverflowResource.class); private static final String insertFileName = "unseqTsFile"; - private static final String updateDeleteFileName = "overflowFile"; private static final String positionFileName = "positionFile"; private static final int FOOTER_LENGTH = 4; private static final int POS_LENGTH = 8; @@ -55,12 +59,14 @@ public class OverflowResource { private String insertFilePath; private String positionFilePath; private File insertFile; - private File updateFile; private OverflowIO insertIO; private Map>> insertMetadatas; private List appendInsertMetadatas; + private VersionController versionController; + private ModificationFile modificationFile; - public OverflowResource(String parentPath, String dataPath) throws IOException { + public OverflowResource(String parentPath, String dataPath, VersionController versionController) + throws IOException { this.insertMetadatas = new HashMap<>(); this.appendInsertMetadatas = new ArrayList<>(); this.parentPath = parentPath; @@ -71,7 +77,6 @@ public OverflowResource(String parentPath, String dataPath) throws IOException { } insertFile = new File(dataFile, insertFileName); insertFilePath = insertFile.getPath(); - updateFile = new File(dataFile, updateDeleteFileName); positionFilePath = new File(dataFile, positionFileName).getPath(); Pair position = readPositionInfo(); try { @@ -90,6 +95,8 @@ public OverflowResource(String parentPath, String dataPath) throws IOException { LOGGER.error("Failed to construct the OverflowIO.", e); throw e; } + this.versionController = versionController; + modificationFile = new ModificationFile(insertFilePath + ModificationFile.FILE_SUFFIX); } private Pair readPositionInfo() { @@ -110,9 +117,6 @@ private Pair readPositionInfo() { if (insertFile.exists()) { left = insertFile.length(); } - if (updateFile.exists()) { - right = updateFile.length(); - } return new Pair(left, right); } } @@ -160,7 +164,7 @@ private void readMetadata() throws IOException { } public List getInsertMetadatas(String deviceId, String measurementId, - TSDataType dataType) { + TSDataType dataType) { List chunkMetaDatas = new ArrayList<>(); if (insertMetadatas.containsKey(deviceId)) { if (insertMetadatas.get(deviceId).containsKey(measurementId)) { @@ -176,7 +180,7 @@ public List getInsertMetadatas(String deviceId, String measuremen } public void flush(FileSchema fileSchema, IMemTable memTable, - Map> overflowTrees, String processorName) + Map> overflowTrees, String processorName) throws IOException { // insert data long startPos = insertIO.getPos(); @@ -196,7 +200,8 @@ public void flush(FileSchema fileSchema, IMemTable memTable) throws IOException if (memTable != null && !memTable.isEmpty()) { insertIO.toTail(); long lastPosition = insertIO.getPos(); - MemTableFlushUtil.flushMemTable(fileSchema, insertIO, memTable); + MemTableFlushUtil.flushMemTable(fileSchema, insertIO, memTable, + versionController.nextVersion()); List rowGroupMetaDatas = insertIO.getChunkGroupMetaDatas(); appendInsertMetadatas.addAll(rowGroupMetaDatas); if (!rowGroupMetaDatas.isEmpty()) { @@ -237,15 +242,12 @@ public String getPositionFilePath() { return positionFilePath; } - public File getUpdateDeleteFile() { - return updateFile; - } - public void close() throws IOException { insertMetadatas.clear(); // updateDeleteMetadatas.clear(); insertIO.close(); // updateDeleteIO.close(); + modificationFile.close(); } public void deleteResource() throws IOException { @@ -277,4 +279,17 @@ private void addInsertMetadata(String deviceId, String measurementId, } insertMetadatas.get(deviceId).get(measurementId).add(chunkMetaData); } + + /** + * Delete data of a timeseries whose time ranges from 0 to timestamp. + * + * @param deviceId the deviceId of the timeseries. + * @param measurementId the measurementId of the timeseries. + * @param timestamp the upper-bound of deletion time. + */ + public void delete(String deviceId, String measurementId, long timestamp, long version) + throws IOException { + modificationFile.write(new Deletion(deviceId + IoTDBConstant.PATH_SEPARATOR + + measurementId, version, timestamp)); + } } \ No newline at end of file diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowSupport.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowSupport.java index 90563a665136..d608f280982b 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowSupport.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowSupport.java @@ -21,6 +21,7 @@ import java.util.HashMap; import java.util.Map; + import org.apache.iotdb.db.engine.memtable.IMemTable; import org.apache.iotdb.db.engine.memtable.PrimitiveMemTable; import org.apache.iotdb.db.engine.memtable.TimeValuePairSorter; @@ -75,15 +76,13 @@ public void update(String deviceId, String measurementId, long startTime, long e indexTrees.get(deviceId).get(measurementId).update(startTime, endTime, value); } - @Deprecated - public void delete(String deviceId, String measurementId, long timestamp, TSDataType dataType) { - if (!indexTrees.containsKey(deviceId)) { - indexTrees.put(deviceId, new HashMap<>()); + public void delete(String deviceId, String measurementId, long timestamp, boolean isFlushing) { + if (isFlushing) { + memTable = memTable.copy(); + memTable.delele(deviceId, measurementId, timestamp); + } else { + memTable.delele(deviceId, measurementId, timestamp); } - if (!indexTrees.get(deviceId).containsKey(measurementId)) { - indexTrees.get(deviceId).put(measurementId, new OverflowSeriesImpl(measurementId, dataType)); - } - indexTrees.get(deviceId).get(measurementId).delete(timestamp); } public TimeValuePairSorter queryOverflowInsertInMemory(String deviceId, String measurementId, @@ -92,8 +91,8 @@ public TimeValuePairSorter queryOverflowInsertInMemory(String deviceId, String m } public BatchData queryOverflowUpdateInMemory(String deviceId, String measurementId, - TSDataType dataType, - BatchData data) { + TSDataType dataType, + BatchData data) { if (indexTrees.containsKey(deviceId)) { if (indexTrees.get(deviceId).containsKey(measurementId) && indexTrees.get(deviceId).get(measurementId).getDataType().equals(dataType)) { diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/version/SimpleFileVersionController.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/version/SimpleFileVersionController.java new file mode 100644 index 000000000000..288c1fb3f935 --- /dev/null +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/version/SimpleFileVersionController.java @@ -0,0 +1,95 @@ +/** + * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) + *

+ * Licensed 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.version; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.util.Arrays; +import java.util.Comparator; + +/** + * SimpleFileVersionController uses a local file and its file name to store the version. + */ +public class SimpleFileVersionController implements VersionController { + + /** + * Every time currVersion - prevVersion >= SAVE_INTERVAL, currVersion is persisted and prevVersion + * is set to currVersion. When recovering from file, the version number is automatically increased + * by SAVE_INTERVAL to avoid conflicts. + */ + public static final long SAVE_INTERVAL = 100; + private static final String FILE_PREFIX = "Version-"; + private long prevVersion; + private long currVersion; + private String directoryPath; + + public SimpleFileVersionController(String directoryPath) throws IOException { + this.directoryPath = directoryPath; + restore(); + } + + @Override + public synchronized long nextVersion() { + currVersion ++; + checkPersist(); + return currVersion; + } + + /** + * Test only method, no need for concurrency. + * @return the current version. + */ + @Override + public long currVersion() { + return currVersion; + } + + private void checkPersist() { + if ((currVersion - prevVersion) >= SAVE_INTERVAL) { + persist(); + } + } + + private void persist() { + File oldFile = new File(directoryPath,FILE_PREFIX + prevVersion); + File newFile = new File(directoryPath, FILE_PREFIX + currVersion); + oldFile.renameTo(newFile); + prevVersion = currVersion; + } + + private void restore() throws IOException { + File directory = new File(directoryPath); + File[] versionFiles = directory.listFiles((dir, name) -> name.startsWith(FILE_PREFIX)); + File versionFile = null; + if (versionFiles != null && versionFiles.length > 0) { + Arrays.sort(versionFiles, Comparator.comparing(File::getName)); + versionFile = versionFiles[versionFiles.length - 1]; + for(int i = 0; i < versionFiles.length - 1; i ++) { + versionFiles[i].delete(); + } + } else { + versionFile = new File(directory, FILE_PREFIX + "0"); + new FileOutputStream(versionFile).close(); + } + // extract version from "Version-123456" + prevVersion = Long.parseLong(versionFile.getName().split("-")[1]); + // prevent overlapping in case of failure + currVersion = prevVersion + SAVE_INTERVAL; + persist(); + } +} diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/version/SysTimeVersionController.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/version/SysTimeVersionController.java new file mode 100644 index 000000000000..21e9787d4747 --- /dev/null +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/version/SysTimeVersionController.java @@ -0,0 +1,39 @@ +/** + * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) + *

+ * Licensed 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.version; + +/** + * SysTimeVersionController uses system timestamp as the version number. + */ +public class SysTimeVersionController implements VersionController { + + public static final SysTimeVersionController INSTANCE = new SysTimeVersionController(); + + private SysTimeVersionController() { + + } + + @Override + public long nextVersion() { + return System.currentTimeMillis(); + } + + @Override + public long currVersion() { + return System.currentTimeMillis(); + } +} diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/version/VersionController.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/version/VersionController.java new file mode 100644 index 000000000000..68ca4b9e46a4 --- /dev/null +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/version/VersionController.java @@ -0,0 +1,34 @@ +/** + * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) + *

+ * Licensed 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.version; + +/** + * VersionController controls the version(a monotonic increasing long) of a FileNode. + */ +public interface VersionController { + /** + * Get the next version number. + * @return the next version number. + */ + long nextVersion(); + + /** + * Get the current version number. + * @return the current version number. + */ + long currVersion(); +} From 6559289aef032227bc5ff1b9036cc0dc0235b8cb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=B1=9F=E5=A4=A9?= Date: Sun, 20 Jan 2019 10:49:36 +0800 Subject: [PATCH 04/24] move tests from the old repo --- .../overflow/ioV2/OverflowProcessor.java | 21 +- .../modification/DeletionFileNodeTest.java | 235 ++++++++++++++++++ .../io/LocalTextModificationAccessorTest.java | 6 +- 3 files changed, 247 insertions(+), 15 deletions(-) create mode 100644 iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowProcessor.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowProcessor.java index cb41970707b1..763fd7eb313b 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowProcessor.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowProcessor.java @@ -30,6 +30,7 @@ import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantLock; + import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBConstant; import org.apache.iotdb.db.conf.IoTDBDescriptor; @@ -94,7 +95,7 @@ public class OverflowProcessor extends Processor { private VersionController versionController; public OverflowProcessor(String processorName, Map parameters, - FileSchema fileSchema, VersionController versionController) + FileSchema fileSchema, VersionController versionController) throws IOException { super(processorName); this.fileSchema = fileSchema; @@ -262,7 +263,7 @@ public void delete(String deviceId, String measurementId, long timestamp, long v * @throws IOException */ public OverflowSeriesDataSource query(String deviceId, String measurementId, Filter filter, - TSDataType dataType) + TSDataType dataType) throws IOException { queryFlushLock.lock(); try { @@ -303,7 +304,7 @@ public OverflowSeriesDataSource query(String deviceId, String measurementId, Fil * @return insert data in SeriesChunkInMemTable */ private TimeValuePairSorter queryOverflowInsertInMemory(String deviceId, String measurementId, - TSDataType dataType) { + TSDataType dataType) { MemSeriesLazyMerger memSeriesLazyMerger = new MemSeriesLazyMerger(); if (flushStatus.isFlushing()) { @@ -341,8 +342,8 @@ private TimeValuePairSorter queryOverflowInsertInMemory(String deviceId, String * @return the seriesPath of unseqTsFile, List of TimeSeriesChunkMetaData for the special time-series. */ private Pair> queryWorkDataInOverflowInsert(String deviceId, - String measurementId, - TSDataType dataType) { + String measurementId, + TSDataType dataType) { Pair> pair = new Pair>( workResource.getInsertFilePath(), workResource.getInsertMetadatas(deviceId, measurementId, dataType)); @@ -358,7 +359,7 @@ private Pair> queryWorkDataInOverflowInsert(String d * @return MergeSeriesDataSource */ public MergeSeriesDataSource queryMerge(String deviceId, String measurementId, - TSDataType dataType) { + TSDataType dataType) { Pair> mergeInsert = queryMergeDataInOverflowInsert(deviceId, measurementId, dataType); @@ -366,8 +367,8 @@ public MergeSeriesDataSource queryMerge(String deviceId, String measurementId, } public OverflowSeriesDataSource queryMerge(String deviceId, String measurementId, - TSDataType dataType, - boolean isMerge) { + TSDataType dataType, + boolean isMerge) { Pair> mergeInsert = queryMergeDataInOverflowInsert(deviceId, measurementId, dataType); @@ -389,8 +390,8 @@ public OverflowSeriesDataSource queryMerge(String deviceId, String measurementId * @return the seriesPath of unseqTsFile, List of TimeSeriesChunkMetaData for the special time-series. */ private Pair> queryMergeDataInOverflowInsert(String deviceId, - String measurementId, - TSDataType dataType) { + String measurementId, + TSDataType dataType) { if (!isMerge) { return new Pair>(null, null); } diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java new file mode 100644 index 000000000000..1cfa9467ffe4 --- /dev/null +++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java @@ -0,0 +1,235 @@ +/** + * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) + *

+ * Licensed 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.modification; + +import java.io.File; +import java.io.IOException; +import java.util.Collection; +import java.util.Iterator; + +import org.apache.iotdb.db.conf.IoTDBDescriptor; +import org.apache.iotdb.db.conf.directories.Directories; +import org.apache.iotdb.db.engine.filenode.FileNodeManager; +import org.apache.iotdb.db.engine.modification.io.LocalTextModificationAccessor; +import org.apache.iotdb.db.engine.querycontext.QueryDataSource; +import org.apache.iotdb.db.exception.FileNodeManagerException; +import org.apache.iotdb.db.exception.MetadataArgsErrorException; +import org.apache.iotdb.db.exception.PathErrorException; +import org.apache.iotdb.db.metadata.MManager; +import org.apache.iotdb.db.utils.EnvironmentUtils; +import org.apache.iotdb.db.utils.TimeValuePair; +import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; +import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; +import org.apache.iotdb.tsfile.read.common.Path; +import org.apache.iotdb.tsfile.read.expression.impl.SingleSeriesExpression; +import org.apache.iotdb.tsfile.write.record.TSRecord; +import org.apache.iotdb.tsfile.write.record.datapoint.DoubleDataPoint; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import static junit.framework.TestCase.assertTrue; +import static org.junit.Assert.assertEquals; + +public class DeletionFileNodeTest { + + private String processorName = "root.test"; + + private static String[] measurements = new String[10]; + private String dataType = TSDataType.DOUBLE.toString(); + private String encoding = TSEncoding.PLAIN.toString(); + private String[] args = new String[0]; + + static { + for (int i = 0; i < 10; i++) { + measurements[i] = "m" + i; + } + } + + @Before + public void setup() throws MetadataArgsErrorException, + PathErrorException, IOException, FileNodeManagerException { + MManager.getInstance().setStorageLevelToMTree(processorName); + for (int i = 0; i < 10; i++) { + MManager.getInstance().addPathToMTree(processorName + "." + measurements[i], dataType, + encoding, args); + FileNodeManager.getInstance().addTimeSeries(new Path(processorName, measurements[i]), dataType, + encoding, args); + } + } + + @After + public void teardown() throws IOException, FileNodeManagerException { + EnvironmentUtils.cleanEnv(); + } + + @Test + public void testDeleteInBufferWriteCache() throws + FileNodeManagerException { + + for (int i = 1; i <= 100; i++) { + TSRecord record = new TSRecord(i, processorName); + for (int j = 0; j < 10; j++) { + record.addTuple(new DoubleDataPoint(measurements[j], i * 1.0)); + } + FileNodeManager.getInstance().insert(record, false); + } + + FileNodeManager.getInstance().delete(processorName, measurements[3], 50); + FileNodeManager.getInstance().delete(processorName, measurements[4], 50); + FileNodeManager.getInstance().delete(processorName, measurements[5], 30); + FileNodeManager.getInstance().delete(processorName, measurements[5], 50); + + SingleSeriesExpression expression = new SingleSeriesExpression(new Path(processorName, + measurements[5]), null); + QueryDataSource dataSource = FileNodeManager.getInstance().query(expression); + Iterator timeValuePairs = + dataSource.getSeqDataSource().getReadableChunk().getIterator(); + int count = 0; + while (timeValuePairs.hasNext()) { + timeValuePairs.next(); + count++; + } + assertEquals(count, 50); + } + + @Test + public void testDeleteInBufferWriteFile() throws FileNodeManagerException, IOException { + for (int i = 1; i <= 100; i++) { + TSRecord record = new TSRecord(i, processorName); + for (int j = 0; j < 10; j++) { + record.addTuple(new DoubleDataPoint(measurements[j], i * 1.0)); + } + FileNodeManager.getInstance().insert(record, false); + } + FileNodeManager.getInstance().closeAll(); + + FileNodeManager.getInstance().delete(processorName, measurements[5], 50); + FileNodeManager.getInstance().delete(processorName, measurements[4], 40); + FileNodeManager.getInstance().delete(processorName, measurements[3], 30); + + Modification[] realModifications = new Modification[]{ + new Deletion(processorName + "." + measurements[5], 102, 50), + new Deletion(processorName + "." + measurements[4], 103, 40), + new Deletion(processorName + "." + measurements[3], 104, 30), + }; + + String fileNodePath = Directories.getInstance().getTsFileFolder(0) + File.separator + + processorName; + File fileNodeDir = new File(fileNodePath); + File[] modFiles = fileNodeDir.listFiles((dir, name) + -> name.endsWith(ModificationFile.FILE_SUFFIX)); + assertEquals(modFiles.length, 1); + + LocalTextModificationAccessor accessor = + new LocalTextModificationAccessor(modFiles[0].getPath()); + Collection modifications = accessor.read(); + assertEquals(modifications.size(), 3); + int i = 0; + for (Modification modification : modifications) { + assertTrue(modification.equals(realModifications[i++])); + } + } + + @Test + public void testDeleteInOverflowCache() throws FileNodeManagerException { + // insert into BufferWrite + for (int i = 101; i <= 200; i++) { + TSRecord record = new TSRecord(i, processorName); + for (int j = 0; j < 10; j++) { + record.addTuple(new DoubleDataPoint(measurements[j], i * 1.0)); + } + FileNodeManager.getInstance().insert(record, false); + } + FileNodeManager.getInstance().closeAll(); + + // insert into Overflow + for (int i = 1; i <= 100; i++) { + TSRecord record = new TSRecord(i, processorName); + for (int j = 0; j < 10; j++) { + record.addTuple(new DoubleDataPoint(measurements[j], i * 1.0)); + } + FileNodeManager.getInstance().insert(record, false); + } + + FileNodeManager.getInstance().delete(processorName, measurements[3], 50); + FileNodeManager.getInstance().delete(processorName, measurements[4], 50); + FileNodeManager.getInstance().delete(processorName, measurements[5], 30); + FileNodeManager.getInstance().delete(processorName, measurements[5], 50); + + SingleSeriesExpression expression = new SingleSeriesExpression(new Path(processorName, + measurements[5]), null); + QueryDataSource dataSource = FileNodeManager.getInstance().query(expression); + Iterator timeValuePairs = + dataSource.getOverflowSeriesDataSource().getReadableMemChunk().getIterator(); + int count = 0; + while (timeValuePairs.hasNext()) { + timeValuePairs.next(); + count++; + } + assertEquals(count, 50); + } + + @Test + public void testDeleteInOverflowFile() throws FileNodeManagerException, IOException { + // insert into BufferWrite + for (int i = 101; i <= 200; i++) { + TSRecord record = new TSRecord(i, processorName); + for (int j = 0; j < 10; j++) { + record.addTuple(new DoubleDataPoint(measurements[j], i * 1.0)); + } + FileNodeManager.getInstance().insert(record, false); + } + FileNodeManager.getInstance().closeAll(); + + // insert into Overflow + for (int i = 1; i <= 100; i++) { + TSRecord record = new TSRecord(i, processorName); + for (int j = 0; j < 10; j++) { + record.addTuple(new DoubleDataPoint(measurements[j], i * 1.0)); + } + FileNodeManager.getInstance().insert(record, false); + } + FileNodeManager.getInstance().closeAll(); + + FileNodeManager.getInstance().delete(processorName, measurements[5], 50); + FileNodeManager.getInstance().delete(processorName, measurements[4], 40); + FileNodeManager.getInstance().delete(processorName, measurements[3], 30); + + Modification[] realModifications = new Modification[]{ + new Deletion(processorName + "." + measurements[5], 102, 50), + new Deletion(processorName + "." + measurements[4], 103, 40), + new Deletion(processorName + "." + measurements[3], 104, 30), + }; + + String fileNodePath = IoTDBDescriptor.getInstance().getConfig().overflowDataDir + File.separator + + processorName + File.separator + "0" + File.separator; + File fileNodeDir = new File(fileNodePath); + File[] modFiles = fileNodeDir.listFiles((dir, name) + -> name.endsWith(ModificationFile.FILE_SUFFIX)); + assertEquals(modFiles.length, 1); + + LocalTextModificationAccessor accessor = + new LocalTextModificationAccessor(modFiles[0].getPath()); + Collection modifications = accessor.read(); + assertEquals(modifications.size(), 3); + int i = 0; + for (Modification modification : modifications) { + assertTrue(modification.equals(realModifications[i++])); + } + } +} diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/io/LocalTextModificationAccessorTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/io/LocalTextModificationAccessorTest.java index 125d243e76fe..95bec53af9ac 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/io/LocalTextModificationAccessorTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/io/LocalTextModificationAccessorTest.java @@ -67,11 +67,7 @@ public void readMyWrite() { public void readNull() throws IOException { String tempFileName = "mod.temp"; LocalTextModificationAccessor accessor = null; - try { - accessor = new LocalTextModificationAccessor(tempFileName); - } catch (IOException e) { - fail(e.getMessage()); - } + accessor = new LocalTextModificationAccessor(tempFileName); new File(tempFileName).delete(); Collection modifications = accessor.read(); assertNull(modifications); From ba0bed9799137d2fdd174636f3857bf638198db0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=B1=9F=E5=A4=A9?= Date: Sun, 20 Jan 2019 10:52:25 +0800 Subject: [PATCH 05/24] fix BufferWriteProcessor --- .../db/engine/bufferwrite/BufferWriteProcessor.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessor.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessor.java index fd05cd200397..a574b752b54f 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessor.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessor.java @@ -28,6 +28,7 @@ import java.util.concurrent.Future; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.ReentrantLock; + import org.apache.iotdb.db.conf.IoTDBConfig; import org.apache.iotdb.db.conf.IoTDBConstant; import org.apache.iotdb.db.conf.IoTDBDescriptor; @@ -95,8 +96,8 @@ public class BufferWriteProcessor extends Processor { * @throws BufferWriteProcessorException BufferWriteProcessorException */ public BufferWriteProcessor(String baseDir, String processorName, String fileName, - Map parameters, - FileSchema fileSchema) throws BufferWriteProcessorException { + Map parameters, VersionController versionController, + FileSchema fileSchema) throws BufferWriteProcessorException { super(processorName); this.fileSchema = fileSchema; this.baseDir = baseDir; @@ -220,8 +221,8 @@ private void checkMemThreshold4Flush(long addedMemory) throws BufferWriteProcess * @return corresponding chunk data and chunk metadata in memory */ public Pair> queryBufferWriteData(String deviceId, - String measurementId, - TSDataType dataType) { + String measurementId, + TSDataType dataType) { flushQueryLock.lock(); try { MemSeriesLazyMerger memSeriesLazyMerger = new MemSeriesLazyMerger(); From 9cd5d11955130200bf995d2092b15df54f22b9fc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=B1=9F=E5=A4=A9?= Date: Sun, 20 Jan 2019 10:53:57 +0800 Subject: [PATCH 06/24] fix MemTableFlushUtils --- .../apache/iotdb/db/engine/memtable/MemTableFlushUtil.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/MemTableFlushUtil.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/MemTableFlushUtil.java index 80477e017672..f8fe7bc429e2 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/MemTableFlushUtil.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/MemTableFlushUtil.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.List; + import org.apache.iotdb.db.utils.TimeValuePair; import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor; import org.apache.iotdb.tsfile.file.footer.ChunkGroupFooter; @@ -42,7 +43,7 @@ public class MemTableFlushUtil { .getConfig().pageSizeInByte; private static int writeOneSeries(List tvPairs, IChunkWriter seriesWriterImpl, - TSDataType dataType) + TSDataType dataType) throws IOException { int count = 0; switch (dataType) { @@ -96,7 +97,7 @@ private static int writeOneSeries(List tvPairs, IChunkWriter seri * the function for flushing memtable. */ public static void flushMemTable(FileSchema fileSchema, TsFileIOWriter tsFileIoWriter, - IMemTable imemTable) + IMemTable imemTable, long version) throws IOException { for (String deviceId : imemTable.getMemTableMap().keySet()) { long startPos = tsFileIoWriter.getPos(); From 1bf0ddf03bcb72db19194bd66dfa85320e4ef579 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=B1=9F=E5=A4=A9?= Date: Sun, 20 Jan 2019 10:58:56 +0800 Subject: [PATCH 07/24] fix tsfile --- .../file/metadata/ChunkGroupMetaData.java | 19 ++++++++++++++++++- .../tsfile/write/writer/TsFileIOWriter.java | 10 ++++++---- 2 files changed, 24 insertions(+), 5 deletions(-) diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkGroupMetaData.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkGroupMetaData.java index 452174457e19..8aac6c0be973 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkGroupMetaData.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkGroupMetaData.java @@ -26,6 +26,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; + import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -56,6 +57,8 @@ private ChunkGroupMetaData() { chunkMetaDataList = new ArrayList<>(); } + private long version; + /** * constructor of ChunkGroupMetaData. * @@ -83,6 +86,7 @@ public static ChunkGroupMetaData deserializeFrom(InputStream inputStream) throws ChunkGroupMetaData chunkGroupMetaData = new ChunkGroupMetaData(); chunkGroupMetaData.deviceID = ReadWriteIOUtils.readString(inputStream); + chunkGroupMetaData.version = ReadWriteIOUtils.readLong(inputStream); int size = ReadWriteIOUtils.readInt(inputStream); chunkGroupMetaData.serializedSize = @@ -111,6 +115,7 @@ public static ChunkGroupMetaData deserializeFrom(ByteBuffer buffer) throws IOExc ChunkGroupMetaData chunkGroupMetaData = new ChunkGroupMetaData(); chunkGroupMetaData.deviceID = (ReadWriteIOUtils.readString(buffer)); + chunkGroupMetaData.version = ReadWriteIOUtils.readLong(buffer); int size = ReadWriteIOUtils.readInt(buffer); @@ -133,7 +138,8 @@ public int getSerializedSize() { } void reCalculateSerializedSize() { - serializedSize = Integer.BYTES + deviceID.length() + Integer.BYTES; // size of chunkMetaDataList + serializedSize = Integer.BYTES + Long.BYTES + + deviceID.length() + Integer.BYTES; // size of chunkMetaDataList for (ChunkMetaData chunk : chunkMetaDataList) { serializedSize += chunk.getSerializedSize(); } @@ -175,6 +181,7 @@ public String getDeviceID() { public int serializeTo(OutputStream outputStream) throws IOException { int byteLen = 0; byteLen += ReadWriteIOUtils.write(deviceID, outputStream); + byteLen += ReadWriteIOUtils.write(version, outputStream); byteLen += ReadWriteIOUtils.write(chunkMetaDataList.size(), outputStream); for (ChunkMetaData chunkMetaData : chunkMetaDataList) { @@ -195,6 +202,7 @@ public int serializeTo(ByteBuffer buffer) throws IOException { int byteLen = 0; byteLen += ReadWriteIOUtils.write(deviceID, buffer); + byteLen += ReadWriteIOUtils.write(version, buffer); byteLen += ReadWriteIOUtils.write(chunkMetaDataList.size(), buffer); for (ChunkMetaData chunkMetaData : chunkMetaDataList) { @@ -205,4 +213,13 @@ public int serializeTo(ByteBuffer buffer) throws IOException { return byteLen; } + + public long getVersion() { + return version; + } + + public void setVersion(long version) { + this.version = version; + } + } diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/TsFileIOWriter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/TsFileIOWriter.java index 3d3e95f5f2a1..5da1897894e3 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/TsFileIOWriter.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/TsFileIOWriter.java @@ -27,6 +27,7 @@ import java.util.List; import java.util.Map; import java.util.TreeMap; + import org.apache.iotdb.tsfile.common.conf.TSFileConfig; import org.apache.iotdb.tsfile.common.constant.StatisticConstant; import org.apache.iotdb.tsfile.file.MetaMarker; @@ -143,9 +144,9 @@ public void startFlushChunkGroup(String deviceId) throws IOException { * @throws IOException if I/O error occurs */ public int startFlushChunk(MeasurementSchema descriptor, CompressionType compressionCodecName, - TSDataType tsDataType, TSEncoding encodingType, Statistics statistics, long maxTime, - long minTime, - int datasize, int numOfPages) throws IOException { + TSDataType tsDataType, TSEncoding encodingType, Statistics statistics, long maxTime, + long minTime, + int datasize, int numOfPages) throws IOException { LOG.debug("start series chunk:{}, file position {}", descriptor, out.getPosition()); currentChunkMetaData = new ChunkMetaData(descriptor.getMeasurementId(), tsDataType, @@ -192,8 +193,9 @@ public void endChunk(long totalValueCount) { * * @param chunkGroupFooter -use to serialize */ - public void endChunkGroup(ChunkGroupFooter chunkGroupFooter) throws IOException { + public void endChunkGroup(ChunkGroupFooter chunkGroupFooter, long version) throws IOException { chunkGroupFooter.serializeTo(out.wrapAsStream()); + currentChunkGroupMetaData.setVersion(version); chunkGroupMetaDataList.add(currentChunkGroupMetaData); LOG.debug("end chunk group:{}", currentChunkGroupMetaData); currentChunkGroupMetaData = null; From a6dbdbcce83956f7201349596d308b5c6c89f6be Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=B1=9F=E5=A4=A9?= Date: Sun, 20 Jan 2019 11:00:24 +0800 Subject: [PATCH 08/24] fix tsfile writer --- .../main/java/org/apache/iotdb/tsfile/write/TsFileWriter.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/TsFileWriter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/TsFileWriter.java index 93af2f322dbe..3b8e8195e799 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/TsFileWriter.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/TsFileWriter.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.util.HashMap; import java.util.Map; + import org.apache.iotdb.tsfile.common.conf.TSFileConfig; import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor; import org.apache.iotdb.tsfile.exception.write.NoMeasurementException; @@ -73,6 +74,7 @@ public class TsFileWriter { **/ private long recordCountForNextMemCheck = 100; private long chunkGroupSizeThreshold; + private long version = 0; /** * init this TsFileWriter. @@ -261,7 +263,7 @@ private boolean flushAllChunkGroups() throws IOException { chunkGroupFooter.getDataSize(), fileWriter.getPos() - pos)); } - fileWriter.endChunkGroup(chunkGroupFooter); + fileWriter.endChunkGroup(chunkGroupFooter, version++); } long actualTotalChunkGroupSize = fileWriter.getPos() - totalMemStart; LOG.info("total chunk group size:{}", actualTotalChunkGroupSize); From 8af13f1773e50585cc572d6ef2ba151c315a7867 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=B1=9F=E5=A4=A9?= Date: Sun, 20 Jan 2019 13:54:18 +0800 Subject: [PATCH 09/24] multiple fixes --- .../apache/iotdb/db/conf/IoTDBConstant.java | 2 +- .../overflow/ioV2/OverflowProcessor.java | 2 +- .../apache/iotdb/db/monitor/IStatistic.java | 1 + .../apache/iotdb/db/monitor/StatMonitor.java | 6 +-- .../db/qp/executor/OverflowQPExecutor.java | 2 +- .../org/apache/iotdb/db/utils/AuthUtils.java | 2 +- .../writelog/replay/ConcreteLogReplayer.java | 7 ++- .../bufferwrite/BufferWriteBenchmark.java | 7 +-- .../BufferWriteProcessorNewTest.java | 26 ++++++---- .../bufferwrite/BufferWriteProcessorTest.java | 50 ++++++++++--------- .../RestorableTsFileIOWriterTest.java | 4 +- .../BufferwriteFileSizeControlTest.java | 3 +- .../BufferwriteMetaSizeControlTest.java | 5 +- .../OverflowFileSizeControlTest.java | 3 +- .../OverflowMetaSizeControlTest.java | 3 +- .../modification/DeletionFileNodeTest.java | 6 +-- .../ioV2/OverflowProcessorBenchmark.java | 3 +- .../overflow/ioV2/OverflowProcessorTest.java | 18 ++++--- .../overflow/ioV2/OverflowResourceTest.java | 6 ++- .../overflow/ioV2/OverflowSupportTest.java | 4 +- .../SimpleFileVersionControllerTest.java | 47 +++++++++++++++++ .../version/SysTimeVersionControllerTest.java | 33 ++++++++++++ .../file/metadata/ChunkGroupMetaData.java | 4 +- .../tsfile/write/TsFileIOWriterTest.java | 2 +- 24 files changed, 174 insertions(+), 72 deletions(-) create mode 100644 iotdb/src/test/java/org/apache/iotdb/db/engine/version/SimpleFileVersionControllerTest.java create mode 100644 iotdb/src/test/java/org/apache/iotdb/db/engine/version/SysTimeVersionControllerTest.java diff --git a/iotdb/src/main/java/org/apache/iotdb/db/conf/IoTDBConstant.java b/iotdb/src/main/java/org/apache/iotdb/db/conf/IoTDBConstant.java index 3b568720c441..e8933a7a0dbc 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/conf/IoTDBConstant.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/conf/IoTDBConstant.java @@ -47,7 +47,7 @@ public class IoTDBConstant { public static final String OVERFLOW_LOG_NODE_SUFFIX = "-overflow"; public static final String PATH_ROOT = "root"; - public static final char PATH_SEPARATER = '.'; + public static final char PATH_SEPARATOR = '.'; public static final String ADMIN_NAME = "root"; public static final String ADMIN_PW = "root"; public static final String PROFILE_SUFFIX = ".profile"; diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowProcessor.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowProcessor.java index 763fd7eb313b..83c65c9912fe 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowProcessor.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowProcessor.java @@ -99,6 +99,7 @@ public OverflowProcessor(String processorName, Map parameters, throws IOException { super(processorName); this.fileSchema = fileSchema; + this.versionController = versionController; String overflowDirPath = TsFileDBConf.overflowDataDir; if (overflowDirPath.length() > 0 && overflowDirPath.charAt(overflowDirPath.length() - 1) != File.separatorChar) { @@ -122,7 +123,6 @@ public OverflowProcessor(String processorName, Map parameters, processorName + IoTDBConstant.OVERFLOW_LOG_NODE_SUFFIX, getOverflowRestoreFile(), FileNodeManager.getInstance().getRestoreFilePath(processorName)); } - this.versionController = versionController; } private void recovery(File parentFile) throws IOException { diff --git a/iotdb/src/main/java/org/apache/iotdb/db/monitor/IStatistic.java b/iotdb/src/main/java/org/apache/iotdb/db/monitor/IStatistic.java index 8c4338de0b53..7d0414495fd2 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/monitor/IStatistic.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/monitor/IStatistic.java @@ -22,6 +22,7 @@ import java.util.HashMap; import java.util.List; import java.util.concurrent.atomic.AtomicLong; + import org.apache.iotdb.tsfile.write.record.TSRecord; public interface IStatistic { diff --git a/iotdb/src/main/java/org/apache/iotdb/db/monitor/StatMonitor.java b/iotdb/src/main/java/org/apache/iotdb/db/monitor/StatMonitor.java index a3497e79c634..a6d3cefb5ebc 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/monitor/StatMonitor.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/monitor/StatMonitor.java @@ -26,6 +26,7 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; + import org.apache.iotdb.db.concurrent.IoTDBThreadPoolFactory; import org.apache.iotdb.db.concurrent.ThreadName; import org.apache.iotdb.db.conf.IoTDBConfig; @@ -38,7 +39,6 @@ import org.apache.iotdb.db.metadata.MManager; import org.apache.iotdb.db.service.IService; import org.apache.iotdb.db.service.ServiceType; -import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; import org.apache.iotdb.tsfile.write.record.TSRecord; import org.apache.iotdb.tsfile.write.record.datapoint.DataPoint; import org.apache.iotdb.tsfile.write.record.datapoint.LongDataPoint; @@ -99,7 +99,7 @@ public static StatMonitor getInstance() { * @return TSRecord contains the DataPoints of a statGroupDeltaName */ public static TSRecord convertToTSRecord(HashMap hashMap, - String statGroupDeltaName, long curTime) { + String statGroupDeltaName, long curTime) { TSRecord tsRecord = new TSRecord(curTime, statGroupDeltaName); tsRecord.dataPointList = new ArrayList() { { @@ -360,7 +360,7 @@ public void run() { for (Map.Entry entry : statisticMap.entrySet()) { for (String statParamName : entry.getValue().getStatParamsHashMap().keySet()) { fManager.delete(entry.getKey(), statParamName, - currentTimeMillis - statMonitorRetainIntervalSec * 1000, TSDataType.INT64); + currentTimeMillis - statMonitorRetainIntervalSec * 1000); } } } catch (FileNodeManagerException e) { diff --git a/iotdb/src/main/java/org/apache/iotdb/db/qp/executor/OverflowQPExecutor.java b/iotdb/src/main/java/org/apache/iotdb/db/qp/executor/OverflowQPExecutor.java index 967e05703669..6cdb7e1b0892 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/qp/executor/OverflowQPExecutor.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/qp/executor/OverflowQPExecutor.java @@ -227,7 +227,7 @@ protected boolean delete(Path path, long timestamp) throws ProcessorException { } mManager.getFileNameByPath(path.getFullPath()); TSDataType type = mManager.getSeriesType(path.getFullPath()); - fileNodeManager.delete(deviceId, measurementId, timestamp, type); + fileNodeManager.delete(deviceId, measurementId, timestamp); return true; } catch (PathErrorException e) { throw new ProcessorException(e.getMessage()); diff --git a/iotdb/src/main/java/org/apache/iotdb/db/utils/AuthUtils.java b/iotdb/src/main/java/org/apache/iotdb/db/utils/AuthUtils.java index 3f7de77a15de..8479ac87d7b4 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/utils/AuthUtils.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/utils/AuthUtils.java @@ -168,7 +168,7 @@ public static String encryptPassword(String password) { public static boolean pathBelongsTo(String pathA, String pathB) { return pathA.equals(pathB) || (pathA.startsWith(pathB) - && pathA.charAt(pathB.length()) == IoTDBConstant.PATH_SEPARATER); + && pathA.charAt(pathB.length()) == IoTDBConstant.PATH_SEPARATOR); } /** diff --git a/iotdb/src/main/java/org/apache/iotdb/db/writelog/replay/ConcreteLogReplayer.java b/iotdb/src/main/java/org/apache/iotdb/db/writelog/replay/ConcreteLogReplayer.java index d63e731070c6..96366d961dd9 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/writelog/replay/ConcreteLogReplayer.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/writelog/replay/ConcreteLogReplayer.java @@ -20,6 +20,7 @@ package org.apache.iotdb.db.writelog.replay; import java.util.List; + import org.apache.iotdb.db.engine.filenode.FileNodeManager; import org.apache.iotdb.db.exception.FileNodeManagerException; import org.apache.iotdb.db.exception.PathErrorException; @@ -88,12 +89,10 @@ private void update(UpdatePlan updatePlan) throws FileNodeManagerException, Path } } - private void delete(DeletePlan deletePlan) throws FileNodeManagerException, PathErrorException { - MManager memManager = MManager.getInstance(); + private void delete(DeletePlan deletePlan) throws FileNodeManagerException { for (Path path : deletePlan.getPaths()) { FileNodeManager.getInstance() - .delete(path.getDevice(), path.getMeasurement(), deletePlan.getDeleteTime(), - memManager.getSeriesType(path.getFullPath())); + .delete(path.getDevice(), path.getMeasurement(), deletePlan.getDeleteTime()); } } } diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteBenchmark.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteBenchmark.java index d5d983e3088c..d77c1250ec53 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteBenchmark.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteBenchmark.java @@ -24,6 +24,7 @@ import java.util.HashMap; import java.util.Map; import org.apache.commons.io.FileUtils; +import org.apache.iotdb.db.engine.version.SysTimeVersionController; import org.apache.iotdb.db.exception.BufferWriteProcessorException; import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; @@ -93,9 +94,9 @@ public void act() throws Exception { } }); - BufferWriteProcessor bufferWriteProcessor = new BufferWriteProcessor("BufferBenchmark", "bench", - "benchFile", - parameters, fileSchema); + BufferWriteProcessor bufferWriteProcessor = new BufferWriteProcessor("BufferBenchmark", + "bench", "benchFile", + parameters, SysTimeVersionController.INSTANCE, fileSchema); long startTime = System.currentTimeMillis(); for (int i = 0; i < numOfPoint; i++) { diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessorNewTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessorNewTest.java index 49283851d0d0..e6a678bdaf0b 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessorNewTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessorNewTest.java @@ -1,6 +1,6 @@ /** * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) - * + *

* 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 @@ -8,9 +8,9 @@ * 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. @@ -28,9 +28,11 @@ import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; + import org.apache.iotdb.db.conf.directories.Directories; import org.apache.iotdb.db.engine.MetadataManagerHelper; import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk; +import org.apache.iotdb.db.engine.version.SysTimeVersionController; import org.apache.iotdb.db.exception.BufferWriteProcessorException; import org.apache.iotdb.db.utils.EnvironmentUtils; import org.apache.iotdb.db.utils.FileSchemaUtils; @@ -91,18 +93,19 @@ public void tearDown() throws Exception { @Test public void testWriteAndFlush() - throws BufferWriteProcessorException, WriteProcessException, IOException, InterruptedException { + throws BufferWriteProcessorException, WriteProcessException, IOException, InterruptedException { bufferwrite = new BufferWriteProcessor(Directories.getInstance().getFolderForTest(), - processorName, filename, - parameters, FileSchemaUtils.constructFileSchema(processorName)); + processorName, filename, + parameters, SysTimeVersionController.INSTANCE, + FileSchemaUtils.constructFileSchema(processorName)); assertEquals(filename, bufferwrite.getFileName()); assertEquals(processorName + File.separator + filename, bufferwrite.getFileRelativePath()); assertEquals(true, bufferwrite.isNewProcessor()); bufferwrite.setNewProcessor(false); assertEquals(false, bufferwrite.isNewProcessor()); Pair> pair = bufferwrite - .queryBufferWriteData(processorName, - measurementId, dataType); + .queryBufferWriteData(processorName, + measurementId, dataType); ReadOnlyMemChunk left = pair.left; List right = pair.right; assertEquals(true, left.isEmpty()); @@ -142,8 +145,9 @@ public void testWriteAndFlush() // test recovery BufferWriteProcessor bufferWriteProcessor = new BufferWriteProcessor( - Directories.getInstance().getFolderForTest(), processorName, filename, parameters, - FileSchemaUtils.constructFileSchema(processorName)); + Directories.getInstance().getFolderForTest(), processorName, filename, parameters, + SysTimeVersionController.INSTANCE, + FileSchemaUtils.constructFileSchema(processorName)); pair = bufferWriteProcessor.queryBufferWriteData(processorName, measurementId, dataType); left = pair.left; right = pair.right; diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessorTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessorTest.java index 6c7f53db34c3..b78d78feaf70 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessorTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessorTest.java @@ -1,6 +1,6 @@ /** * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) - * + *

* 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 @@ -8,9 +8,9 @@ * 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. @@ -29,10 +29,12 @@ import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; + import org.apache.iotdb.db.conf.directories.Directories; import org.apache.iotdb.db.engine.MetadataManagerHelper; import org.apache.iotdb.db.engine.PathUtils; import org.apache.iotdb.db.engine.querycontext.ReadOnlyMemChunk; +import org.apache.iotdb.db.engine.version.SysTimeVersionController; import org.apache.iotdb.db.exception.ProcessorException; import org.apache.iotdb.db.utils.EnvironmentUtils; import org.apache.iotdb.db.utils.FileSchemaUtils; @@ -110,10 +112,10 @@ public void tearDown() throws Exception { @Test public void testWriteAndAbnormalRecover() - throws WriteProcessException, InterruptedException, IOException, ProcessorException { + throws WriteProcessException, InterruptedException, IOException, ProcessorException { bufferwrite = new BufferWriteProcessor(directories.getFolderForTest(), deviceId, insertPath, - parameters, - FileSchemaUtils.constructFileSchema(deviceId)); + parameters, SysTimeVersionController.INSTANCE, + FileSchemaUtils.constructFileSchema(deviceId)); for (int i = 1; i < 100; i++) { bufferwrite.write(deviceId, measurementId, i, dataType, String.valueOf(i)); } @@ -136,13 +138,14 @@ public void testWriteAndAbnormalRecover() bufferwrite.close(); file.renameTo(restoreFile); BufferWriteProcessor bufferWriteProcessor = new BufferWriteProcessor( - directories.getFolderForTest(), deviceId, - insertPath, parameters, FileSchemaUtils.constructFileSchema(deviceId)); + directories.getFolderForTest(), deviceId, + insertPath, parameters, SysTimeVersionController.INSTANCE, + FileSchemaUtils.constructFileSchema(deviceId)); assertEquals(true, insertFile.exists()); assertEquals(insertFileLength, insertFile.length()); Pair> pair = bufferWriteProcessor - .queryBufferWriteData(deviceId, - measurementId, dataType); + .queryBufferWriteData(deviceId, + measurementId, dataType); assertEquals(true, pair.left.isEmpty()); assertEquals(1, pair.right.size()); ChunkMetaData chunkMetaData = pair.right.get(0); @@ -154,10 +157,10 @@ public void testWriteAndAbnormalRecover() @Test public void testWriteAndNormalRecover() - throws WriteProcessException, ProcessorException, InterruptedException { + throws WriteProcessException, ProcessorException, InterruptedException { bufferwrite = new BufferWriteProcessor(directories.getFolderForTest(), deviceId, insertPath, - parameters, - FileSchemaUtils.constructFileSchema(deviceId)); + parameters, SysTimeVersionController.INSTANCE, + FileSchemaUtils.constructFileSchema(deviceId)); for (int i = 1; i < 100; i++) { bufferwrite.write(deviceId, measurementId, i, dataType, String.valueOf(i)); } @@ -169,11 +172,12 @@ public void testWriteAndNormalRecover() File restoreFile = new File(dataFile, restoreFilePath); assertEquals(true, restoreFile.exists()); BufferWriteProcessor bufferWriteProcessor = new BufferWriteProcessor( - directories.getFolderForTest(), deviceId, - insertPath, parameters, FileSchemaUtils.constructFileSchema(deviceId)); + directories.getFolderForTest(), deviceId, + insertPath, parameters, SysTimeVersionController.INSTANCE, + FileSchemaUtils.constructFileSchema(deviceId)); Pair> pair = bufferWriteProcessor - .queryBufferWriteData(deviceId, - measurementId, dataType); + .queryBufferWriteData(deviceId, + measurementId, dataType); assertEquals(true, pair.left.isEmpty()); assertEquals(1, pair.right.size()); ChunkMetaData chunkMetaData = pair.right.get(0); @@ -186,10 +190,10 @@ public void testWriteAndNormalRecover() @Test public void testWriteAndQuery() - throws WriteProcessException, InterruptedException, ProcessorException { + throws WriteProcessException, InterruptedException, ProcessorException { bufferwrite = new BufferWriteProcessor(directories.getFolderForTest(), deviceId, insertPath, - parameters, - FileSchemaUtils.constructFileSchema(deviceId)); + parameters, SysTimeVersionController.INSTANCE, + FileSchemaUtils.constructFileSchema(deviceId)); assertEquals(false, bufferwrite.isFlush()); assertEquals(true, bufferwrite.canBeClosed()); assertEquals(0, bufferwrite.memoryUsage()); @@ -207,8 +211,8 @@ public void testWriteAndQuery() assertEquals(0, bufferwrite.memoryUsage()); // query result Pair> pair = bufferwrite - .queryBufferWriteData(deviceId, measurementId, - dataType); + .queryBufferWriteData(deviceId, measurementId, + dataType); assertEquals(true, pair.left.isEmpty()); assertEquals(1, pair.right.size()); ChunkMetaData chunkMetaData = pair.right.get(0); diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/bufferwrite/RestorableTsFileIOWriterTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/bufferwrite/RestorableTsFileIOWriterTest.java index a267a41088cf..d12de5277119 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/engine/bufferwrite/RestorableTsFileIOWriterTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/bufferwrite/RestorableTsFileIOWriterTest.java @@ -165,7 +165,7 @@ public void testWriteAndRecover() throws IOException { memTable.write("d1", "s2", TSDataType.INT32, 3, "1"); memTable.write("d2", "s2", TSDataType.INT32, 2, "1"); memTable.write("d2", "s2", TSDataType.INT32, 4, "1"); - MemTableFlushUtil.flushMemTable(schema, writer, memTable); + MemTableFlushUtil.flushMemTable(schema, writer, memTable, 0); writer.flush(); writer.appendMetadata(); writer.getOutput().close(); @@ -218,7 +218,7 @@ public void testFlushAndGetMetadata() throws IOException { MemTableTestUtils.measurementId0, MemTableTestUtils.dataType0); - MemTableFlushUtil.flushMemTable(MemTableTestUtils.getFileSchema(), writer, memTable); + MemTableFlushUtil.flushMemTable(MemTableTestUtils.getFileSchema(), writer, memTable, 0); writer.flush(); assertEquals(0, diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/memcontrol/BufferwriteFileSizeControlTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/memcontrol/BufferwriteFileSizeControlTest.java index be9bdaaa59ee..2da46fc3a5ab 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/engine/memcontrol/BufferwriteFileSizeControlTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/memcontrol/BufferwriteFileSizeControlTest.java @@ -36,6 +36,7 @@ import org.apache.iotdb.db.engine.bufferwrite.Action; import org.apache.iotdb.db.engine.bufferwrite.BufferWriteProcessor; import org.apache.iotdb.db.engine.bufferwrite.FileNodeConstants; +import org.apache.iotdb.db.engine.version.SysTimeVersionController; import org.apache.iotdb.db.exception.BufferWriteProcessorException; import org.apache.iotdb.db.metadata.ColumnSchema; import org.apache.iotdb.db.metadata.MManager; @@ -143,7 +144,7 @@ public void test() throws BufferWriteProcessorException, WriteProcessException { try { processor = new BufferWriteProcessor(Directories.getInstance().getFolderForTest(), nsp, filename, - parameters, constructFileSchema(nsp)); + parameters, SysTimeVersionController.INSTANCE, constructFileSchema(nsp)); } catch (BufferWriteProcessorException e) { e.printStackTrace(); fail(e.getMessage()); diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/memcontrol/BufferwriteMetaSizeControlTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/memcontrol/BufferwriteMetaSizeControlTest.java index c9bef10c932c..fa84019e0dbc 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/engine/memcontrol/BufferwriteMetaSizeControlTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/memcontrol/BufferwriteMetaSizeControlTest.java @@ -36,6 +36,7 @@ import org.apache.iotdb.db.engine.bufferwrite.Action; import org.apache.iotdb.db.engine.bufferwrite.BufferWriteProcessor; import org.apache.iotdb.db.engine.bufferwrite.FileNodeConstants; +import org.apache.iotdb.db.engine.version.SysTimeVersionController; import org.apache.iotdb.db.exception.BufferWriteProcessorException; import org.apache.iotdb.db.metadata.ColumnSchema; import org.apache.iotdb.db.metadata.MManager; @@ -142,8 +143,8 @@ public void test() throws BufferWriteProcessorException, WriteProcessException { try { processor = new BufferWriteProcessor(Directories.getInstance().getFolderForTest(), nsp, - filename, - parameters, constructFileSchema(nsp)); + filename, + parameters, SysTimeVersionController.INSTANCE, constructFileSchema(nsp)); } catch (BufferWriteProcessorException e) { e.printStackTrace(); fail(e.getMessage()); diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/memcontrol/OverflowFileSizeControlTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/memcontrol/OverflowFileSizeControlTest.java index f18dd12f99ce..09bdce981dee 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/engine/memcontrol/OverflowFileSizeControlTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/memcontrol/OverflowFileSizeControlTest.java @@ -31,6 +31,7 @@ import org.apache.iotdb.db.engine.bufferwrite.Action; import org.apache.iotdb.db.engine.bufferwrite.FileNodeConstants; import org.apache.iotdb.db.engine.overflow.ioV2.OverflowProcessor; +import org.apache.iotdb.db.engine.version.SysTimeVersionController; import org.apache.iotdb.db.exception.OverflowProcessorException; import org.apache.iotdb.db.utils.EnvironmentUtils; import org.apache.iotdb.db.utils.FileSchemaUtils; @@ -120,7 +121,7 @@ public void testInsert() throws InterruptedException, IOException, WriteProcessE // insert one point: int try { ofprocessor = new OverflowProcessor(nameSpacePath, parameters, - FileSchemaUtils.constructFileSchema(deviceId)); + FileSchemaUtils.constructFileSchema(deviceId), SysTimeVersionController.INSTANCE); for (int i = 1; i < 1000000; i++) { TSRecord record = new TSRecord(i, deviceId); record.addTuple(DataPoint.getDataPoint(dataTypes[0], measurementIds[0], String.valueOf(i))); diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/memcontrol/OverflowMetaSizeControlTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/memcontrol/OverflowMetaSizeControlTest.java index 6feeb94f8f2d..05f458fc0234 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/engine/memcontrol/OverflowMetaSizeControlTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/memcontrol/OverflowMetaSizeControlTest.java @@ -31,6 +31,7 @@ import org.apache.iotdb.db.engine.bufferwrite.Action; import org.apache.iotdb.db.engine.bufferwrite.FileNodeConstants; import org.apache.iotdb.db.engine.overflow.ioV2.OverflowProcessor; +import org.apache.iotdb.db.engine.version.SysTimeVersionController; import org.apache.iotdb.db.exception.OverflowProcessorException; import org.apache.iotdb.db.utils.EnvironmentUtils; import org.apache.iotdb.db.utils.FileSchemaUtils; @@ -120,7 +121,7 @@ public void testInsert() throws InterruptedException, IOException, WriteProcessE // insert one point: int try { ofprocessor = new OverflowProcessor(nameSpacePath, parameters, - FileSchemaUtils.constructFileSchema(deviceId)); + FileSchemaUtils.constructFileSchema(deviceId), SysTimeVersionController.INSTANCE); for (int i = 1; i < 1000000; i++) { TSRecord record = new TSRecord(i, deviceId); record.addTuple(DataPoint.getDataPoint(dataTypes[0], measurementIds[0], String.valueOf(i))); diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java index 1cfa9467ffe4..8b833d031e25 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java @@ -211,9 +211,9 @@ public void testDeleteInOverflowFile() throws FileNodeManagerException, IOExcept FileNodeManager.getInstance().delete(processorName, measurements[3], 30); Modification[] realModifications = new Modification[]{ - new Deletion(processorName + "." + measurements[5], 102, 50), - new Deletion(processorName + "." + measurements[4], 103, 40), - new Deletion(processorName + "." + measurements[3], 104, 30), + new Deletion(processorName + "." + measurements[5], 103, 50), + new Deletion(processorName + "." + measurements[4], 104, 40), + new Deletion(processorName + "." + measurements[3], 105, 30), }; String fileNodePath = IoTDBDescriptor.getInstance().getConfig().overflowDataDir + File.separator diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowProcessorBenchmark.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowProcessorBenchmark.java index 4b05859d9cb9..c05433a9b4e0 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowProcessorBenchmark.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowProcessorBenchmark.java @@ -28,6 +28,7 @@ import org.apache.iotdb.db.conf.IoTDBDescriptor; import org.apache.iotdb.db.engine.bufferwrite.Action; import org.apache.iotdb.db.engine.bufferwrite.FileNodeConstants; +import org.apache.iotdb.db.engine.version.SysTimeVersionController; import org.apache.iotdb.db.exception.OverflowProcessorException; import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; @@ -92,7 +93,7 @@ public void act() throws Exception { } }); OverflowProcessor overflowProcessor = new OverflowProcessor("Overflow_bench", parameters, - fileSchema); + fileSchema, SysTimeVersionController.INSTANCE); long startTime = System.currentTimeMillis(); for (int i = 0; i < numOfPoint; i++) { for (int j = 0; j < numOfDevice; j++) { diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowProcessorTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowProcessorTest.java index fe7e2caf7984..6e2f33b18a60 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowProcessorTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowProcessorTest.java @@ -32,6 +32,7 @@ import org.apache.iotdb.db.engine.bufferwrite.FileNodeConstants; import org.apache.iotdb.db.engine.querycontext.MergeSeriesDataSource; import org.apache.iotdb.db.engine.querycontext.OverflowSeriesDataSource; +import org.apache.iotdb.db.engine.version.SysTimeVersionController; import org.apache.iotdb.db.exception.OverflowProcessorException; import org.apache.iotdb.db.utils.EnvironmentUtils; import org.apache.iotdb.db.utils.TimeValuePair; @@ -76,8 +77,10 @@ public void tearDown() throws Exception { @Test public void testInsertUpdate() throws IOException, OverflowProcessorException, InterruptedException { - processor = new OverflowProcessor(processorName, parameters, OverflowTestUtils.getFileSchema()); - assertEquals(true, new File(PathUtils.getOverflowWriteDir(processorName), "0").exists()); + processor = new OverflowProcessor(processorName, parameters, OverflowTestUtils.getFileSchema(), + SysTimeVersionController.INSTANCE); + assertEquals(true, new File(PathUtils.getOverflowWriteDir(processorName), + "0").exists()); assertEquals(false, processor.isFlush()); assertEquals(false, processor.isMerge()); // write update data @@ -141,7 +144,8 @@ public void testInsertUpdate() @Test public void testWriteMemoryAndQuery() throws IOException, OverflowProcessorException { - processor = new OverflowProcessor(processorName, parameters, OverflowTestUtils.getFileSchema()); + processor = new OverflowProcessor(processorName, parameters, OverflowTestUtils.getFileSchema(), + SysTimeVersionController.INSTANCE); OverflowTestUtils.produceInsertData(processor); processor.close(); // test query @@ -155,7 +159,8 @@ public void testWriteMemoryAndQuery() throws IOException, OverflowProcessorExcep @Test public void testFlushAndQuery() throws IOException, OverflowProcessorException { - processor = new OverflowProcessor(processorName, parameters, OverflowTestUtils.getFileSchema()); + processor = new OverflowProcessor(processorName, parameters, OverflowTestUtils.getFileSchema(), + SysTimeVersionController.INSTANCE); processor.flush(); // waiting for the end of flush. try { @@ -173,13 +178,14 @@ public void testFlushAndQuery() throws IOException, OverflowProcessorException { @Test public void testRecovery() throws OverflowProcessorException, IOException { - processor = new OverflowProcessor(processorName, parameters, OverflowTestUtils.getFileSchema()); + processor = new OverflowProcessor(processorName, parameters, OverflowTestUtils.getFileSchema(), + SysTimeVersionController.INSTANCE); processor.close(); processor.switchWorkToMerge(); assertEquals(true, processor.isMerge()); processor.clear(); OverflowProcessor overflowProcessor = new OverflowProcessor(processorName, parameters, - OverflowTestUtils.getFileSchema()); + OverflowTestUtils.getFileSchema(), SysTimeVersionController.INSTANCE); // recovery query assertEquals(false, overflowProcessor.isMerge()); overflowProcessor.switchWorkToMerge(); diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowResourceTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowResourceTest.java index a4e21b504271..e05677e30fb7 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowResourceTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowResourceTest.java @@ -25,6 +25,8 @@ import java.io.FileOutputStream; import java.io.IOException; import java.util.List; + +import org.apache.iotdb.db.engine.version.SysTimeVersionController; import org.apache.iotdb.db.utils.EnvironmentUtils; import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData; import org.junit.After; @@ -46,7 +48,7 @@ public class OverflowResourceTest { @Before public void setUp() throws Exception { - work = new OverflowResource(filePath, dataPath); + work = new OverflowResource(filePath, dataPath, SysTimeVersionController.INSTANCE); insertFile = new File(new File(filePath, dataPath), insertFileName); updateFile = new File(new File(filePath, dataPath), updateDeleteFileName); positionFile = new File(new File(filePath, dataPath), positionFileName); @@ -82,7 +84,7 @@ public void testOverflowInsert() throws IOException { fileOutputStream.write(new byte[20]); fileOutputStream.close(); assertEquals(originlength + 20, insertFile.length()); - work = new OverflowResource(filePath, dataPath); + work = new OverflowResource(filePath, dataPath, SysTimeVersionController.INSTANCE); chunkMetaDatas = work .getInsertMetadatas(OverflowTestUtils.deviceId1, OverflowTestUtils.measurementId1, OverflowTestUtils.dataType1); diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowSupportTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowSupportTest.java index b24bab37d5ec..d0741c3f9158 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowSupportTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowSupportTest.java @@ -49,7 +49,7 @@ public void setUp() throws Exception { support.update(deviceId1, measurementId1, 20, 30, dataType1, BytesUtils.intToBytes(20)); // time :[2,10] [20,30] value: int [10,10] int[20,20] // d1 s2 - support.delete(deviceId1, measurementId2, 10, dataType1); + support.delete(deviceId1, measurementId2, 10, false); support.update(deviceId1, measurementId2, 20, 30, dataType1, BytesUtils.intToBytes(20)); // time: [0,-10] [20,30] value[20,20] // d2 s1 @@ -58,7 +58,7 @@ public void setUp() throws Exception { // time: [5,9] [10,40] value [10.5,10.5] [20.5,20.5] // d2 s2 support.update(deviceId2, measurementId2, 2, 10, dataType2, BytesUtils.floatToBytes(5.5f)); - support.delete(deviceId2, measurementId2, 20, dataType2); + support.delete(deviceId2, measurementId2, 20, false); // time : [0,-20] } diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/version/SimpleFileVersionControllerTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/version/SimpleFileVersionControllerTest.java new file mode 100644 index 000000000000..b33b5065a500 --- /dev/null +++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/version/SimpleFileVersionControllerTest.java @@ -0,0 +1,47 @@ +/** + * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) + *

+ * Licensed 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.version; + +import java.io.File; +import java.io.IOException; + +import org.apache.commons.io.FileUtils; +import org.junit.Test; + +import static org.apache.iotdb.db.engine.version.SimpleFileVersionController.SAVE_INTERVAL; +import static org.junit.Assert.assertEquals; + +public class SimpleFileVersionControllerTest { + @Test + public void test() throws IOException { + String tempFilePath = "version.tmp"; + + try { + new File(tempFilePath).mkdir(); + VersionController versionController = new SimpleFileVersionController(tempFilePath); + assertEquals(versionController.currVersion(), SAVE_INTERVAL); + for (int i = 0; i < 150; i++) { + versionController.nextVersion(); + } + assertEquals(versionController.currVersion(), SAVE_INTERVAL + 150); + versionController = new SimpleFileVersionController(tempFilePath); + assertEquals(versionController.currVersion(), SAVE_INTERVAL + 200); + } finally { + FileUtils.deleteDirectory(new File(tempFilePath)); + } + } +} \ No newline at end of file diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/version/SysTimeVersionControllerTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/version/SysTimeVersionControllerTest.java new file mode 100644 index 000000000000..c7214bada033 --- /dev/null +++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/version/SysTimeVersionControllerTest.java @@ -0,0 +1,33 @@ +/** + * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) + *

+ * Licensed 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.version; + +import org.junit.Test; + +import static org.junit.Assert.assertTrue; + +public class SysTimeVersionControllerTest { + + @Test + public void test() { + VersionController versionController = SysTimeVersionController.INSTANCE; + long diff = versionController.currVersion() - System.currentTimeMillis(); + assertTrue(diff >= -1 && diff <= 1); + diff = versionController.nextVersion() - System.currentTimeMillis(); + assertTrue(diff >= -1 && diff <= 1); + } +} \ No newline at end of file diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkGroupMetaData.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkGroupMetaData.java index 8aac6c0be973..a5ca9abd6121 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkGroupMetaData.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkGroupMetaData.java @@ -90,7 +90,7 @@ public static ChunkGroupMetaData deserializeFrom(InputStream inputStream) throws int size = ReadWriteIOUtils.readInt(inputStream); chunkGroupMetaData.serializedSize = - Integer.BYTES + chunkGroupMetaData.deviceID.length() + Integer.BYTES; + Integer.BYTES + Long.BYTES + chunkGroupMetaData.deviceID.length() + Integer.BYTES; List chunkMetaDataList = new ArrayList<>(); @@ -120,7 +120,7 @@ public static ChunkGroupMetaData deserializeFrom(ByteBuffer buffer) throws IOExc int size = ReadWriteIOUtils.readInt(buffer); chunkGroupMetaData.serializedSize = - Integer.BYTES + chunkGroupMetaData.deviceID.length() + Integer.BYTES; + Integer.BYTES + Long.BYTES + chunkGroupMetaData.deviceID.length() + Integer.BYTES; List chunkMetaDataList = new ArrayList<>(); for (int i = 0; i < size; i++) { diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/write/TsFileIOWriterTest.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/write/TsFileIOWriterTest.java index b52e42dd2bf1..a0a335d92087 100644 --- a/tsfile/src/test/java/org/apache/iotdb/tsfile/write/TsFileIOWriterTest.java +++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/write/TsFileIOWriterTest.java @@ -62,7 +62,7 @@ public void before() throws IOException { measurementSchema.getType(), measurementSchema.getEncodingType(), statistics, 0, 0, 0, 0); writer.endChunk(0); ChunkGroupFooter footer = new ChunkGroupFooter(deviceId, 0, 1); - writer.endChunkGroup(footer); + writer.endChunkGroup(footer, 0); // end file writer.endFile(fileSchema); From fb5802d49d9aeaa3ab20c15a6bb75a89c02164f0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=B1=9F=E5=A4=A9?= Date: Sun, 20 Jan 2019 14:21:53 +0800 Subject: [PATCH 10/24] revert the delete order: disk first then memory --- .../iotdb/db/engine/filenode/FileNodeProcessor.java | 12 ++++++------ .../db/engine/overflow/ioV2/OverflowProcessor.java | 4 ++-- 2 files changed, 8 insertions(+), 8 deletions(-) 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 6a4eb28fda28..47fa2e97284e 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 @@ -1976,12 +1976,6 @@ public String getFileNodeRestoreFilePath() { public void delete(String deviceId, String measurementId, long timestamp) throws IOException { // TODO: how to avoid partial deletion? long version = versionController.nextVersion(); - // delete data in memory - if (bufferWriteProcessor != null) { - bufferWriteProcessor.delete(deviceId, measurementId, timestamp); - } - OverflowProcessor overflowProcessor = getOverflowProcessor(getProcessorName()); - overflowProcessor.delete(deviceId, measurementId, timestamp, version); String fullPath = deviceId + IoTDBConstant.PATH_SEPARATOR + measurementId; @@ -1994,5 +1988,11 @@ public void delete(String deviceId, String measurementId, long timestamp) throws fileNode.getModFile().write(deletion); } } + // delete data in memory + OverflowProcessor overflowProcessor = getOverflowProcessor(getProcessorName()); + overflowProcessor.delete(deviceId, measurementId, timestamp, version); + if (bufferWriteProcessor != null) { + bufferWriteProcessor.delete(deviceId, measurementId, timestamp); + } } } \ No newline at end of file diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowProcessor.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowProcessor.java index 83c65c9912fe..d834ffa93c40 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowProcessor.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowProcessor.java @@ -243,11 +243,11 @@ private byte[] convertStringToBytes(TSDataType type, String o) { * @param version the version number of this deletion. */ public void delete(String deviceId, String measurementId, long timestamp, long version) throws IOException { - workSupport.delete(deviceId, measurementId, timestamp, false); workResource.delete(deviceId, measurementId, timestamp, version); + workSupport.delete(deviceId, measurementId, timestamp, false); if (flushStatus.isFlushing()) { - flushSupport.delete(deviceId, measurementId, timestamp, true); mergeResource.delete(deviceId, measurementId, timestamp, version); + flushSupport.delete(deviceId, measurementId, timestamp, true); } } From dbcf2edf038694dedb9f67c2bef4b64a72440d68 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=B1=9F=E5=A4=A9?= Date: Sun, 20 Jan 2019 15:14:38 +0800 Subject: [PATCH 11/24] fix the header error in package-info of modification --- .../org/apache/iotdb/db/engine/modification/package-info.java | 1 - 1 file changed, 1 deletion(-) diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/package-info.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/package-info.java index 49697e1696c7..4bf8677242bd 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/package-info.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/package-info.java @@ -13,7 +13,6 @@ * See the License for the specific language governing permissions and * limitations under the License. *

- * modification is the functional module responsible for processing UPDATE and DELETE. */ /** From 38924fb67c001c9b9267f272840c3db396eeb5f9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=B1=9F=E5=A4=A9?= Date: Sun, 20 Jan 2019 18:24:36 +0800 Subject: [PATCH 12/24] combine modification with query --- .../db/engine/filenode/FileNodeProcessor.java | 17 ++++ .../overflow/ioV2/OverflowResource.java | 12 +++ .../reader/sequence/SealedTsFilesReader.java | 21 +++-- .../org/apache/iotdb/db/utils/QueryUtils.java | 84 +++++++++++++++++++ .../tsfile/file/metadata/ChunkMetaData.java | 25 ++++-- .../iotdb/tsfile/read/common/Chunk.java | 9 ++ .../read/controller/ChunkLoaderImpl.java | 4 +- .../controller/MetadataQuerierByFileImpl.java | 1 + .../tsfile/read/reader/chunk/ChunkReader.java | 20 ++--- .../reader/chunk/ChunkReaderWithFilter.java | 2 +- .../chunk/ChunkReaderWithoutFilter.java | 2 +- .../tsfile/read/reader/page/PageReader.java | 54 +++++++++--- .../read/reader/series/FileSeriesReader.java | 5 +- .../series/FileSeriesReaderWithFilter.java | 1 - .../series/FileSeriesReaderWithoutFilter.java | 1 - 15 files changed, 214 insertions(+), 44 deletions(-) create mode 100644 iotdb/src/main/java/org/apache/iotdb/db/utils/QueryUtils.java 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 47fa2e97284e..b2aacee46b3e 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 @@ -26,6 +26,7 @@ import java.time.Instant; import java.time.ZonedDateTime; import java.util.ArrayList; +import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -48,6 +49,7 @@ import org.apache.iotdb.db.engine.bufferwrite.BufferWriteProcessor; import org.apache.iotdb.db.engine.bufferwrite.FileNodeConstants; import org.apache.iotdb.db.engine.modification.Deletion; +import org.apache.iotdb.db.engine.modification.Modification; import org.apache.iotdb.db.engine.overflow.ioV2.OverflowProcessor; import org.apache.iotdb.db.engine.pool.MergeManager; import org.apache.iotdb.db.engine.querycontext.GlobalSortedSeriesDataSource; @@ -70,7 +72,9 @@ import org.apache.iotdb.db.monitor.StatMonitor; import org.apache.iotdb.db.query.factory.SeriesReaderFactory; import org.apache.iotdb.db.query.reader.IReader; +import org.apache.iotdb.db.service.IoTDB; import org.apache.iotdb.db.utils.MemUtils; +import org.apache.iotdb.db.utils.QueryUtils; import org.apache.iotdb.db.utils.TimeValuePair; import org.apache.iotdb.tsfile.common.conf.TSFileConfig; import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor; @@ -783,6 +787,19 @@ public > QueryDataSource query(String deviceId, String m } bufferwritedata = bufferWriteProcessor .queryBufferWriteData(deviceId, measurementId, dataType); + + try { + List pathModifications = QueryUtils.getPathModifications( + currentIntervalFileNode.getModFile(), deviceId + + IoTDBConstant.PATH_SEPARATOR + measurementId + ); + if (pathModifications.size() > 0) { + QueryUtils.modifyChunkMetaData(bufferwritedata.right, pathModifications); + } + } catch (IOException e) { + throw new FileNodeProcessorException(e); + } + unsealedTsFile.setTimeSeriesChunkMetaDatas(bufferwritedata.right); } GlobalSortedSeriesDataSource globalSortedSeriesDataSource = new GlobalSortedSeriesDataSource( diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowResource.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowResource.java index 4844d3d4173d..3a528f0d6172 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowResource.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowResource.java @@ -34,9 +34,12 @@ import org.apache.iotdb.db.engine.memtable.IMemTable; import org.apache.iotdb.db.engine.memtable.MemTableFlushUtil; import org.apache.iotdb.db.engine.modification.Deletion; +import org.apache.iotdb.db.engine.modification.Modification; import org.apache.iotdb.db.engine.modification.ModificationFile; import org.apache.iotdb.db.engine.version.VersionController; +import org.apache.iotdb.db.service.IoTDB; import org.apache.iotdb.db.utils.MemUtils; +import org.apache.iotdb.db.utils.QueryUtils; import org.apache.iotdb.tsfile.file.metadata.ChunkGroupMetaData; import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData; import org.apache.iotdb.tsfile.file.metadata.TsDeviceMetadata; @@ -153,6 +156,7 @@ private void readMetadata() throws IOException { insertMetadatas.put(deviceId, new HashMap<>()); } for (ChunkMetaData chunkMetaData : rowGroupMetaData.getChunkMetaDataList()) { + chunkMetaData.setVersion(rowGroupMetaData.getVersion()); String measurementId = chunkMetaData.getMeasurementUid(); if (!insertMetadatas.get(deviceId).containsKey(measurementId)) { insertMetadatas.get(deviceId).put(measurementId, new ArrayList<>()); @@ -176,6 +180,14 @@ public List getInsertMetadatas(String deviceId, String measuremen } } } + try { + List modifications = QueryUtils.getPathModifications(modificationFile, + deviceId + IoTDBConstant.PATH_SEPARATOR + measurementId); + QueryUtils.modifyChunkMetaData(chunkMetaDatas, modifications); + } catch (IOException e) { + LOGGER.error("Cannot access the modification file of Overflow {}, because:", parentPath, + e); + } return chunkMetaDatas; } 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 4778d6182f68..0676f566d15a 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 @@ -1,6 +1,6 @@ /** * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) - * + *

* 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 @@ -8,9 +8,9 @@ * 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. @@ -21,10 +21,14 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Collection; import java.util.List; + import org.apache.iotdb.db.engine.filenode.IntervalFileNode; +import org.apache.iotdb.db.engine.modification.Modification; import org.apache.iotdb.db.query.control.FileReaderManager; import org.apache.iotdb.db.query.reader.IReader; +import org.apache.iotdb.db.utils.QueryUtils; import org.apache.iotdb.db.utils.TimeValuePair; import org.apache.iotdb.db.utils.TimeValuePairUtils; import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData; @@ -164,10 +168,17 @@ private void initSingleTsFileReader(IntervalFileNode fileNode) throws IOExceptio // to avoid too many opened files TsFileSequenceReader tsFileReader = FileReaderManager.getInstance() - .get(fileNode.getFilePath(), false); + .get(fileNode.getFilePath(), false); MetadataQuerierByFileImpl metadataQuerier = new MetadataQuerierByFileImpl(tsFileReader); List metaDataList = metadataQuerier.getChunkMetaDataList(seriesPath); + + List pathModifications = QueryUtils.getPathModifications(fileNode.getModFile(), + seriesPath.getFullPath()); + if (pathModifications.size() > 0) { + QueryUtils.modifyChunkMetaData(metaDataList, pathModifications); + } + ChunkLoader chunkLoader = new ChunkLoaderImpl(tsFileReader); if (filter == null) { diff --git a/iotdb/src/main/java/org/apache/iotdb/db/utils/QueryUtils.java b/iotdb/src/main/java/org/apache/iotdb/db/utils/QueryUtils.java new file mode 100644 index 000000000000..98c9df661a20 --- /dev/null +++ b/iotdb/src/main/java/org/apache/iotdb/db/utils/QueryUtils.java @@ -0,0 +1,84 @@ +/** + * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) + *

+ * 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.utils; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +import org.apache.iotdb.db.engine.modification.Deletion; +import org.apache.iotdb.db.engine.modification.Modification; +import org.apache.iotdb.db.engine.modification.ModificationFile; +import org.apache.iotdb.tsfile.file.metadata.ChunkMetaData; + +public class QueryUtils { + + /** + * modifyChunkMetaData iterates the chunkMetaData and applies all available modifications on it to + * generate a ModifiedChunkMetadata. + * @param chunkMetaData the original chunkMetaData. + * @param modifications all possible modifications. + * @return a list of all generated ModifiedChunkMetadata. + */ + public static void modifyChunkMetaData(List chunkMetaData, + List modifications) { + int modIndex = 0; + + for (int metaIndex = 0; metaIndex < chunkMetaData.size(); metaIndex++) { + ChunkMetaData metaData = chunkMetaData.get(metaIndex); + for (int j = modIndex; j < modifications.size(); j++) { + // iterate each modification to find the max deletion time + Modification modification = modifications.get(modIndex); + if (modification.getVersionNum() > metaData.getVersion()) { + if (modification instanceof Deletion) { + Deletion deletion = (Deletion) modification; + if (metaData.getDeletedAt() < deletion.getTimestamp()) { + metaData.setDeletedAt(deletion.getTimestamp()); + modIndex = j; + } + } + } else { + // skip old modifications for next metadata + modIndex++; + } + } + } + // remove chunks that are completely deleted + chunkMetaData.removeIf(metaData -> metaData.getDeletedAt() >= metaData.getEndTime()); + } + + public static List getPathModifications(ModificationFile modFile, String path) + throws IOException { + // TODO: use query context to avoid multiple calls of getModifications() + Collection allModifications = modFile.getModifications(); + List pathModifications = new ArrayList<>(); + if (allModifications.size() > 0) { + allModifications.forEach(modification -> { + if (modification.getPath().equals(path)) { + pathModifications.add(modification); + } + }); + } + return pathModifications; + } + +} diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetaData.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetaData.java index c721129e4f7b..0832bbc9770f 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetaData.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetaData.java @@ -50,11 +50,15 @@ public class ChunkMetaData { private TSDataType tsDataType; + /** version is set according to its belonging ChunkGroup only when being queried, so it is not + persisted. + */ + private long version; + /** - * The maximum time of the tombstones that take effect on this chunk. Only data with larger. - * timestamps than this should be exposed to user. + * All data with timestamp <= deletedAt are considered deleted. */ - private long maxTombstoneTime; + private long deletedAt; private TsDigest valuesStatistics; @@ -251,12 +255,19 @@ public int serializeTo(ByteBuffer buffer) { return byteLen; } - public long getMaxTombstoneTime() { - return maxTombstoneTime; + public long getVersion() { + return version; } - public void setMaxTombstoneTime(long maxTombstoneTime) { - this.maxTombstoneTime = maxTombstoneTime; + public void setVersion(long version) { + this.version = version; } + public long getDeletedAt() { + return deletedAt; + } + + public void setDeletedAt(long deletedAt) { + this.deletedAt = deletedAt; + } } diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/Chunk.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/Chunk.java index 975121f683d0..56d259ac84bc 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/Chunk.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/common/Chunk.java @@ -29,6 +29,7 @@ public class Chunk { private ChunkHeader chunkHeader; private ByteBuffer chunkData; + private long deletedAt; public Chunk(ChunkHeader header, ByteBuffer buffer) { this.chunkHeader = header; @@ -42,4 +43,12 @@ public ChunkHeader getHeader() { public ByteBuffer getData() { return chunkData; } + + public long getDeletedAt() { + return deletedAt; + } + + public void setDeletedAt(long deletedAt) { + this.deletedAt = deletedAt; + } } diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/ChunkLoaderImpl.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/ChunkLoaderImpl.java index 42d017acba77..96456594b698 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/ChunkLoaderImpl.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/controller/ChunkLoaderImpl.java @@ -60,7 +60,9 @@ public Chunk loadObjectByKey(ChunkMetaData metaData) throws IOException { @Override public Chunk getChunk(ChunkMetaData chunkMetaData) throws IOException { Chunk chunk = chunkCache.get(chunkMetaData); - return new Chunk(chunk.getHeader(), chunk.getData().duplicate()); + Chunk chunkRet = new Chunk(chunk.getHeader(), chunk.getData().duplicate()); + chunkRet.setDeletedAt(chunkMetaData.getDeletedAt()); + return chunkRet; } @Override 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 f1f41eb7f956..371d2d50fbfa 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 @@ -165,6 +165,7 @@ private List loadChunkMetadata(Path path) throws IOException { .getChunkMetaDataList(); for (ChunkMetaData chunkMetaData : chunkMetaDataListInOneChunkGroup) { if (path.getMeasurement().equals(chunkMetaData.getMeasurementUid())) { + chunkMetaData.setVersion(chunkGroupMetaData.getVersion()); chunkMetaDataList.add(chunkMetaData); } } diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReader.java index beeb161fc6ab..0744c4891be1 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReader.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReader.java @@ -48,8 +48,7 @@ public abstract class ChunkReader { private Filter filter; private BatchData data; - - private long maxTombstoneTime; + protected long deletedAt; public ChunkReader(Chunk chunk) { this(chunk, null); @@ -64,6 +63,7 @@ public ChunkReader(Chunk chunk) { public ChunkReader(Chunk chunk, Filter filter) { this.filter = filter; this.chunkDataBuffer = chunk.getData(); + this.deletedAt = chunk.getDeletedAt(); chunkHeader = chunk.getHeader(); this.unCompressor = UnCompressor.getUnCompressor(chunkHeader.getCompressionType()); valueDecoder = Decoder @@ -128,20 +128,14 @@ private PageReader constructPageReaderForNextPage(int compressedPageBodyLength) chunkDataBuffer.get(compressedPageBody, 0, compressedPageBodyLength); valueDecoder.reset(); - return new PageReader(ByteBuffer.wrap(unCompressor.uncompress(compressedPageBody)), - chunkHeader.getDataType(), - valueDecoder, timeDecoder, filter); + PageReader reader = new PageReader(ByteBuffer.wrap(unCompressor.uncompress(compressedPageBody)), + chunkHeader.getDataType(), + valueDecoder, timeDecoder, filter); + reader.setDeletedAt(deletedAt); + return reader; } public void close() { } - public long getMaxTombstoneTime() { - return this.maxTombstoneTime; - } - - public void setMaxTombstoneTime(long maxTombStoneTime) { - this.maxTombstoneTime = maxTombStoneTime; - } - } diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReaderWithFilter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReaderWithFilter.java index f52d25d0c280..4778f120bbfc 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReaderWithFilter.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReaderWithFilter.java @@ -35,7 +35,7 @@ public ChunkReaderWithFilter(Chunk chunk, Filter filter) { @Override public boolean pageSatisfied(PageHeader pageHeader) { - if (pageHeader.getMax_timestamp() < getMaxTombstoneTime()) { + if (pageHeader.getMax_timestamp() < deletedAt) { return false; } DigestForFilter digest = new DigestForFilter(pageHeader.getMin_timestamp(), diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReaderWithoutFilter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReaderWithoutFilter.java index d9e9a42bdf0c..42afeb950039 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReaderWithoutFilter.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReaderWithoutFilter.java @@ -30,7 +30,7 @@ public ChunkReaderWithoutFilter(Chunk chunk) { @Override public boolean pageSatisfied(PageHeader pageHeader) { - return pageHeader.getMax_timestamp() > getMaxTombstoneTime(); + return pageHeader.getMax_timestamp() > deletedAt; } } diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/page/PageReader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/page/PageReader.java index 04d7054ae2ca..ac6f4a41c323 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/page/PageReader.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/page/PageReader.java @@ -49,6 +49,8 @@ public class PageReader { private Filter filter = null; + private long deletedAt; + public PageReader(ByteBuffer pageData, TSDataType dataType, Decoder valueDecoder, Decoder timeDecoder, Filter filter) { @@ -110,25 +112,48 @@ private BatchData getAllPageData() throws IOException { while (timeDecoder.hasNext(timeBuffer)) { long timestamp = timeDecoder.readLong(timeBuffer); - pageData.putTime(timestamp); switch (dataType) { case BOOLEAN: - pageData.putBoolean(valueDecoder.readBoolean(valueBuffer)); + boolean aBoolean = valueDecoder.readBoolean(valueBuffer); + if (timestamp > deletedAt) { + pageData.putTime(timestamp); + pageData.putBoolean(aBoolean); + } break; case INT32: - pageData.putInt(valueDecoder.readInt(valueBuffer)); + int anInt = valueDecoder.readInt(valueBuffer); + if (timestamp > deletedAt) { + pageData.putTime(timestamp); + pageData.putInt(anInt); + } break; case INT64: - pageData.putLong(valueDecoder.readLong(valueBuffer)); + long aLong = valueDecoder.readLong(valueBuffer); + if (timestamp > deletedAt) { + pageData.putTime(timestamp); + pageData.putLong(aLong); + } break; case FLOAT: - pageData.putFloat(valueDecoder.readFloat(valueBuffer)); + float aFloat = valueDecoder.readFloat(valueBuffer); + if (timestamp > deletedAt) { + pageData.putTime(timestamp); + pageData.putFloat(aFloat); + } break; case DOUBLE: - pageData.putDouble(valueDecoder.readDouble(valueBuffer)); + double aDouble = valueDecoder.readDouble(valueBuffer); + if (timestamp > deletedAt) { + pageData.putTime(timestamp); + pageData.putDouble(aDouble); + } break; case TEXT: - pageData.putBinary(valueDecoder.readBinary(valueBuffer)); + Binary aBinary = valueDecoder.readBinary(valueBuffer); + if (timestamp > deletedAt) { + pageData.putTime(timestamp); + pageData.putBinary(aBinary); + } break; default: throw new UnSupportedDataTypeException(String.valueOf(dataType)); @@ -146,42 +171,42 @@ private BatchData getAllPageData(Filter filter) throws IOException { switch (dataType) { case BOOLEAN: boolean aBoolean = valueDecoder.readBoolean(valueBuffer); - if (filter.satisfy(timestamp, aBoolean)) { + if (timestamp > deletedAt && filter.satisfy(timestamp, aBoolean)) { pageData.putTime(timestamp); pageData.putBoolean(aBoolean); } break; case INT32: int anInt = valueDecoder.readInt(valueBuffer); - if (filter.satisfy(timestamp, anInt)) { + if (timestamp > deletedAt && filter.satisfy(timestamp, anInt)) { pageData.putTime(timestamp); pageData.putInt(anInt); } break; case INT64: long aLong = valueDecoder.readLong(valueBuffer); - if (filter.satisfy(timestamp, aLong)) { + if (timestamp > deletedAt && filter.satisfy(timestamp, aLong)) { pageData.putTime(timestamp); pageData.putLong(aLong); } break; case FLOAT: float aFloat = valueDecoder.readFloat(valueBuffer); - if (filter.satisfy(timestamp, aFloat)) { + if (timestamp > deletedAt && filter.satisfy(timestamp, aFloat)) { pageData.putTime(timestamp); pageData.putFloat(aFloat); } break; case DOUBLE: double aDouble = valueDecoder.readDouble(valueBuffer); - if (filter.satisfy(timestamp, aDouble)) { + if (timestamp > deletedAt && filter.satisfy(timestamp, aDouble)) { pageData.putTime(timestamp); pageData.putDouble(aDouble); } break; case TEXT: Binary aBinary = valueDecoder.readBinary(valueBuffer); - if (filter.satisfy(timestamp, aBinary)) { + if (timestamp > deletedAt && filter.satisfy(timestamp, aBinary)) { pageData.putTime(timestamp); pageData.putBinary(aBinary); } @@ -199,4 +224,7 @@ public void close() { valueBuffer = null; } + public void setDeletedAt(long deletedAt) { + this.deletedAt = deletedAt; + } } diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/FileSeriesReader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/FileSeriesReader.java index b24658911dc7..055644871e06 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/FileSeriesReader.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/FileSeriesReader.java @@ -78,7 +78,7 @@ public BatchData nextBatch() throws IOException { // current chunk does not have additional batch, init new chunk reader while (chunkToRead < chunkMetaDataList.size()) { - ChunkMetaData chunkMetaData = chunkMetaDataList.get(chunkToRead++); + ChunkMetaData chunkMetaData = nextChunkMeta(); if (chunkSatisfied(chunkMetaData)) { // chunk metadata satisfy the condition initChunkReader(chunkMetaData); @@ -108,4 +108,7 @@ public void close() throws IOException { chunkLoader.close(); } + protected ChunkMetaData nextChunkMeta() { + return chunkMetaDataList.get(chunkToRead++); + } } diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/FileSeriesReaderWithFilter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/FileSeriesReaderWithFilter.java index 31baa5ac5376..58c665308159 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/FileSeriesReaderWithFilter.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/FileSeriesReaderWithFilter.java @@ -47,7 +47,6 @@ public FileSeriesReaderWithFilter(ChunkLoader chunkLoader, protected void initChunkReader(ChunkMetaData chunkMetaData) throws IOException { Chunk chunk = chunkLoader.getChunk(chunkMetaData); this.chunkReader = new ChunkReaderWithFilter(chunk, filter); - this.chunkReader.setMaxTombstoneTime(chunkMetaData.getMaxTombstoneTime()); } @Override diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/FileSeriesReaderWithoutFilter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/FileSeriesReaderWithoutFilter.java index e1095c3e66b6..4e676af08a7e 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/FileSeriesReaderWithoutFilter.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/FileSeriesReaderWithoutFilter.java @@ -40,7 +40,6 @@ public FileSeriesReaderWithoutFilter(ChunkLoader chunkLoader, protected void initChunkReader(ChunkMetaData chunkMetaData) throws IOException { Chunk chunk = chunkLoader.getChunk(chunkMetaData); this.chunkReader = new ChunkReaderWithoutFilter(chunk); - this.chunkReader.setMaxTombstoneTime(chunkMetaData.getMaxTombstoneTime()); } @Override From 7e19ded59fd83a8b2f997ebf9b9f08a2d4000fb5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=B1=9F=E5=A4=A9?= Date: Mon, 21 Jan 2019 10:33:33 +0800 Subject: [PATCH 13/24] add some test --- .../db/qp/strategy/PhysicalGenerator.java | 47 ++-- .../modification/DeletionQueryTest.java | 222 ++++++++++++++++++ .../db/integration/IoTDBCompleteTest.java | 1 + .../db/integration/IoTDBDeletionTest.java | 202 ++++++++++++++++ .../reader/chunk/ChunkReaderByTimestamp.java | 2 +- .../series/SeriesReaderByTimestamp.java | 1 - 6 files changed, 451 insertions(+), 24 deletions(-) create mode 100644 iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionQueryTest.java create mode 100644 iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBDeletionTest.java diff --git a/iotdb/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java b/iotdb/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java index b201989c07a9..630b787880e5 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/qp/strategy/PhysicalGenerator.java @@ -1,6 +1,6 @@ /** * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) - * + *

* 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 @@ -8,9 +8,9 @@ * 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. @@ -20,6 +20,7 @@ package org.apache.iotdb.db.qp.strategy; import java.util.List; + import org.apache.iotdb.db.auth.AuthException; import org.apache.iotdb.db.exception.ProcessorException; import org.apache.iotdb.db.exception.qp.LogicalOperatorException; @@ -27,6 +28,7 @@ import org.apache.iotdb.db.qp.executor.QueryProcessExecutor; import org.apache.iotdb.db.qp.logical.Operator; import org.apache.iotdb.db.qp.logical.crud.BasicFunctionOperator; +import org.apache.iotdb.db.qp.logical.crud.DeleteOperator; import org.apache.iotdb.db.qp.logical.crud.FilterOperator; import org.apache.iotdb.db.qp.logical.crud.InsertOperator; import org.apache.iotdb.db.qp.logical.crud.QueryOperator; @@ -36,6 +38,7 @@ import org.apache.iotdb.db.qp.logical.sys.PropertyOperator; import org.apache.iotdb.db.qp.physical.PhysicalPlan; import org.apache.iotdb.db.qp.physical.crud.AggregationPlan; +import org.apache.iotdb.db.qp.physical.crud.DeletePlan; import org.apache.iotdb.db.qp.physical.crud.FillQueryPlan; import org.apache.iotdb.db.qp.physical.crud.GroupByPlan; import org.apache.iotdb.db.qp.physical.crud.InsertPlan; @@ -62,15 +65,15 @@ public PhysicalGenerator(QueryProcessExecutor executor) { } public PhysicalPlan transformToPhysicalPlan(Operator operator) - throws QueryProcessorException, ProcessorException { + throws QueryProcessorException, ProcessorException { List paths; switch (operator.getType()) { case AUTHOR: AuthorOperator author = (AuthorOperator) operator; try { return new AuthorPlan(author.getAuthorType(), author.getUserName(), author.getRoleName(), - author.getPassWord(), author.getNewPassword(), author.getPrivilegeList(), - author.getNodeName()); + author.getPassWord(), author.getNewPassword(), author.getPrivilegeList(), + author.getNodeName()); } catch (AuthException e) { throw new QueryProcessorException(e.getMessage()); } @@ -82,32 +85,32 @@ public PhysicalPlan transformToPhysicalPlan(Operator operator) case METADATA: MetadataOperator metadata = (MetadataOperator) operator; return new MetadataPlan(metadata.getNamespaceType(), metadata.getPath(), - metadata.getDataType(), - metadata.getEncoding(), metadata.getEncodingArgs(), metadata.getDeletePathList()); + metadata.getDataType(), + metadata.getEncoding(), metadata.getEncodingArgs(), metadata.getDeletePathList()); case PROPERTY: PropertyOperator property = (PropertyOperator) operator; return new PropertyPlan(property.getPropertyType(), property.getPropertyPath(), - property.getMetadataPath()); - // case DELETE: - // DeleteOperator delete = (DeleteOperator) operator; - // paths = delete.getSelectedPaths(); - // if (delete.getTime() <= 0) { - // throw new LogicalOperatorException("For Delete command, time must greater than 0."); - // } - // return new DeletePlan(delete.getTime(), paths); + property.getMetadataPath()); + case DELETE: + DeleteOperator delete = (DeleteOperator) operator; + paths = delete.getSelectedPaths(); + if (delete.getTime() <= 0) { + throw new LogicalOperatorException("For Delete command, time must greater than 0."); + } + return new DeletePlan(delete.getTime(), paths); case INSERT: InsertOperator Insert = (InsertOperator) operator; paths = Insert.getSelectedPaths(); if (paths.size() != 1) { throw new LogicalOperatorException( - "For Insert command, cannot specified more than one seriesPath:" + paths); + "For Insert command, cannot specified more than one seriesPath:" + paths); } if (Insert.getTime() <= 0) { throw new LogicalOperatorException("For Insert command, time must greater than 0."); } return new InsertPlan(paths.get(0).getFullPath(), Insert.getTime(), - Insert.getMeasurementList(), - Insert.getValueList()); + Insert.getMeasurementList(), + Insert.getValueList()); // case UPDATE: // UpdateOperator update = (UpdateOperator) operator; // UpdatePlan updatePlan = new UpdatePlan(); @@ -231,7 +234,7 @@ public PhysicalPlan transformToPhysicalPlan(Operator operator) // } private PhysicalPlan transformQuery(QueryOperator queryOperator) - throws QueryProcessorException, ProcessorException { + throws QueryProcessorException, ProcessorException { QueryPlan queryPlan; @@ -252,7 +255,7 @@ private PhysicalPlan transformQuery(QueryOperator queryOperator) } else if (queryOperator.hasAggregation()) { // ordinary query queryPlan = new AggregationPlan(); ((AggregationPlan) queryPlan) - .setAggregations(queryOperator.getSelectOperator().getAggregations()); + .setAggregations(queryOperator.getSelectOperator().getAggregations()); } else { queryPlan = new QueryPlan(); } 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 new file mode 100644 index 000000000000..0efa2d7317b8 --- /dev/null +++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionQueryTest.java @@ -0,0 +1,222 @@ +/** + * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) + *

+ * Licensed 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.modification; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.iotdb.db.engine.filenode.FileNodeManager; +import org.apache.iotdb.db.exception.FileNodeManagerException; +import org.apache.iotdb.db.exception.MetadataArgsErrorException; +import org.apache.iotdb.db.exception.PathErrorException; +import org.apache.iotdb.db.metadata.MManager; +import org.apache.iotdb.db.query.executor.EngineQueryRouter; +import org.apache.iotdb.db.utils.EnvironmentUtils; +import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; +import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding; +import org.apache.iotdb.tsfile.read.common.Path; +import org.apache.iotdb.tsfile.read.expression.QueryExpression; +import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet; +import org.apache.iotdb.tsfile.write.record.TSRecord; +import org.apache.iotdb.tsfile.write.record.datapoint.DoubleDataPoint; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class DeletionQueryTest { + private String processorName = "root.test"; + + private static String[] measurements = new String[10]; + private String dataType = TSDataType.DOUBLE.toString(); + private String encoding = TSEncoding.PLAIN.toString(); + private String[] args = new String[0]; + private EngineQueryRouter router = new EngineQueryRouter(); + + static { + for (int i = 0; i < 10; i++) { + measurements[i] = "m" + i; + } + } + + @Before + public void setup() throws MetadataArgsErrorException, + PathErrorException, IOException, FileNodeManagerException { + MManager.getInstance().setStorageLevelToMTree(processorName); + for (int i = 0; i < 10; i++) { + MManager.getInstance().addPathToMTree(processorName + "." + measurements[i], dataType, + encoding, args); + FileNodeManager.getInstance().addTimeSeries(new Path(processorName, measurements[i]), dataType, + encoding, args); + } + } + + @After + public void teardown() throws IOException, FileNodeManagerException { + EnvironmentUtils.cleanEnv(); + } + + @Test + public void testDeleteInBufferWriteCache() throws + FileNodeManagerException, IOException { + + for (int i = 1; i <= 100; i++) { + TSRecord record = new TSRecord(i, processorName); + for (int j = 0; j < 10; j++) { + record.addTuple(new DoubleDataPoint(measurements[j], i * 1.0)); + } + FileNodeManager.getInstance().insert(record, false); + } + + FileNodeManager.getInstance().delete(processorName, measurements[3], 50); + FileNodeManager.getInstance().delete(processorName, measurements[4], 50); + FileNodeManager.getInstance().delete(processorName, measurements[5], 30); + FileNodeManager.getInstance().delete(processorName, measurements[5], 50); + + List pathList = new ArrayList<>(); + pathList.add(new Path(processorName, measurements[3])); + pathList.add(new Path(processorName, measurements[4])); + pathList.add(new Path(processorName, measurements[5])); + + QueryExpression queryExpression = QueryExpression.create(pathList, null); + QueryDataSet dataSet = router.query(queryExpression); + + int count = 0; + while (dataSet.hasNext()) { + dataSet.next(); + count++; + } + assertEquals(count, 50); + } + + @Test + public void testDeleteInBufferWriteFile() throws FileNodeManagerException, IOException { + for (int i = 1; i <= 100; i++) { + TSRecord record = new TSRecord(i, processorName); + for (int j = 0; j < 10; j++) { + record.addTuple(new DoubleDataPoint(measurements[j], i * 1.0)); + } + FileNodeManager.getInstance().insert(record, false); + } + FileNodeManager.getInstance().closeAll(); + + FileNodeManager.getInstance().delete(processorName, measurements[5], 50); + FileNodeManager.getInstance().delete(processorName, measurements[4], 40); + FileNodeManager.getInstance().delete(processorName, measurements[3], 30); + + List pathList = new ArrayList<>(); + pathList.add(new Path(processorName, measurements[3])); + pathList.add(new Path(processorName, measurements[4])); + pathList.add(new Path(processorName, measurements[5])); + + QueryExpression queryExpression = QueryExpression.create(pathList, null); + QueryDataSet dataSet = router.query(queryExpression); + + int count = 0; + while (dataSet.hasNext()) { + dataSet.next(); + count++; + } + assertEquals(count, 70); + } + + @Test + public void testDeleteInOverflowCache() throws FileNodeManagerException, IOException { + // insert into BufferWrite + for (int i = 101; i <= 200; i++) { + TSRecord record = new TSRecord(i, processorName); + for (int j = 0; j < 10; j++) { + record.addTuple(new DoubleDataPoint(measurements[j], i * 1.0)); + } + FileNodeManager.getInstance().insert(record, false); + } + FileNodeManager.getInstance().closeAll(); + + // insert into Overflow + for (int i = 1; i <= 100; i++) { + TSRecord record = new TSRecord(i, processorName); + for (int j = 0; j < 10; j++) { + record.addTuple(new DoubleDataPoint(measurements[j], i * 1.0)); + } + FileNodeManager.getInstance().insert(record, false); + } + + FileNodeManager.getInstance().delete(processorName, measurements[3], 50); + FileNodeManager.getInstance().delete(processorName, measurements[4], 50); + FileNodeManager.getInstance().delete(processorName, measurements[5], 30); + FileNodeManager.getInstance().delete(processorName, measurements[5], 50); + + List pathList = new ArrayList<>(); + pathList.add(new Path(processorName, measurements[3])); + pathList.add(new Path(processorName, measurements[4])); + pathList.add(new Path(processorName, measurements[5])); + + QueryExpression queryExpression = QueryExpression.create(pathList, null); + QueryDataSet dataSet = router.query(queryExpression); + + int count = 0; + while (dataSet.hasNext()) { + dataSet.next(); + count++; + } + assertEquals(count, 150); + } + + @Test + public void testDeleteInOverflowFile() throws FileNodeManagerException, IOException { + // insert into BufferWrite + for (int i = 101; i <= 200; i++) { + TSRecord record = new TSRecord(i, processorName); + for (int j = 0; j < 10; j++) { + record.addTuple(new DoubleDataPoint(measurements[j], i * 1.0)); + } + FileNodeManager.getInstance().insert(record, false); + } + FileNodeManager.getInstance().closeAll(); + + // insert into Overflow + for (int i = 1; i <= 100; i++) { + TSRecord record = new TSRecord(i, processorName); + for (int j = 0; j < 10; j++) { + record.addTuple(new DoubleDataPoint(measurements[j], i * 1.0)); + } + FileNodeManager.getInstance().insert(record, false); + } + FileNodeManager.getInstance().closeAll(); + + FileNodeManager.getInstance().delete(processorName, measurements[5], 50); + FileNodeManager.getInstance().delete(processorName, measurements[4], 40); + FileNodeManager.getInstance().delete(processorName, measurements[3], 30); + + List pathList = new ArrayList<>(); + pathList.add(new Path(processorName, measurements[3])); + pathList.add(new Path(processorName, measurements[4])); + pathList.add(new Path(processorName, measurements[5])); + + QueryExpression queryExpression = QueryExpression.create(pathList, null); + QueryDataSet dataSet = router.query(queryExpression); + + int count = 0; + while (dataSet.hasNext()) { + dataSet.next(); + count++; + } + assertEquals(count, 170); + } +} diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBCompleteTest.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBCompleteTest.java index 0e1983736788..f687864c0fce 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBCompleteTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBCompleteTest.java @@ -65,6 +65,7 @@ public void Test() throws ClassNotFoundException, SQLException { SimpleTest(); InsertTest(); SelectTest(); + DeleteTest(); } public void SimpleTest() throws ClassNotFoundException, SQLException { diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBDeletionTest.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBDeletionTest.java new file mode 100644 index 000000000000..86cceaff82f2 --- /dev/null +++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBDeletionTest.java @@ -0,0 +1,202 @@ +/** + * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) + *

+ * 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.integration; + +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.service.IoTDB; +import org.apache.iotdb.db.utils.EnvironmentUtils; +import org.apache.iotdb.jdbc.Config; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class IoTDBDeletionTest { + private static IoTDB deamon; + + private static String[] creationSqls = new String[]{ + "SET STORAGE GROUP TO root.vehicle.d0", "SET STORAGE GROUP TO root.vehicle.d1", + + "CREATE TIMESERIES root.vehicle.d0.s0 WITH DATATYPE=INT32, ENCODING=RLE", + "CREATE TIMESERIES root.vehicle.d0.s1 WITH DATATYPE=INT64, ENCODING=RLE", + "CREATE TIMESERIES root.vehicle.d0.s2 WITH DATATYPE=FLOAT, ENCODING=RLE", + "CREATE TIMESERIES root.vehicle.d0.s3 WITH DATATYPE=TEXT, ENCODING=PLAIN", + "CREATE TIMESERIES root.vehicle.d0.s4 WITH DATATYPE=BOOLEAN, ENCODING=PLAIN", + }; + + private static String intertTemplate = "INSERT INTO root.vehicle.d0(timestamp,s0,s1,s2,s3,s4" + + ") VALUES(%d,%d,%d,%f,%s,%b)"; + private static String deleteAllTemplate = "DELETE FROM root.vehicle.d0.* WHERE time <= 10000"; + + @BeforeClass + public static void setUp() throws Exception { + EnvironmentUtils.closeStatMonitor(); + EnvironmentUtils.closeMemControl(); + deamon = IoTDB.getInstance(); + deamon.active(); + EnvironmentUtils.envSetUp(); + Class.forName(Config.JDBC_DRIVER_NAME); + prepareSeries(); + } + + @AfterClass + public static void tearDown() throws Exception { + deamon.stop(); + Thread.sleep(5000); + + EnvironmentUtils.cleanEnv(); + } + + @Before + public void prepare() throws SQLException { + prepareData(); + } + + @After + public void cleanup() throws SQLException { + cleanData(); + } + + @Test + public void test() throws SQLException { + Connection connection = null; + try { + connection = DriverManager + .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", + "root"); + Statement statement = connection.createStatement(); + statement.execute("DELETE FROM root.vehicle.d0.s0 WHERE time <= 300"); + statement.execute("DELETE FROM root.vehicle.d0.s1,root.vehicle.d0.s2,root.vehicle.d0.s3" + + " WHERE time <= 350"); + statement.execute("DELETE FROM root.vehicle.d0.* WHERE time <= 150"); + + ResultSet set = statement.executeQuery("SELECT * FROM root.vehicle.d0"); + int cnt = 0; + while (set.next()) { + cnt ++; + } + assertEquals(250, cnt); + + set = statement.executeQuery("SELECT s0 FROM root.vehicle.d0"); + cnt = 0; + while (set.next()) { + cnt ++; + } + assertEquals(100, cnt); + + set = statement.executeQuery("SELECT s1,s2,s3 FROM root.vehicle.d0"); + cnt = 0; + while (set.next()) { + cnt ++; + } + assertEquals(50, cnt); + + statement.close(); + } finally { + if (connection != null) { + connection.close(); + } + } + } + + public static void prepareSeries() throws SQLException { + Connection connection = null; + try { + connection = DriverManager + .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", + "root"); + Statement statement = connection.createStatement(); + for (String sql : creationSqls) { + statement.execute(sql); + } + statement.close(); + } catch (Exception e) { + e.printStackTrace(); + } finally { + if (connection != null) { + connection.close(); + } + } + } + + public void prepareData() throws SQLException { + Connection connection = null; + try { + connection = DriverManager + .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", + "root"); + Statement statement = connection.createStatement(); + // prepare BufferWrite file + for (int i = 201; i <= 300; i++) { + statement.execute(String.format(intertTemplate, i, i, i, (double) i, "\'" + i + "\'", + i % 2 == 0)); + } + statement.execute("merge"); + // prepare Overflow file + for (int i = 1; i <= 100; i++) { + statement.execute(String.format(intertTemplate, i, i, i, (double) i, "\'" + i + "\'", + i % 2 == 0)); + } + statement.execute("merge"); + // prepare BufferWrite cache + for (int i = 301; i <= 400; i++) { + statement.execute(String.format(intertTemplate, i, i, i, (double) i, "\'" + i + "\'", + i % 2 == 0)); + } + // prepare Overflow cache + for (int i = 101; i <= 200; i++) { + statement.execute(String.format(intertTemplate, i, i, i, (double) i, "\'" + i + "\'", + i % 2 == 0)); + } + + statement.close(); + } finally { + if (connection != null) { + connection.close(); + } + } + } + + public void cleanData() throws SQLException { + Connection connection = null; + try { + connection = DriverManager + .getConnection(Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", + "root"); + Statement statement = connection.createStatement(); + statement.execute(deleteAllTemplate); + + statement.close(); + } finally { + if (connection != null) { + connection.close(); + } + } + } +} 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 c69cb4b4ccd5..02618f639662 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 @@ -34,7 +34,7 @@ public ChunkReaderByTimestamp(Chunk chunk) { public boolean pageSatisfied(PageHeader pageHeader) { long maxTimestamp = pageHeader.getMax_timestamp(); // if maxTimestamp > currentTimestamp, this page should NOT be skipped - return maxTimestamp >= currentTimestamp && maxTimestamp >= getMaxTombstoneTime(); + return maxTimestamp >= currentTimestamp && maxTimestamp > deletedAt; } public void setCurrentTimestamp(long currentTimestamp) { 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 26503bfe02db..f8911ff96bc5 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 @@ -118,7 +118,6 @@ private boolean constructNextSatisfiedChunkReader() throws IOException { private void initChunkReader(ChunkMetaData chunkMetaData) throws IOException { Chunk chunk = chunkLoader.getChunk(chunkMetaData); this.chunkReader = new ChunkReaderByTimestamp(chunk); - this.chunkReader.setMaxTombstoneTime(chunkMetaData.getMaxTombstoneTime()); } private boolean chunkSatisfied(ChunkMetaData chunkMetaData) { From 3523342275ef345197009ce74c4ae5055a1f790a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=B1=9F=E5=A4=A9?= Date: Mon, 21 Jan 2019 10:48:13 +0800 Subject: [PATCH 14/24] fix unclosed result sets in a test --- .../org/apache/iotdb/db/integration/IoTDBDeletionTest.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBDeletionTest.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBDeletionTest.java index 86cceaff82f2..18465a55c45e 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBDeletionTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBDeletionTest.java @@ -52,7 +52,7 @@ public class IoTDBDeletionTest { private static String intertTemplate = "INSERT INTO root.vehicle.d0(timestamp,s0,s1,s2,s3,s4" + ") VALUES(%d,%d,%d,%f,%s,%b)"; - private static String deleteAllTemplate = "DELETE FROM root.vehicle.d0.* WHERE time <= 10000"; + private static String deleteAllTemplate = "DELETE FROM root.vehicle.d0 WHERE time <= 10000"; @BeforeClass public static void setUp() throws Exception { @@ -94,7 +94,7 @@ public void test() throws SQLException { statement.execute("DELETE FROM root.vehicle.d0.s0 WHERE time <= 300"); statement.execute("DELETE FROM root.vehicle.d0.s1,root.vehicle.d0.s2,root.vehicle.d0.s3" + " WHERE time <= 350"); - statement.execute("DELETE FROM root.vehicle.d0.* WHERE time <= 150"); + statement.execute("DELETE FROM root.vehicle.d0 WHERE time <= 150"); ResultSet set = statement.executeQuery("SELECT * FROM root.vehicle.d0"); int cnt = 0; @@ -102,6 +102,7 @@ public void test() throws SQLException { cnt ++; } assertEquals(250, cnt); + set.close(); set = statement.executeQuery("SELECT s0 FROM root.vehicle.d0"); cnt = 0; @@ -109,6 +110,7 @@ public void test() throws SQLException { cnt ++; } assertEquals(100, cnt); + set.close(); set = statement.executeQuery("SELECT s1,s2,s3 FROM root.vehicle.d0"); cnt = 0; @@ -116,6 +118,7 @@ public void test() throws SQLException { cnt ++; } assertEquals(50, cnt); + set.close(); statement.close(); } finally { From 96a714222b8ca406e3aef41b24838623fd039df5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=B1=9F=E5=A4=A9?= Date: Mon, 21 Jan 2019 11:47:28 +0800 Subject: [PATCH 15/24] prevent valid ModificationFile from being deleted after merging --- .../iotdb/db/engine/filenode/FileNodeProcessor.java | 1 + .../iotdb/db/engine/modification/ModificationFile.java | 10 ++++++++++ 2 files changed, 11 insertions(+) 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 b2aacee46b3e..7a68e5aa3649 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 @@ -1425,6 +1425,7 @@ private void switchWaitingToWorkingv2(List backupIntervalFiles String bufferFilePath = bufferFileNode.getFilePath(); if (bufferFilePath != null) { bufferFiles.add(bufferFilePath); + bufferFiles.add(bufferFileNode.getModFile().getFilePath()); } } // add the restore file, if the last file is not closed diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/ModificationFile.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/ModificationFile.java index 72be89f9fdb3..2815e3674b7f 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/ModificationFile.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/ModificationFile.java @@ -35,6 +35,7 @@ public class ModificationFile { private Collection modifications; private ModificationWriter writer; private ModificationReader reader; + private String filePath; /** * Construct a ModificationFile using a file as its storage. @@ -44,6 +45,7 @@ public ModificationFile(String filePath) { LocalTextModificationAccessor accessor = new LocalTextModificationAccessor(filePath); this.writer = accessor; this.reader = accessor; + this.filePath = filePath; } private void init() throws IOException { @@ -96,4 +98,12 @@ public Collection getModifications() throws IOException { return new ArrayList<>(modifications); } } + + public String getFilePath() { + return filePath; + } + + public void setFilePath(String filePath) { + this.filePath = filePath; + } } From 76d72172968665087775ab31e42052149a5bf67c Mon Sep 17 00:00:00 2001 From: CGF Date: Thu, 24 Jan 2019 16:40:44 +0800 Subject: [PATCH 16/24] fix the spelling error in iotdb integration test --- .../iotdb/db/integration/IoTDBAuthorizationIT.java | 8 ++++---- .../apache/iotdb/db/integration/IoTDBCompleteIT.java | 8 ++++---- .../org/apache/iotdb/db/integration/IoTDBDaemonIT.java | 8 ++++---- .../apache/iotdb/db/integration/IoTDBDeletionTest.java | 8 ++++---- .../apache/iotdb/db/integration/IoTDBLargeDataIT.java | 8 ++++---- .../iotdb/db/integration/IoTDBLimitSlimitIT.java | 10 ++++------ .../iotdb/db/integration/IoTDBMetadataFetchIT.java | 8 ++++---- .../iotdb/db/integration/IoTDBMultiSeriesIT.java | 8 ++++---- .../iotdb/db/integration/IoTDBSeriesReaderIT.java | 8 ++++---- .../apache/iotdb/db/integration/IoTDBTimeZoneIT.java | 4 ---- 10 files changed, 36 insertions(+), 42 deletions(-) diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBAuthorizationIT.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBAuthorizationIT.java index abf32e9d46b2..c057234d9ffc 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBAuthorizationIT.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBAuthorizationIT.java @@ -39,7 +39,7 @@ */ public class IoTDBAuthorizationIT { - private IoTDB deamon; + private IoTDB daemon; public static void main(String[] args) throws Exception { for (int i = 0; i < 10; i++) { @@ -54,14 +54,14 @@ public static void main(String[] args) throws Exception { public void setUp() throws Exception { EnvironmentUtils.closeStatMonitor(); EnvironmentUtils.closeMemControl(); - deamon = IoTDB.getInstance(); - deamon.active(); + daemon = IoTDB.getInstance(); + daemon.active(); EnvironmentUtils.envSetUp(); } @After public void tearDown() throws Exception { - deamon.stop(); + daemon.stop(); Thread.sleep(2000); EnvironmentUtils.cleanEnv(); } diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBCompleteIT.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBCompleteIT.java index 15142125797c..7a660a5f8b6f 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBCompleteIT.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBCompleteIT.java @@ -39,20 +39,20 @@ */ public class IoTDBCompleteIT { - private IoTDB deamon; + private IoTDB daemon; @Before public void setUp() throws Exception { EnvironmentUtils.closeStatMonitor(); EnvironmentUtils.closeMemControl(); - deamon = IoTDB.getInstance(); - deamon.active(); + daemon = IoTDB.getInstance(); + daemon.active(); EnvironmentUtils.envSetUp(); } @After public void tearDown() throws Exception { - deamon.stop(); + daemon.stop(); Thread.sleep(5000); EnvironmentUtils.cleanEnv(); } diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBDaemonIT.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBDaemonIT.java index ab65f6b81198..dca276ac179b 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBDaemonIT.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBDaemonIT.java @@ -47,7 +47,7 @@ */ public class IoTDBDaemonIT { - private static IoTDB deamon; + private static IoTDB daemon; private static Connection connection; @@ -115,8 +115,8 @@ public class IoTDBDaemonIT { public static void setUp() throws Exception { EnvironmentUtils.closeStatMonitor(); EnvironmentUtils.closeMemControl(); - deamon = IoTDB.getInstance(); - deamon.active(); + daemon = IoTDB.getInstance(); + daemon.active(); EnvironmentUtils.envSetUp(); insertData(); @@ -127,7 +127,7 @@ public static void setUp() throws Exception { @AfterClass public static void tearDown() throws Exception { connection.close(); - deamon.stop(); + daemon.stop(); Thread.sleep(5000); EnvironmentUtils.cleanEnv(); diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBDeletionTest.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBDeletionTest.java index 18465a55c45e..7687747010b5 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBDeletionTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBDeletionTest.java @@ -38,7 +38,7 @@ import static org.junit.Assert.assertEquals; public class IoTDBDeletionTest { - private static IoTDB deamon; + private static IoTDB daemon; private static String[] creationSqls = new String[]{ "SET STORAGE GROUP TO root.vehicle.d0", "SET STORAGE GROUP TO root.vehicle.d1", @@ -58,8 +58,8 @@ public class IoTDBDeletionTest { public static void setUp() throws Exception { EnvironmentUtils.closeStatMonitor(); EnvironmentUtils.closeMemControl(); - deamon = IoTDB.getInstance(); - deamon.active(); + daemon = IoTDB.getInstance(); + daemon.active(); EnvironmentUtils.envSetUp(); Class.forName(Config.JDBC_DRIVER_NAME); prepareSeries(); @@ -67,7 +67,7 @@ public static void setUp() throws Exception { @AfterClass public static void tearDown() throws Exception { - deamon.stop(); + daemon.stop(); Thread.sleep(5000); EnvironmentUtils.cleanEnv(); diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBLargeDataIT.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBLargeDataIT.java index ffc429a307bd..71a91c8f11ae 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBLargeDataIT.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBLargeDataIT.java @@ -43,7 +43,7 @@ */ public class IoTDBLargeDataIT { - private static IoTDB deamon; + private static IoTDB daemon; private static boolean testFlag = Constant.testFlag; private static TSFileConfig tsFileConfig = TSFileDescriptor.getInstance().getConfig(); @@ -69,8 +69,8 @@ public static void setUp() throws Exception { tsFileConfig.pageSizeInByte = 1024 * 150; tsFileConfig.groupSizeInByte = 1024 * 1000; - deamon = IoTDB.getInstance(); - deamon.active(); + daemon = IoTDB.getInstance(); + daemon.active(); EnvironmentUtils.envSetUp(); Thread.sleep(5000); @@ -86,7 +86,7 @@ public static void tearDown() throws Exception { connection.close(); - deamon.stop(); + daemon.stop(); Thread.sleep(5000); // recovery value diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBLimitSlimitIT.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBLimitSlimitIT.java index 659cd1efde74..acce84d75c91 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBLimitSlimitIT.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBLimitSlimitIT.java @@ -42,9 +42,7 @@ */ public class IoTDBLimitSlimitIT { - private static IoTDB deamon; - - private static boolean testFlag = Constant.testFlag; + private static IoTDB daemon; private static String[] insertSqls = new String[]{"SET STORAGE GROUP TO root.vehicle", @@ -91,14 +89,14 @@ public class IoTDBLimitSlimitIT { public static void setUp() throws Exception { EnvironmentUtils.closeStatMonitor(); EnvironmentUtils.closeMemControl(); - deamon = IoTDB.getInstance(); - deamon.active(); + daemon = IoTDB.getInstance(); + daemon.active(); EnvironmentUtils.envSetUp(); } @AfterClass public static void tearDown() throws Exception { - deamon.stop(); + daemon.stop(); Thread.sleep(5000); EnvironmentUtils.cleanEnv(); } diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBMetadataFetchIT.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBMetadataFetchIT.java index 37f2bd7284e7..b73e9dc1138a 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBMetadataFetchIT.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBMetadataFetchIT.java @@ -43,7 +43,7 @@ */ public class IoTDBMetadataFetchIT { - private static IoTDB deamon; + private static IoTDB daemon; private DatabaseMetaData databaseMetaData; @@ -79,8 +79,8 @@ public void setUp() throws Exception { EnvironmentUtils.closeStatMonitor(); EnvironmentUtils.closeMemControl(); - deamon = IoTDB.getInstance(); - deamon.active(); + daemon = IoTDB.getInstance(); + daemon.active(); EnvironmentUtils.envSetUp(); insertSQL(); @@ -88,7 +88,7 @@ public void setUp() throws Exception { @After public void tearDown() throws Exception { - deamon.stop(); + daemon.stop(); Thread.sleep(5000); EnvironmentUtils.cleanEnv(); } diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiSeriesIT.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiSeriesIT.java index 018c325a0606..91562760b0f0 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiSeriesIT.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBMultiSeriesIT.java @@ -43,7 +43,7 @@ */ public class IoTDBMultiSeriesIT { - private static IoTDB deamon; + private static IoTDB daemon; private static boolean testFlag = Constant.testFlag; private static TSFileConfig tsFileConfig = TSFileDescriptor.getInstance().getConfig(); @@ -69,8 +69,8 @@ public static void setUp() throws Exception { tsFileConfig.pageSizeInByte = 1024 * 150; tsFileConfig.groupSizeInByte = 1024 * 1000; - deamon = IoTDB.getInstance(); - deamon.active(); + daemon = IoTDB.getInstance(); + daemon.active(); EnvironmentUtils.envSetUp(); Thread.sleep(5000); @@ -86,7 +86,7 @@ public static void tearDown() throws Exception { connection.close(); - deamon.stop(); + daemon.stop(); Thread.sleep(5000); // recovery value 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 86b6812a1389..9473e9e69bc6 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 @@ -51,7 +51,7 @@ */ public class IoTDBSeriesReaderIT { - private static IoTDB deamon; + private static IoTDB daemon; private static TSFileConfig tsFileConfig = TSFileDescriptor.getInstance().getConfig(); private static int maxNumberOfPointsInPage; @@ -76,8 +76,8 @@ public static void setUp() throws Exception { tsFileConfig.pageSizeInByte = 1024 * 1024 * 150; tsFileConfig.groupSizeInByte = 1024 * 1024 * 1000; - deamon = IoTDB.getInstance(); - deamon.active(); + daemon = IoTDB.getInstance(); + daemon.active(); EnvironmentUtils.envSetUp(); Thread.sleep(5000); @@ -90,7 +90,7 @@ public static void setUp() throws Exception { @AfterClass public static void tearDown() throws Exception { connection.close(); - deamon.stop(); + daemon.stop(); Thread.sleep(5000); // recovery value diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBTimeZoneIT.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBTimeZoneIT.java index 6e954d01cffb..7b707b1fd105 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBTimeZoneIT.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBTimeZoneIT.java @@ -52,23 +52,19 @@ public class IoTDBTimeZoneIT { @Before public void setUp() throws Exception { - // if (testFlag) { EnvironmentUtils.closeStatMonitor(); EnvironmentUtils.closeMemControl(); deamon = IoTDB.getInstance(); deamon.active(); EnvironmentUtils.envSetUp(); createTimeseries(); - // } } @After public void tearDown() throws Exception { - // if (testFlag) { deamon.stop(); Thread.sleep(5000); EnvironmentUtils.cleanEnv(); - // } } /** From 99c5cc1c2eed5d0c9babff2ab28fbf29209d0c02 Mon Sep 17 00:00:00 2001 From: jt Date: Thu, 24 Jan 2019 22:00:16 +0800 Subject: [PATCH 17/24] Fix by PR comments. --- .../modification/DeletionFileNodeTest.java | 4 ++-- .../modification/DeletionQueryTest.java | 8 +++---- .../SimpleFileVersionControllerTest.java | 6 ++--- ...DeletionTest.java => IoTDBDeletionIT.java} | 22 +++++++++---------- .../tsfile/file/metadata/ChunkMetaData.java | 6 +++-- .../tsfile/read/reader/chunk/ChunkReader.java | 4 ++++ .../read/reader/series/FileSeriesReader.java | 2 +- .../iotdb/tsfile/write/TsFileWriter.java | 4 ++++ .../tsfile/write/writer/TsFileIOWriter.java | 6 ++--- 9 files changed, 36 insertions(+), 26 deletions(-) rename iotdb/src/test/java/org/apache/iotdb/db/integration/{IoTDBDeletionTest.java => IoTDBDeletionIT.java} (89%) diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java index 8b833d031e25..21e0880856e3 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java @@ -104,7 +104,7 @@ public void testDeleteInBufferWriteCache() throws timeValuePairs.next(); count++; } - assertEquals(count, 50); + assertEquals(50, count); } @Test @@ -181,7 +181,7 @@ public void testDeleteInOverflowCache() throws FileNodeManagerException { timeValuePairs.next(); count++; } - assertEquals(count, 50); + assertEquals(50, count); } @Test 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 0efa2d7317b8..2e069699bfc8 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 @@ -102,7 +102,7 @@ public void testDeleteInBufferWriteCache() throws dataSet.next(); count++; } - assertEquals(count, 50); + assertEquals(50, count); } @Test @@ -133,7 +133,7 @@ public void testDeleteInBufferWriteFile() throws FileNodeManagerException, IOExc dataSet.next(); count++; } - assertEquals(count, 70); + assertEquals(70, count); } @Test @@ -175,7 +175,7 @@ public void testDeleteInOverflowCache() throws FileNodeManagerException, IOExcep dataSet.next(); count++; } - assertEquals(count, 150); + assertEquals(150, count); } @Test @@ -217,6 +217,6 @@ public void testDeleteInOverflowFile() throws FileNodeManagerException, IOExcept dataSet.next(); count++; } - assertEquals(count, 170); + assertEquals(170, count); } } diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/version/SimpleFileVersionControllerTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/version/SimpleFileVersionControllerTest.java index b33b5065a500..b2063569b044 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/engine/version/SimpleFileVersionControllerTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/version/SimpleFileVersionControllerTest.java @@ -33,13 +33,13 @@ public void test() throws IOException { try { new File(tempFilePath).mkdir(); VersionController versionController = new SimpleFileVersionController(tempFilePath); - assertEquals(versionController.currVersion(), SAVE_INTERVAL); + assertEquals(SAVE_INTERVAL, versionController.currVersion()); for (int i = 0; i < 150; i++) { versionController.nextVersion(); } - assertEquals(versionController.currVersion(), SAVE_INTERVAL + 150); + assertEquals(SAVE_INTERVAL + 150, versionController.currVersion()); versionController = new SimpleFileVersionController(tempFilePath); - assertEquals(versionController.currVersion(), SAVE_INTERVAL + 200); + assertEquals(SAVE_INTERVAL + 200, versionController.currVersion()); } finally { FileUtils.deleteDirectory(new File(tempFilePath)); } diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBDeletionTest.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBDeletionIT.java similarity index 89% rename from iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBDeletionTest.java rename to iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBDeletionIT.java index 7687747010b5..4135cfb68394 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBDeletionTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBDeletionIT.java @@ -37,7 +37,7 @@ import static org.junit.Assert.assertEquals; -public class IoTDBDeletionTest { +public class IoTDBDeletionIT { private static IoTDB daemon; private static String[] creationSqls = new String[]{ @@ -50,9 +50,9 @@ public class IoTDBDeletionTest { "CREATE TIMESERIES root.vehicle.d0.s4 WITH DATATYPE=BOOLEAN, ENCODING=PLAIN", }; - private static String intertTemplate = "INSERT INTO root.vehicle.d0(timestamp,s0,s1,s2,s3,s4" + private String insertTemplate = "INSERT INTO root.vehicle.d0(timestamp,s0,s1,s2,s3,s4" + ") VALUES(%d,%d,%d,%f,%s,%b)"; - private static String deleteAllTemplate = "DELETE FROM root.vehicle.d0 WHERE time <= 10000"; + private String deleteAllTemplate = "DELETE FROM root.vehicle.d0 WHERE time <= 10000"; @BeforeClass public static void setUp() throws Exception { @@ -128,7 +128,7 @@ public void test() throws SQLException { } } - public static void prepareSeries() throws SQLException { + private static void prepareSeries() throws SQLException { Connection connection = null; try { connection = DriverManager @@ -148,7 +148,7 @@ public static void prepareSeries() throws SQLException { } } - public void prepareData() throws SQLException { + private void prepareData() throws SQLException { Connection connection = null; try { connection = DriverManager @@ -157,24 +157,24 @@ public void prepareData() throws SQLException { Statement statement = connection.createStatement(); // prepare BufferWrite file for (int i = 201; i <= 300; i++) { - statement.execute(String.format(intertTemplate, i, i, i, (double) i, "\'" + i + "\'", + statement.execute(String.format(insertTemplate, i, i, i, (double) i, "\'" + i + "\'", i % 2 == 0)); } statement.execute("merge"); - // prepare Overflow file + // prepare Unseq-File for (int i = 1; i <= 100; i++) { - statement.execute(String.format(intertTemplate, i, i, i, (double) i, "\'" + i + "\'", + statement.execute(String.format(insertTemplate, i, i, i, (double) i, "\'" + i + "\'", i % 2 == 0)); } statement.execute("merge"); // prepare BufferWrite cache for (int i = 301; i <= 400; i++) { - statement.execute(String.format(intertTemplate, i, i, i, (double) i, "\'" + i + "\'", + statement.execute(String.format(insertTemplate, i, i, i, (double) i, "\'" + i + "\'", i % 2 == 0)); } // prepare Overflow cache for (int i = 101; i <= 200; i++) { - statement.execute(String.format(intertTemplate, i, i, i, (double) i, "\'" + i + "\'", + statement.execute(String.format(insertTemplate, i, i, i, (double) i, "\'" + i + "\'", i % 2 == 0)); } @@ -186,7 +186,7 @@ public void prepareData() throws SQLException { } } - public void cleanData() throws SQLException { + private void cleanData() throws SQLException { Connection connection = null; try { connection = DriverManager diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetaData.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetaData.java index afe14d802bdd..e1c2eeec4174 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetaData.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/ChunkMetaData.java @@ -49,8 +49,10 @@ public class ChunkMetaData { private TSDataType tsDataType; - /** version is set according to its belonging ChunkGroup only when being queried, so it is not - persisted. + /** + * version is used to define the order of operations(insertion, deletion, update). + * version is set according to its belonging ChunkGroup only when being queried, so it is not + * persisted. */ private long version; diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReader.java index d5316452c283..588b3118ef9e 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReader.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/chunk/ChunkReader.java @@ -47,6 +47,10 @@ public abstract class ChunkReader { private Filter filter; private BatchData data; + + /** + * Data whose timestamp <= deletedAt should be considered deleted(not be returned). + */ protected long deletedAt; public ChunkReader(Chunk chunk) { diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/FileSeriesReader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/FileSeriesReader.java index 65fc63d7d16b..f42c8ae49e42 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/FileSeriesReader.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/series/FileSeriesReader.java @@ -107,7 +107,7 @@ public void close() throws IOException { chunkLoader.close(); } - protected ChunkMetaData nextChunkMeta() { + private ChunkMetaData nextChunkMeta() { return chunkMetaDataList.get(chunkToRead++); } } diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/TsFileWriter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/TsFileWriter.java index ee9a83a99770..8756329bcace 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/TsFileWriter.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/TsFileWriter.java @@ -73,6 +73,10 @@ public class TsFileWriter { **/ private long recordCountForNextMemCheck = 100; private long chunkGroupSizeThreshold; + /** + * In an individual TsFile, version number is not meaningful, added + * only for tests. + */ private long version = 0; /** diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/TsFileIOWriter.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/TsFileIOWriter.java index f01a8c69e069..0287b82726d0 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/TsFileIOWriter.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/write/writer/TsFileIOWriter.java @@ -138,21 +138,21 @@ public void startFlushChunkGroup(String deviceId) throws IOException { * @param statistics - statistic of the whole series * @param maxTime - maximum timestamp of the whole series in this stage * @param minTime - minimum timestamp of the whole series in this stage - * @param datasize - the serialized size of all pages + * @param dataSize - the serialized size of all pages * @return the serialized size of CHunkHeader * @throws IOException if I/O error occurs */ public int startFlushChunk(MeasurementSchema descriptor, CompressionType compressionCodecName, TSDataType tsDataType, TSEncoding encodingType, Statistics statistics, long maxTime, long minTime, - int datasize, int numOfPages) throws IOException { + int dataSize, int numOfPages) throws IOException { LOG.debug("start series chunk:{}, file position {}", descriptor, out.getPosition()); currentChunkMetaData = new ChunkMetaData(descriptor.getMeasurementId(), tsDataType, out.getPosition(), minTime, maxTime); - ChunkHeader header = new ChunkHeader(descriptor.getMeasurementId(), datasize, tsDataType, + ChunkHeader header = new ChunkHeader(descriptor.getMeasurementId(), dataSize, tsDataType, compressionCodecName, encodingType, numOfPages); header.serializeTo(out.wrapAsStream()); From 228f61497d8e866c857970451d9fd3b6eeaae860 Mon Sep 17 00:00:00 2001 From: xiangdong huang Date: Thu, 14 Feb 2019 17:12:27 +0800 Subject: [PATCH 18/24] replace the file header with standard Apache Header --- .../db/engine/modification/Deletion.java | 29 ++++++++++-------- .../db/engine/modification/Modification.java | 29 ++++++++++-------- .../engine/modification/ModificationFile.java | 29 ++++++++++-------- .../io/LocalTextModificationAccessor.java | 29 ++++++++++-------- .../modification/io/ModificationReader.java | 29 ++++++++++-------- .../modification/io/ModificationWriter.java | 29 ++++++++++-------- .../db/engine/modification/package-info.java | 30 ++++++++++--------- 7 files changed, 112 insertions(+), 92 deletions(-) diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/Deletion.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/Deletion.java index c25d5e2b7410..81d800a1c8ea 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/Deletion.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/Deletion.java @@ -1,17 +1,20 @@ /** - * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) - *

- * Licensed 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. + * 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.modification; diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/Modification.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/Modification.java index 3504749ad3b3..8d0956ff52c7 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/Modification.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/Modification.java @@ -1,17 +1,20 @@ /** - * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) - *

- * Licensed 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. + * 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.modification; diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/ModificationFile.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/ModificationFile.java index c04ff1d12ec4..7aa58aba5f5c 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/ModificationFile.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/ModificationFile.java @@ -1,17 +1,20 @@ /** - * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) - *

- * Licensed 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. + * 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.modification; 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 9fd14414e839..709ad43d5654 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 @@ -1,17 +1,20 @@ /** - * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) - *

- * Licensed 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. + * 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.modification.io; diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/io/ModificationReader.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/io/ModificationReader.java index d770eccbc6b5..1abfaddf779d 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/io/ModificationReader.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/io/ModificationReader.java @@ -1,17 +1,20 @@ /** - * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) - *

- * Licensed 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. + * 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.modification.io; diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/io/ModificationWriter.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/io/ModificationWriter.java index 5c3806b3e1b6..a817ca46f44c 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/io/ModificationWriter.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/io/ModificationWriter.java @@ -1,17 +1,20 @@ /** - * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) - *

- * Licensed 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. + * 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.modification.io; diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/package-info.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/package-info.java index 4bf8677242bd..e2fdfa80a833 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/package-info.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/package-info.java @@ -1,18 +1,20 @@ /** - * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) - *

- * Licensed 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. - *

+ * 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. */ /** From 014aa86d034ef5bd6120cacd835df43929f9e9c6 Mon Sep 17 00:00:00 2001 From: xiangdong huang Date: Fri, 15 Feb 2019 14:48:18 +0800 Subject: [PATCH 19/24] add hashCode() method --- .../apache/iotdb/db/engine/modification/Deletion.java | 7 +++++++ .../iotdb/db/engine/modification/Modification.java | 9 ++++++++- 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/Deletion.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/Deletion.java index 81d800a1c8ea..bf59f04cc107 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/Deletion.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/Deletion.java @@ -19,6 +19,8 @@ package org.apache.iotdb.db.engine.modification; +import java.util.Objects; + /** * Deletion is a delete operation on a timeseries. */ @@ -45,4 +47,9 @@ public boolean equals(Object obj) { Deletion del = (Deletion) obj; return super.equals(obj) && del.timestamp == this.timestamp; } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), timestamp); + } } diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/Modification.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/Modification.java index 8d0956ff52c7..81186a1053fd 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/Modification.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/modification/Modification.java @@ -19,6 +19,8 @@ package org.apache.iotdb.db.engine.modification; +import java.util.Objects; + /** * Modification represents an UPDATE or DELETE operation on a certain timeseries. */ @@ -28,7 +30,7 @@ public abstract class Modification { protected String path; protected long versionNum; - public Modification(Type type, String path, long versionNum) { + Modification(Type type, String path, long versionNum) { this.type = type; this.path = path; this.versionNum = versionNum; @@ -70,4 +72,9 @@ public boolean equals(Object obj) { return mod.type.equals(this.type) && mod.path.equals(this.path) && mod.versionNum == this.versionNum; } + + @Override + public int hashCode() { + return Objects.hash(type, path, versionNum); + } } From ebcdccf0cfdcdabe90bf9368aac13dfa01f99061 Mon Sep 17 00:00:00 2001 From: xiangdong huang Date: Fri, 15 Feb 2019 17:21:10 +0800 Subject: [PATCH 20/24] fix sonar tips --- .../version/SimpleFileVersionController.java | 22 ++++++++++++++----- .../SimpleFileVersionControllerTest.java | 5 ++++- 2 files changed, 20 insertions(+), 7 deletions(-) diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/version/SimpleFileVersionController.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/version/SimpleFileVersionController.java index 288c1fb3f935..5a638067fedd 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/version/SimpleFileVersionController.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/version/SimpleFileVersionController.java @@ -21,12 +21,14 @@ import java.io.IOException; import java.util.Arrays; import java.util.Comparator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * SimpleFileVersionController uses a local file and its file name to store the version. */ public class SimpleFileVersionController implements VersionController { - + private static final Logger LOGGER = LoggerFactory.getLogger(SimpleFileVersionController.class); /** * Every time currVersion - prevVersion >= SAVE_INTERVAL, currVersion is persisted and prevVersion * is set to currVersion. When recovering from file, the version number is automatically increased @@ -46,7 +48,11 @@ public SimpleFileVersionController(String directoryPath) throws IOException { @Override public synchronized long nextVersion() { currVersion ++; - checkPersist(); + try { + checkPersist(); + } catch (IOException e) { + LOGGER.error(e.getMessage()); + } return currVersion; } @@ -59,16 +65,20 @@ public long currVersion() { return currVersion; } - private void checkPersist() { + private void checkPersist() throws IOException { if ((currVersion - prevVersion) >= SAVE_INTERVAL) { persist(); } } - private void persist() { - File oldFile = new File(directoryPath,FILE_PREFIX + prevVersion); + private void persist() throws IOException { + File oldFile = new File(directoryPath, FILE_PREFIX + prevVersion); File newFile = new File(directoryPath, FILE_PREFIX + currVersion); - oldFile.renameTo(newFile); + if (!oldFile.renameTo(newFile)) { + throw new IOException(String + .format("can not rename file %s to file %s", oldFile.getAbsolutePath(), + newFile.getAbsolutePath())); + } prevVersion = currVersion; } diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/version/SimpleFileVersionControllerTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/version/SimpleFileVersionControllerTest.java index b33b5065a500..032500dac18d 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/engine/version/SimpleFileVersionControllerTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/version/SimpleFileVersionControllerTest.java @@ -20,6 +20,7 @@ import java.io.IOException; import org.apache.commons.io.FileUtils; +import org.junit.Assert; import org.junit.Test; import static org.apache.iotdb.db.engine.version.SimpleFileVersionController.SAVE_INTERVAL; @@ -31,7 +32,9 @@ public void test() throws IOException { String tempFilePath = "version.tmp"; try { - new File(tempFilePath).mkdir(); + if (!new File(tempFilePath).mkdir()) { + Assert.fail("can not create version.tmp folder"); + } VersionController versionController = new SimpleFileVersionController(tempFilePath); assertEquals(versionController.currVersion(), SAVE_INTERVAL); for (int i = 0; i < 150; i++) { From 0986affecaa47096df4c98f22af7808d930312e9 Mon Sep 17 00:00:00 2001 From: xiangdong huang Date: Sat, 16 Feb 2019 20:58:38 +0800 Subject: [PATCH 21/24] fix Apache Header in some files --- .../db/engine/memtable/PrimitiveMemTable.java | 1 + .../BufferWriteProcessorNewTest.java | 3 +- .../bufferwrite/BufferWriteProcessorTest.java | 3 +- .../modification/DeletionFileNodeTest.java | 29 ++++++++++--------- .../modification/ModificationFileTest.java | 29 ++++++++++--------- .../io/LocalTextModificationAccessorTest.java | 29 ++++++++++--------- .../SimpleFileVersionControllerTest.java | 29 ++++++++++--------- .../version/SysTimeVersionControllerTest.java | 29 ++++++++++--------- 8 files changed, 83 insertions(+), 69 deletions(-) diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/PrimitiveMemTable.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/PrimitiveMemTable.java index e84fe6e49d30..455196a37848 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/PrimitiveMemTable.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/PrimitiveMemTable.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.iotdb.db.engine.memtable; import java.util.HashMap; diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessorNewTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessorNewTest.java index fc3ded2ee73a..8299d7508dba 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessorNewTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessorNewTest.java @@ -1,6 +1,4 @@ /** - * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) - * * 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 @@ -18,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.iotdb.db.engine.bufferwrite; import static org.junit.Assert.assertEquals; diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessorTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessorTest.java index cef6a2b5ad9c..6b3ccbec8a24 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessorTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessorTest.java @@ -1,6 +1,4 @@ /** - * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) - * * 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 @@ -18,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.iotdb.db.engine.bufferwrite; import static org.junit.Assert.assertEquals; diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java index 8b833d031e25..528c62ebe745 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java @@ -1,17 +1,20 @@ /** - * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) - *

- * Licensed 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. + * 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.modification; diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/ModificationFileTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/ModificationFileTest.java index 6907a3d8d63d..35d5dfd29989 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/ModificationFileTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/ModificationFileTest.java @@ -1,17 +1,20 @@ /** - * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) - *

- * Licensed 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. + * 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.modification; diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/io/LocalTextModificationAccessorTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/io/LocalTextModificationAccessorTest.java index 95bec53af9ac..91f92cc0185b 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/io/LocalTextModificationAccessorTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/io/LocalTextModificationAccessorTest.java @@ -1,17 +1,20 @@ /** - * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) - *

- * Licensed 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. + * 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.modification.io; diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/version/SimpleFileVersionControllerTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/version/SimpleFileVersionControllerTest.java index 032500dac18d..fe40ee991bdd 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/engine/version/SimpleFileVersionControllerTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/version/SimpleFileVersionControllerTest.java @@ -1,17 +1,20 @@ /** - * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) - *

- * Licensed 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. + * 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.version; diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/version/SysTimeVersionControllerTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/version/SysTimeVersionControllerTest.java index c7214bada033..4e063adb106a 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/engine/version/SysTimeVersionControllerTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/version/SysTimeVersionControllerTest.java @@ -1,17 +1,20 @@ /** - * Copyright © 2019 Apache IoTDB(incubating) (dev@iotdb.apache.org) - *

- * Licensed 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. + * 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.version; From ba941fddc2d5bd3a7c75b9ab80eeb39474fc5b10 Mon Sep 17 00:00:00 2001 From: jt Date: Sat, 16 Feb 2019 21:31:47 +0800 Subject: [PATCH 22/24] set the initial deletedAt in PageReader to -1 --- .../org/apache/iotdb/tsfile/read/reader/page/PageReader.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/page/PageReader.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/page/PageReader.java index 13e4e0e192bf..a686d997ab92 100644 --- a/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/page/PageReader.java +++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/read/reader/page/PageReader.java @@ -48,7 +48,7 @@ public class PageReader { private Filter filter = null; - private long deletedAt; + private long deletedAt = -1; public PageReader(ByteBuffer pageData, TSDataType dataType, Decoder valueDecoder, Decoder timeDecoder, From 74a14561a399b0ee65ed06f14addc9d144e92e36 Mon Sep 17 00:00:00 2001 From: jt Date: Sat, 16 Feb 2019 22:11:26 +0800 Subject: [PATCH 23/24] secure some file streams fix some uts with incorrect answers --- .../db/engine/filenode/FileNodeProcessor.java | 9 +++++++++ .../io/LocalTextModificationAccessor.java | 3 ++- .../modification/DeletionFileNodeTest.java | 16 ++++++++++------ .../engine/modification/DeletionQueryTest.java | 2 +- .../iotdb/db/integration/IoTDBCompleteIT.java | 4 ++-- .../db/integration/IoTDBMetadataFetchIT.java | 2 +- 6 files changed, 25 insertions(+), 11 deletions(-) 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 51a7468fb477..5ca65c56bd0c 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 @@ -1902,6 +1902,15 @@ public void delete() throws ProcessorException { } closeBufferWrite(); closeOverflow(); + for (IntervalFileNode fileNode : newFileNodes) { + if (fileNode.getModFile() != null) { + try { + fileNode.getModFile().close(); + } catch (IOException e) { + throw new FileNodeProcessorException(e); + } + } + } } @Override 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 33725402c476..ee9abaff74de 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 @@ -70,9 +70,10 @@ public Collection read() throws IOException { modificationList.add(decodeModification(line)); } } catch (IOException e) { - reader.close(); logger.error("An error occurred when reading modifications, and the remaining modifications " + "were ignored.", e); + } finally { + reader.close(); } return modificationList; } diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java index 8311f3ea8571..54fa267d8438 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java @@ -71,7 +71,7 @@ public void setup() throws MetadataArgsErrorException, MManager.getInstance().addPathToMTree(processorName + "." + measurements[i], dataType, encoding, args); FileNodeManager.getInstance().addTimeSeries(new Path(processorName, measurements[i]), dataType, - encoding, args); + encoding); } } @@ -140,11 +140,15 @@ public void testDeleteInBufferWriteFile() throws FileNodeManagerException, IOExc LocalTextModificationAccessor accessor = new LocalTextModificationAccessor(modFiles[0].getPath()); - Collection modifications = accessor.read(); - assertEquals(modifications.size(), 3); - int i = 0; - for (Modification modification : modifications) { - assertTrue(modification.equals(realModifications[i++])); + try { + Collection modifications = accessor.read(); + assertEquals(modifications.size(), 3); + int i = 0; + for (Modification modification : modifications) { + assertTrue(modification.equals(realModifications[i++])); + } + } finally { + accessor.close(); } } 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 2e069699bfc8..666b8df76f45 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 @@ -63,7 +63,7 @@ public void setup() throws MetadataArgsErrorException, MManager.getInstance().addPathToMTree(processorName + "." + measurements[i], dataType, encoding, args); FileNodeManager.getInstance().addTimeSeries(new Path(processorName, measurements[i]), dataType, - encoding, args); + encoding); } } diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBCompleteIT.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBCompleteIT.java index 3b5423a358c3..5f61c4bf6eee 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBCompleteIT.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBCompleteIT.java @@ -105,11 +105,11 @@ public void SimpleTest() throws ClassNotFoundException, SQLException { + " StorageGroup: root.vehicle \n" + " }\n" + " },\n" + " d5:{\n" + " s9:{\n" + " DataType: FLOAT,\n" + " Encoding: PLAIN,\n" - + " args: {compressor=SNAPPY, MAX_POINT_NUMBER=10},\n" + + " args: {MAX_POINT_NUMBER=10, compressor=SNAPPY},\n" + " StorageGroup: root.vehicle \n" + " }\n" + " },\n" + " d6:{\n" + " s10:{\n" + " DataType: DOUBLE,\n" + " Encoding: RLE,\n" - + " args: {compressor=UNCOMPRESSOR, MAX_POINT_NUMBER=10},\n" + + " args: {MAX_POINT_NUMBER=10, compressor=UNCOMPRESSOR},\n" + " StorageGroup: root.vehicle \n" + " }\n" + " }\n" + " }\n" + "}", "DELETE TIMESERIES root.vehicle.*", "SHOW TIMESERIES", diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBMetadataFetchIT.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBMetadataFetchIT.java index 04a4c1bf6d8f..69fc3c61186b 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBMetadataFetchIT.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBMetadataFetchIT.java @@ -366,7 +366,7 @@ private void ShowTimeseriesInJson() { + " StorageGroup: root.ln.wf01.wt01 \n" + " },\n" + " temperature:{\n" + " DataType: FLOAT,\n" + " Encoding: RLE,\n" - + " args: {compressor=SNAPPY, MAX_POINT_NUMBER=3},\n" + + " args: {MAX_POINT_NUMBER=3, compressor=SNAPPY},\n" + " StorageGroup: root.ln.wf01.wt01 \n" + " }\n" + " }\n" + " }\n" + " }\n" + "}"; From ba51118541bfd3d66ddb401975cade1a1c156271 Mon Sep 17 00:00:00 2001 From: jt Date: Sun, 17 Feb 2019 10:13:13 +0800 Subject: [PATCH 24/24] fix by comments --- .../bufferwrite/BufferWriteProcessor.java | 4 ++-- .../db/engine/filenode/FileNodeProcessor.java | 12 +++++++++- .../db/engine/memtable/AbstractMemTable.java | 2 +- .../iotdb/db/engine/memtable/IMemTable.java | 2 +- .../io/LocalTextModificationAccessor.java | 3 ++- .../engine/overflow/ioV2/OverflowSupport.java | 5 ++--- .../version/SimpleFileVersionController.java | 22 ++++++++++++++----- .../modification/DeletionFileNodeTest.java | 2 +- .../iotdb/db/integration/IoTDBCompleteIT.java | 4 ++-- .../db/integration/IoTDBMetadataFetchIT.java | 2 +- 10 files changed, 39 insertions(+), 19 deletions(-) diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessor.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessor.java index 7feac6b41ece..16a5c0d71364 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessor.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/bufferwrite/BufferWriteProcessor.java @@ -515,11 +515,11 @@ public WriteLogNode getLogNode() { * @param timestamp the upper-bound of deletion time. */ public void delete(String deviceId, String measurementId, long timestamp) { - workMemTable.delele(deviceId, measurementId, timestamp); + workMemTable.delete(deviceId, measurementId, timestamp); if (isFlush) { // flushing MemTable cannot be directly modified since another thread is reading it flushMemTable = flushMemTable.copy(); - flushMemTable.delele(deviceId, measurementId, timestamp); + flushMemTable.delete(deviceId, measurementId, timestamp); } } } 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 88952a552cb4..63eaa9fa18a5 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 @@ -1884,6 +1884,15 @@ public void delete() throws ProcessorException { } closeBufferWrite(); closeOverflow(); + for (IntervalFileNode fileNode : newFileNodes) { + if (fileNode.getModFile() != null) { + try { + fileNode.getModFile().close(); + } catch (IOException e) { + throw new FileNodeProcessorException(e); + } + } + } } @Override @@ -1951,7 +1960,8 @@ public void delete(String deviceId, String measurementId, long timestamp) throws currentIntervalFileNode.getModFile().write(deletion); } for (IntervalFileNode fileNode : newFileNodes) { - if(fileNode != currentIntervalFileNode && fileNode.containsDevice(deviceId)) { + if(fileNode != currentIntervalFileNode && fileNode.containsDevice(deviceId) + && fileNode.getStartTime(deviceId) <= timestamp) { fileNode.getModFile().write(deletion); } } diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java index ed95b930c375..bb95a25feca5 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/AbstractMemTable.java @@ -102,7 +102,7 @@ public TimeValuePairSorter query(String deviceId, String measurement, TSDataType } @Override - public void delele(String deviceId, String measurementId, long timestamp) { + public void delete(String deviceId, String measurementId, long timestamp) { Map deviceMap = memTableMap.get(deviceId); if (deviceMap != null) { IWritableMemChunk chunk = deviceMap.get(measurementId); diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java index efc906576b7d..4833ffac09e2 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/memtable/IMemTable.java @@ -54,7 +54,7 @@ void write(String deviceId, String measurement, TSDataType dataType, * @param measurementId the measurementId of the timeseries to be deleted. * @param timestamp the upper-bound of deletion time. */ - void delele(String deviceId, String measurementId, long timestamp); + void delete(String deviceId, String measurementId, long timestamp); /** * Make a copy of this MemTable. 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 33725402c476..ee9abaff74de 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 @@ -70,9 +70,10 @@ public Collection read() throws IOException { modificationList.add(decodeModification(line)); } } catch (IOException e) { - reader.close(); logger.error("An error occurred when reading modifications, and the remaining modifications " + "were ignored.", e); + } finally { + reader.close(); } return modificationList; } diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowSupport.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowSupport.java index ddf818f28e08..f4f1652d4e99 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowSupport.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/overflow/ioV2/OverflowSupport.java @@ -25,7 +25,6 @@ import org.apache.iotdb.db.engine.memtable.PrimitiveMemTable; import org.apache.iotdb.db.engine.memtable.TimeValuePairSorter; import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType; -import org.apache.iotdb.tsfile.read.common.BatchData; import org.apache.iotdb.tsfile.write.record.TSRecord; import org.apache.iotdb.tsfile.write.record.datapoint.DataPoint; @@ -74,9 +73,9 @@ public void update(String deviceId, String measurementId, long startTime, long e public void delete(String deviceId, String measurementId, long timestamp, boolean isFlushing) { if (isFlushing) { memTable = memTable.copy(); - memTable.delele(deviceId, measurementId, timestamp); + memTable.delete(deviceId, measurementId, timestamp); } else { - memTable.delele(deviceId, measurementId, timestamp); + memTable.delete(deviceId, measurementId, timestamp); } } diff --git a/iotdb/src/main/java/org/apache/iotdb/db/engine/version/SimpleFileVersionController.java b/iotdb/src/main/java/org/apache/iotdb/db/engine/version/SimpleFileVersionController.java index 5a638067fedd..a38f4beee0b6 100644 --- a/iotdb/src/main/java/org/apache/iotdb/db/engine/version/SimpleFileVersionController.java +++ b/iotdb/src/main/java/org/apache/iotdb/db/engine/version/SimpleFileVersionController.java @@ -87,17 +87,27 @@ private void restore() throws IOException { File[] versionFiles = directory.listFiles((dir, name) -> name.startsWith(FILE_PREFIX)); File versionFile = null; if (versionFiles != null && versionFiles.length > 0) { - Arrays.sort(versionFiles, Comparator.comparing(File::getName)); - versionFile = versionFiles[versionFiles.length - 1]; - for(int i = 0; i < versionFiles.length - 1; i ++) { - versionFiles[i].delete(); + long maxVersion = 0; + int maxVersionIndex = 0; + for (int i = 0; i < versionFiles.length; i ++) { + // extract version from "Version-123456" + long fileVersion = Long.parseLong(versionFiles[i].getName().split("-")[1]); + if (fileVersion > maxVersion) { + maxVersion = fileVersion; + maxVersionIndex = i; + } + } + prevVersion = maxVersion; + for(int i = 0; i < versionFiles.length; i ++) { + if (i != maxVersionIndex) { + versionFiles[i].delete(); + } } } else { versionFile = new File(directory, FILE_PREFIX + "0"); + prevVersion = 0; new FileOutputStream(versionFile).close(); } - // extract version from "Version-123456" - prevVersion = Long.parseLong(versionFile.getName().split("-")[1]); // prevent overlapping in case of failure currVersion = prevVersion + SAVE_INTERVAL; persist(); diff --git a/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java b/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java index 528c62ebe745..70281fba9f59 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/engine/modification/DeletionFileNodeTest.java @@ -71,7 +71,7 @@ public void setup() throws MetadataArgsErrorException, MManager.getInstance().addPathToMTree(processorName + "." + measurements[i], dataType, encoding, args); FileNodeManager.getInstance().addTimeSeries(new Path(processorName, measurements[i]), dataType, - encoding, args); + encoding); } } diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBCompleteIT.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBCompleteIT.java index 163a083d773f..0bb646b7a427 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBCompleteIT.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBCompleteIT.java @@ -104,11 +104,11 @@ public void SimpleTest() throws ClassNotFoundException, SQLException { + " StorageGroup: root.vehicle \n" + " }\n" + " },\n" + " d5:{\n" + " s9:{\n" + " DataType: FLOAT,\n" + " Encoding: PLAIN,\n" - + " args: {compressor=SNAPPY, MAX_POINT_NUMBER=10},\n" + + " args: {MAX_POINT_NUMBER=10, compressor=SNAPPY},\n" + " StorageGroup: root.vehicle \n" + " }\n" + " },\n" + " d6:{\n" + " s10:{\n" + " DataType: DOUBLE,\n" + " Encoding: RLE,\n" - + " args: {compressor=UNCOMPRESSOR, MAX_POINT_NUMBER=10},\n" + + " args: {MAX_POINT_NUMBER=10, compressor=UNCOMPRESSOR},\n" + " StorageGroup: root.vehicle \n" + " }\n" + " }\n" + " }\n" + "}", "DELETE TIMESERIES root.vehicle.*", "SHOW TIMESERIES", diff --git a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBMetadataFetchIT.java b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBMetadataFetchIT.java index a3fa58436c7b..94cc02c3550b 100644 --- a/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBMetadataFetchIT.java +++ b/iotdb/src/test/java/org/apache/iotdb/db/integration/IoTDBMetadataFetchIT.java @@ -366,7 +366,7 @@ private void ShowTimeseriesInJson() { + " StorageGroup: root.ln.wf01.wt01 \n" + " },\n" + " temperature:{\n" + " DataType: FLOAT,\n" + " Encoding: RLE,\n" - + " args: {compressor=SNAPPY, MAX_POINT_NUMBER=3},\n" + + " args: {MAX_POINT_NUMBER=3, compressor=SNAPPY},\n" + " StorageGroup: root.ln.wf01.wt01 \n" + " }\n" + " }\n" + " }\n" + " }\n" + "}";