提交 f8f9b096 编写于 作者: T Tian Jiang

add MergedTsFileSplitter

上级 1357cede
......@@ -208,6 +208,21 @@ public class AlignedChunkData implements ChunkData {
}
}
@Override
public void writeDecodePage(long[] times, Object[] values, int start, int end)
throws IOException {
pageNumbers.set(pageNumbers.size() - 1, pageNumbers.get(pageNumbers.size() - 1) + 1);
satisfiedLengthQueue.offer(end - start);
// serialize needDecode==true
dataSize += ReadWriteIOUtils.write(true, stream);
dataSize += ReadWriteIOUtils.write(end - start, stream);
for (int i = start; i < end; i++) {
long time = times[i];
dataSize += ReadWriteIOUtils.write(time, stream);
}
}
public void writeDecodeValuePage(long[] times, TsPrimitiveType[] values, TSDataType dataType)
throws IOException {
pageNumbers.set(pageNumbers.size() - 1, pageNumbers.get(pageNumbers.size() - 1) + 1);
......@@ -419,6 +434,11 @@ public class AlignedChunkData implements ChunkData {
stream.close();
}
@Override
public String firstMeasurement() {
return chunkHeaderList.get(0).getMeasurementID();
}
@Override
public String toString() {
return "AlignedChunkData{"
......
......@@ -45,6 +45,10 @@ public interface ChunkData extends TsFileData {
void writeDecodePage(long[] times, Object[] values, int satisfiedLength) throws IOException;
void writeDecodePage(long[] times, Object[] values, int start, int end) throws IOException;
String firstMeasurement();
@Override
default boolean isModification() {
return false;
......
/*
* 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.queryengine.execution.load;
public class LoadUtils {
private LoadUtils() {
// Util class
}
}
......@@ -191,6 +191,40 @@ public class NonAlignedChunkData implements ChunkData {
}
}
public void writeDecodePage(long[] times, Object[] values, int start, int end)
throws IOException {
pageNumber += 1;
dataSize += ReadWriteIOUtils.write(true, stream);
dataSize += ReadWriteIOUtils.write(end - start, stream);
for (int i = start; i < end; i++) {
dataSize += ReadWriteIOUtils.write(times[i], stream);
switch (chunkHeader.getDataType()) {
case INT32:
dataSize += ReadWriteIOUtils.write((int) values[i], stream);
break;
case INT64:
dataSize += ReadWriteIOUtils.write((long) values[i], stream);
break;
case FLOAT:
dataSize += ReadWriteIOUtils.write((float) values[i], stream);
break;
case DOUBLE:
dataSize += ReadWriteIOUtils.write((double) values[i], stream);
break;
case BOOLEAN:
dataSize += ReadWriteIOUtils.write((boolean) values[i], stream);
break;
case TEXT:
dataSize += ReadWriteIOUtils.write((Binary) values[i], stream);
break;
default:
throw new UnSupportedDataTypeException(
String.format("Data type %s is not supported.", chunkHeader.getDataType()));
}
}
}
private void deserializeTsFileData(InputStream stream) throws IOException, PageException {
if (needDecodeChunk) {
buildChunkWriter(stream);
......@@ -283,6 +317,11 @@ public class NonAlignedChunkData implements ChunkData {
stream.close();
}
@Override
public String firstMeasurement() {
return chunkHeader.getMeasurementID();
}
@Override
public String toString() {
return "NonAlignedChunkData{"
......
......@@ -183,13 +183,13 @@ public class TsFileSplitter {
pageIndex2Times.put(pageIndex, times);
}
int satisfiedLength = 0;
int start = 0;
long endTime =
timePartitionSlot.getStartTime()
+ TimePartitionUtils.getTimePartitionInterval();
for (int i = 0; i < times.length; i++) {
if (times[i] >= endTime) {
chunkData.writeDecodePage(times, values, satisfiedLength);
chunkData.writeDecodePage(times, values, start, i);
if (isAligned) {
pageIndex2ChunkData
.computeIfAbsent(pageIndex, o -> new ArrayList<>())
......@@ -199,16 +199,15 @@ public class TsFileSplitter {
}
timePartitionSlot = TimePartitionUtils.getTimePartition(times[i]);
satisfiedLength = 0;
endTime =
timePartitionSlot.getStartTime()
+ TimePartitionUtils.getTimePartitionInterval();
chunkData =
ChunkData.createChunkData(isAligned, curDevice, header, timePartitionSlot);
start = i;
}
satisfiedLength += 1;
}
chunkData.writeDecodePage(times, values, satisfiedLength);
chunkData.writeDecodePage(times, values, start, times.length);
if (isAligned) {
pageIndex2ChunkData
.computeIfAbsent(pageIndex, o -> new ArrayList<>())
......
/*
* 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.queryengine.execution.load;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.SynchronousQueue;
import java.util.concurrent.TimeUnit;
import org.apache.iotdb.commons.concurrent.IoTDBThreadPoolFactory;
import org.apache.iotdb.commons.concurrent.IoTThreadFactory;
import org.junit.Test;
public class MergedTsFileSplitterTest extends TestBase {
private List<TsFileData> resultSet = new ArrayList<>();
@Test
public void testSplit() throws IOException {
long start = System.currentTimeMillis();
MergedTsFileSplitter splitter = new MergedTsFileSplitter(files, this::consumeSplit,
IoTDBThreadPoolFactory.newThreadPool(16, Integer.MAX_VALUE, 20, TimeUnit.SECONDS,
new SynchronousQueue<>(),
new IoTThreadFactory("MergedTsFileSplitter"), "MergedTsFileSplitter"));
try {
splitter.splitTsFileByDataPartition();
for (TsFileData tsFileData : resultSet) {
// System.out.println(tsFileData);
}
} finally {
splitter.close();
}
System.out.printf("%d splits after %dms\n", resultSet.size(),
System.currentTimeMillis() - start);
}
public boolean consumeSplit(TsFileData data) {
resultSet.add(data);
if (resultSet.size() % 1000 == 0) {
System.out.printf("%d chunks split\n", resultSet.size());
System.out.printf("Maxmem: %d, freemem: %d\n", Runtime.getRuntime().maxMemory(), Runtime.getRuntime().freeMemory());
}
return true;
}
}
/*
* 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.queryengine.execution.load;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.apache.iotdb.db.utils.TimePartitionUtils;
import org.apache.iotdb.tsfile.exception.write.WriteProcessException;
import org.apache.iotdb.tsfile.file.metadata.enums.TSDataType;
import org.apache.iotdb.tsfile.read.common.Path;
import org.apache.iotdb.tsfile.utils.TsFileGeneratorUtils;
import org.apache.iotdb.tsfile.write.TsFileWriter;
import org.apache.iotdb.tsfile.write.record.TSRecord;
import org.apache.iotdb.tsfile.write.record.datapoint.DoubleDataPoint;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
import org.junit.After;
import org.junit.Before;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class TestBase {
private static final Logger logger = LoggerFactory.getLogger(TestBase.class);
public static final String BASE_OUTPUT_PATH = "target".concat(File.separator);
public static final String PARTIAL_PATH_STRING =
"%s" + File.separator + "%d" + File.separator + "%d" + File.separator;
public static final String TEST_TSFILE_PATH =
BASE_OUTPUT_PATH + "testTsFile".concat(File.separator) + PARTIAL_PATH_STRING;
protected int fileNum = 100;
// series number of each file, sn non-aligned series and 1 aligned series with sn measurements
protected int seriesNum = 1000;
// number of chunks of each series in a file, each series has only one chunk in a file
protected double chunkTimeRangeRatio = 0.3;
// the interval between two consecutive points of a series
protected long pointInterval = 50_000;
protected List<File> files = new ArrayList<>();
@Before
public void setup() throws IOException, WriteProcessException {
setupFiles();
logger.info("Files set up");
}
@After
public void cleanup() {
for (File file : files) {
file.delete();
}
}
public void setupFiles() throws IOException, WriteProcessException {
for (int i = 0; i < fileNum; i++) {
File file = new File(getTestTsFilePath("root.sg1", 0, 0, i));
files.add(file);
try (TsFileWriter writer = new TsFileWriter(file)) {
// 3 non-aligned series under d1 and 1 aligned series with 3 measurements under d2
for (int sn = 0; sn < seriesNum; sn++) {
writer.registerTimeseries(
new Path("d1"), new MeasurementSchema("s" + sn, TSDataType.DOUBLE));
}
List<MeasurementSchema> alignedSchemas = new ArrayList<>();
for (int sn = 0; sn < seriesNum; sn++) {
alignedSchemas.add(new MeasurementSchema("s" + sn, TSDataType.DOUBLE));
}
writer.registerAlignedTimeseries(new Path("d2"), alignedSchemas);
long timePartitionInterval = TimePartitionUtils.getTimePartitionInterval();
long chunkTimeRange = (long) (timePartitionInterval * chunkTimeRangeRatio);
int chunkPointNum = (int) (chunkTimeRange / pointInterval);
for (int pn = 0; pn < chunkPointNum; pn++) {
long currTime = chunkTimeRange * fileNum + pointInterval * pn;
TSRecord record = new TSRecord(currTime, "d1");
for (int sn = 0; sn < seriesNum; sn++) {
record.addTuple(new DoubleDataPoint("s" + sn, pn * 1.0));
}
writer.write(record);
record.deviceId = "d2";
writer.writeAligned(record);
}
writer.flushAllChunkGroups();
}
}
}
public static String getTestTsFilePath(
String logicalStorageGroupName,
long VirtualStorageGroupId,
long TimePartitionId,
long tsFileVersion) {
String filePath =
String.format(
TEST_TSFILE_PATH, logicalStorageGroupName, VirtualStorageGroupId, TimePartitionId);
return TsFileGeneratorUtils.getTsFilePath(filePath, tsFileVersion);
}
}
/*
* 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.queryengine.execution.load;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import org.junit.Test;
public class TsFileSplitterTest extends TestBase{
private List<TsFileData> resultSet = new ArrayList<>();
@Test
public void testSplit() throws IOException {
long start = System.currentTimeMillis();
for (File file : files) {
TsFileSplitter splitter = new TsFileSplitter(file, this::consumeSplit);
splitter.splitTsFileByDataPartition();
}
for (TsFileData tsFileData : resultSet) {
// System.out.println(tsFileData);
}
System.out.printf("%d splits after %dms\n", resultSet.size(), System.currentTimeMillis() - start);
}
public boolean consumeSplit(TsFileData data) {
resultSet.add(data);
return true;
}
}
......@@ -107,7 +107,7 @@ public class WrappedThreadPoolExecutor extends ThreadPoolExecutor
Thread.currentThread().interrupt();
}
}
if (t != null) {
if (t != null && !(t instanceof CancellationException)) {
logger.error("Exception in thread pool {}", mbeanName, t);
}
}
......
......@@ -321,4 +321,8 @@ public class ChunkHeader {
public void increasePageNums(int i) {
numOfPages += i;
}
public boolean hasStatistic() {
return (getChunkType() & 0x3F) == MetaMarker.CHUNK_HEADER;
}
}
......@@ -65,13 +65,22 @@ public class PageHeader {
return new PageHeader(uncompressedSize, compressedSize, statistics);
}
// for compatibility, the previous implementation does not provide parameter 'hasStatistic'
public static PageHeader deserializeFrom(ByteBuffer buffer, TSDataType dataType) {
return deserializeFrom(buffer, dataType, true);
}
public static PageHeader deserializeFrom(
ByteBuffer buffer, TSDataType dataType, boolean hasStatistic) {
int uncompressedSize = ReadWriteForEncodingUtils.readUnsignedVarInt(buffer);
if (uncompressedSize == 0) { // Empty Page
return new PageHeader(0, 0, null);
}
int compressedSize = ReadWriteForEncodingUtils.readUnsignedVarInt(buffer);
Statistics<? extends Serializable> statistics = Statistics.deserialize(buffer, dataType);
Statistics<? extends Serializable> statistics = null;
if (hasStatistic) {
statistics = Statistics.deserialize(buffer, dataType);
}
return new PageHeader(uncompressedSize, compressedSize, statistics);
}
......
......@@ -22,7 +22,6 @@ package org.apache.iotdb.tsfile.read;
import org.apache.iotdb.tsfile.common.conf.TSFileConfig;
import org.apache.iotdb.tsfile.common.conf.TSFileDescriptor;
import org.apache.iotdb.tsfile.common.constant.TsFileConstant;
import org.apache.iotdb.tsfile.compress.IUnCompressor;
import org.apache.iotdb.tsfile.encoding.decoder.Decoder;
import org.apache.iotdb.tsfile.exception.TsFileRuntimeException;
import org.apache.iotdb.tsfile.exception.TsFileStatisticsMistakesException;
......@@ -59,6 +58,7 @@ import org.apache.iotdb.tsfile.utils.BloomFilter;
import org.apache.iotdb.tsfile.utils.Pair;
import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils;
import org.apache.iotdb.tsfile.utils.ReadWriteIOUtils;
import org.apache.iotdb.tsfile.utils.TsFileUtils;
import org.apache.iotdb.tsfile.utils.TsPrimitiveType;
import org.apache.iotdb.tsfile.write.schema.IMeasurementSchema;
import org.apache.iotdb.tsfile.write.schema.MeasurementSchema;
......@@ -1397,15 +1397,7 @@ public class TsFileSequenceReader implements AutoCloseable {
}
public ByteBuffer readPage(PageHeader header, CompressionType type) throws IOException {
ByteBuffer buffer = readData(-1, header.getCompressedSize());
if (header.getUncompressedSize() == 0 || type == CompressionType.UNCOMPRESSED) {
return buffer;
} // FIXME if the buffer is not array-implemented.
IUnCompressor unCompressor = IUnCompressor.getUnCompressor(type);
ByteBuffer uncompressedBuffer = ByteBuffer.allocate(header.getUncompressedSize());
unCompressor.uncompress(
buffer.array(), buffer.position(), buffer.remaining(), uncompressedBuffer.array(), 0);
return uncompressedBuffer;
return TsFileUtils.uncompressPage(header, type, readData(-1, header.getCompressedSize()));
}
/**
......
......@@ -17,10 +17,14 @@
*/
package org.apache.iotdb.tsfile.utils;
import org.apache.iotdb.tsfile.compress.IUnCompressor;
import org.apache.iotdb.tsfile.file.header.PageHeader;
import org.apache.iotdb.tsfile.file.metadata.enums.CompressionType;
import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
import java.io.File;
import java.io.IOException;
import java.nio.ByteBuffer;
public class TsFileUtils {
......@@ -55,4 +59,16 @@ public class TsFileUtils {
File folder = tsFile.getParentFile().getParentFile().getParentFile().getParentFile();
return folder.getName().equals("sequence");
}
public static ByteBuffer uncompressPage(
PageHeader header, CompressionType type, ByteBuffer buffer) throws IOException {
if (header.getUncompressedSize() == 0 || type == CompressionType.UNCOMPRESSED) {
return buffer;
} // FIXME if the buffer is not array-implemented.
IUnCompressor unCompressor = IUnCompressor.getUnCompressor(type);
ByteBuffer uncompressedBuffer = ByteBuffer.allocate(header.getUncompressedSize());
unCompressor.uncompress(
buffer.array(), buffer.position(), buffer.remaining(), uncompressedBuffer.array(), 0);
return uncompressedBuffer;
}
}
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册