Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -114,14 +114,20 @@ public TimeseriesMetadata(TimeseriesMetadata timeseriesMetadata) {
}

public static TimeseriesMetadata deserializeFrom(ByteBuffer buffer, boolean needChunkMetadata) {
return deserializeFrom(buffer, needChunkMetadata, needChunkMetadata);
}

public static TimeseriesMetadata deserializeFrom(
ByteBuffer buffer, boolean needChunkMetadataForNonBlob, boolean needChunkMetadataForBlob) {
TimeseriesMetadata timeseriesMetaData = new TimeseriesMetadata();
timeseriesMetaData.setTimeSeriesMetadataType(ReadWriteIOUtils.readByte(buffer));
timeseriesMetaData.setMeasurementId(ReadWriteIOUtils.readVarIntString(buffer));
timeseriesMetaData.setTsDataType(ReadWriteIOUtils.readDataType(buffer));
int chunkMetaDataListDataSize = ReadWriteForEncodingUtils.readUnsignedVarInt(buffer);
timeseriesMetaData.setDataSizeOfChunkMetaDataList(chunkMetaDataListDataSize);
timeseriesMetaData.setStatistics(Statistics.deserialize(buffer, timeseriesMetaData.dataType));
if (needChunkMetadata) {
if ((timeseriesMetaData.getTsDataType() != TSDataType.BLOB && needChunkMetadataForNonBlob)
|| (timeseriesMetaData.getTsDataType() == TSDataType.BLOB && needChunkMetadataForBlob)) {
ByteBuffer byteBuffer = buffer.slice();
byteBuffer.limit(chunkMetaDataListDataSize);
timeseriesMetaData.chunkMetadataList = new ArrayList<>();
Expand All @@ -138,6 +144,14 @@ public static TimeseriesMetadata deserializeFrom(ByteBuffer buffer, boolean need

public static TimeseriesMetadata deserializeFrom(
TsFileInput tsFileInput, boolean needChunkMetadata) throws IOException {
return deserializeFrom(tsFileInput, needChunkMetadata, needChunkMetadata);
}

public static TimeseriesMetadata deserializeFrom(
TsFileInput tsFileInput,
boolean needChunkMetadataForNonBlob,
boolean needChunkMetadataForBlob)
throws IOException {
InputStream inputStream = tsFileInput.wrapAsInputStream();
TimeseriesMetadata timeseriesMetaData = new TimeseriesMetadata();
timeseriesMetaData.setTimeSeriesMetadataType(ReadWriteIOUtils.readByte(inputStream));
Expand All @@ -148,7 +162,8 @@ public static TimeseriesMetadata deserializeFrom(
timeseriesMetaData.setStatistics(
Statistics.deserialize(inputStream, timeseriesMetaData.dataType));
long startOffset = tsFileInput.position();
if (needChunkMetadata) {
if ((timeseriesMetaData.getTsDataType() != TSDataType.BLOB && needChunkMetadataForNonBlob)
|| (timeseriesMetaData.getTsDataType() == TSDataType.BLOB && needChunkMetadataForBlob)) {
timeseriesMetaData.chunkMetadataList = new ArrayList<>();
while (tsFileInput.position() < startOffset + chunkMetaDataListDataSize) {
timeseriesMetaData.chunkMetadataList.add(
Expand All @@ -168,6 +183,14 @@ public static TimeseriesMetadata deserializeFrom(
*/
public static TimeseriesMetadata deserializeFrom(
ByteBuffer buffer, Set<String> excludedMeasurements, boolean needChunkMetadata) {
return deserializeFrom(buffer, excludedMeasurements, needChunkMetadata, needChunkMetadata);
}

public static TimeseriesMetadata deserializeFrom(
ByteBuffer buffer,
Set<String> excludedMeasurements,
boolean needChunkMetadataForNonBlob,
boolean needChunkMetadataForBlob) {
byte timeseriesType = ReadWriteIOUtils.readByte(buffer);
String measurementID = ReadWriteIOUtils.readVarIntString(buffer);
TSDataType tsDataType = ReadWriteIOUtils.readDataType(buffer);
Expand All @@ -181,7 +204,9 @@ public static TimeseriesMetadata deserializeFrom(
timeseriesMetaData.setDataSizeOfChunkMetaDataList(chunkMetaDataListDataSize);
timeseriesMetaData.setStatistics(statistics);

if (!excludedMeasurements.contains(measurementID) && needChunkMetadata) {
if (!excludedMeasurements.contains(measurementID)
&& ((tsDataType != TSDataType.BLOB && needChunkMetadataForNonBlob)
|| (tsDataType == TSDataType.BLOB && needChunkMetadataForBlob))) {
// measurement is not in the excluded set and need chunk metadata
ByteBuffer byteBuffer = buffer.slice();
byteBuffer.limit(chunkMetaDataListDataSize);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -76,22 +76,22 @@ public void update(long[] time, int batchSize, int arrayOffset) {

@Override
public Long getMinValue() {
throw new StatisticsClassException(String.format(STATS_UNSUPPORTED_MSG, TIME, "min value"));
return getStartTime();
}

@Override
public Long getMaxValue() {
throw new StatisticsClassException(String.format(STATS_UNSUPPORTED_MSG, TIME, "max value"));
return getEndTime();
}

@Override
public Long getFirstValue() {
throw new StatisticsClassException(String.format(STATS_UNSUPPORTED_MSG, TIME, "first value"));
return getStartTime();
}

@Override
public Long getLastValue() {
throw new StatisticsClassException(String.format(STATS_UNSUPPORTED_MSG, TIME, "last value"));
return getEndTime();
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -75,16 +75,19 @@
import java.io.IOException;
import java.io.Serializable;
import java.nio.ByteBuffer;
import java.util.ArrayDeque;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
import java.util.Deque;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.NoSuchElementException;
import java.util.Objects;
import java.util.Queue;
Expand Down Expand Up @@ -1252,11 +1255,32 @@ private void generateMetadataIndex(
Map<IDeviceID, List<TimeseriesMetadata>> timeseriesMetadataMap,
boolean needChunkMetadata)
throws IOException {
generateMetadataIndex(
metadataIndex,
buffer,
deviceId,
type,
timeseriesMetadataMap,
needChunkMetadata,
needChunkMetadata);
}

private void generateMetadataIndex(
IMetadataIndexEntry metadataIndex,
ByteBuffer buffer,
IDeviceID deviceId,
MetadataIndexNodeType type,
Map<IDeviceID, List<TimeseriesMetadata>> timeseriesMetadataMap,
boolean needChunkMetadataForNonBlob,
boolean needChunkMetadataForBlob)
throws IOException {
try {
if (type.equals(MetadataIndexNodeType.LEAF_MEASUREMENT)) {
List<TimeseriesMetadata> timeseriesMetadataList = new ArrayList<>();
while (buffer.hasRemaining()) {
timeseriesMetadataList.add(TimeseriesMetadata.deserializeFrom(buffer, needChunkMetadata));
timeseriesMetadataList.add(
TimeseriesMetadata.deserializeFrom(
buffer, needChunkMetadataForNonBlob, needChunkMetadataForBlob));
}
timeseriesMetadataMap
.computeIfAbsent(deviceId, k -> new ArrayList<>())
Expand Down Expand Up @@ -1284,7 +1308,8 @@ private void generateMetadataIndex(
deviceId,
metadataIndexNode.getNodeType(),
timeseriesMetadataMap,
needChunkMetadata);
needChunkMetadataForNonBlob,
needChunkMetadataForBlob);
} else {
// when the buffer length is over than Integer.MAX_VALUE,
// using tsFileInput to get timeseriesMetadataList
Expand All @@ -1295,7 +1320,8 @@ private void generateMetadataIndex(
deviceId,
metadataIndexNode.getNodeType(),
timeseriesMetadataMap,
needChunkMetadata);
needChunkMetadataForNonBlob,
needChunkMetadataForBlob);
}
}
}
Expand All @@ -1316,13 +1342,35 @@ private void generateMetadataIndexUsingTsFileInput(
Map<IDeviceID, List<TimeseriesMetadata>> timeseriesMetadataMap,
boolean needChunkMetadata)
throws IOException {
generateMetadataIndexUsingTsFileInput(
metadataIndex,
start,
end,
deviceId,
type,
timeseriesMetadataMap,
needChunkMetadata,
needChunkMetadata);
}

private void generateMetadataIndexUsingTsFileInput(
IMetadataIndexEntry metadataIndex,
long start,
long end,
IDeviceID deviceId,
MetadataIndexNodeType type,
Map<IDeviceID, List<TimeseriesMetadata>> timeseriesMetadataMap,
boolean needChunkMetadataForNonBlob,
boolean needChunkMetadataForBlob)
throws IOException {
try {
tsFileInput.position(start);
if (type.equals(MetadataIndexNodeType.LEAF_MEASUREMENT)) {
List<TimeseriesMetadata> timeseriesMetadataList = new ArrayList<>();
while (tsFileInput.position() < end) {
timeseriesMetadataList.add(
TimeseriesMetadata.deserializeFrom(tsFileInput, needChunkMetadata));
TimeseriesMetadata.deserializeFrom(
tsFileInput, needChunkMetadataForNonBlob, needChunkMetadataForBlob));
}
timeseriesMetadataMap
.computeIfAbsent(deviceId, k -> new ArrayList<>())
Expand All @@ -1349,7 +1397,8 @@ private void generateMetadataIndexUsingTsFileInput(
deviceId,
metadataIndexNode.getNodeType(),
timeseriesMetadataMap,
needChunkMetadata);
needChunkMetadataForNonBlob,
needChunkMetadataForBlob);
}
}
} catch (StopReadTsFileByInterruptException e) {
Expand Down Expand Up @@ -1398,6 +1447,11 @@ public Map<IDeviceID, List<TimeseriesMetadata>> getAllTimeseriesMetadata(
return timeseriesMetadataMap;
}

public Iterator<Pair<IDeviceID, List<TimeseriesMetadata>>> iterAllTimeseriesMetadata(
boolean needChunkMetadataForNonBlob, boolean needChunkMetadataForBlob) throws IOException {
return new TimeseriesMetadataIterator(needChunkMetadataForNonBlob, needChunkMetadataForBlob);
}

/* This method will only deserialize the TimeseriesMetadata, not including chunk metadata list */
private List<TimeseriesMetadata> getDeviceTimeseriesMetadataWithoutChunkMetadata(IDeviceID device)
throws IOException {
Expand Down Expand Up @@ -2771,4 +2825,132 @@ public boolean equals(Object o) {
public int hashCode() {
return Objects.hash(file);
}

private class TimeseriesMetadataIterator
implements Iterator<Pair<IDeviceID, List<TimeseriesMetadata>>> {

private final Deque<MetadataIndexNode> nodeStack = new ArrayDeque<>();
private final boolean needChunkMetadataForNonBlob;
private final boolean needCHunkMetadataForBlob;
private Pair<IDeviceID, List<TimeseriesMetadata>> nextValue;
private MetadataIndexNode currentLeafDeviceNode;
private int currentLeafDeviceNodeIndex;

public TimeseriesMetadataIterator(
boolean needChunkMetadataForNonBlob, boolean needChunkMetadataForBlob) throws IOException {
this.needChunkMetadataForNonBlob = needChunkMetadataForNonBlob;
this.needCHunkMetadataForBlob = needChunkMetadataForBlob;
if (tsFileMetaData == null) {
readFileMetadata();
}

nodeStack.add(tsFileMetaData.getMetadataIndex());
}

@Override
public boolean hasNext() {
if (nextValue != null) {
return true;
}

try {
loadNextValue();
} catch (IOException e) {
logger.warn("Cannot read timeseries metadata from {},", file, e);
return false;
}
return nextValue != null;
}

@Override
public Pair<IDeviceID, List<TimeseriesMetadata>> next() {
if (!hasNext()) {
throw new NoSuchElementException();
}
Pair<IDeviceID, List<TimeseriesMetadata>> ret = nextValue;
nextValue = null;
return ret;
}

private void loadNextLeafDeviceNode() throws IOException {
while (!nodeStack.isEmpty()) {
MetadataIndexNode node = nodeStack.pop();
MetadataIndexNodeType nodeType = node.getNodeType();
if (nodeType.equals(MetadataIndexNodeType.LEAF_DEVICE)) {
currentLeafDeviceNode = node;
currentLeafDeviceNodeIndex = 0;
return;
}

List<IMetadataIndexEntry> childrenIndex = node.getChildren();
for (int i = 0; i < childrenIndex.size(); i++) {
long endOffset;
IMetadataIndexEntry childIndex = childrenIndex.get(i);
endOffset = node.getEndOffset();
if (i != childrenIndex.size() - 1) {
endOffset = childrenIndex.get(i + 1).getOffset();
}

MetadataIndexNode child;
if (endOffset - childIndex.getOffset() < Integer.MAX_VALUE) {
ByteBuffer buffer = readData(childIndex.getOffset(), endOffset);
child = MetadataIndexNode.deserializeFrom(buffer, true);
} else {
tsFileInput.position(childIndex.getOffset());
child = MetadataIndexNode.deserializeFrom(tsFileInput.wrapAsInputStream(), true);
}
nodeStack.push(child);
}
}
}

private void loadNextValue() throws IOException {
if (currentLeafDeviceNode == null
|| currentLeafDeviceNodeIndex >= currentLeafDeviceNode.getChildren().size()) {
currentLeafDeviceNode = null;
loadNextLeafDeviceNode();
}
if (currentLeafDeviceNode == null) {
return;
}

IMetadataIndexEntry childIndex =
currentLeafDeviceNode.getChildren().get(currentLeafDeviceNodeIndex);
int childNum = currentLeafDeviceNode.getChildren().size();
IDeviceID deviceId = ((DeviceMetadataIndexEntry) childIndex).getDeviceID();

Map<IDeviceID, List<TimeseriesMetadata>> nextValueMap = new HashMap<>(1);
long endOffset = currentLeafDeviceNode.getEndOffset();
if (currentLeafDeviceNodeIndex != childNum - 1) {
endOffset =
currentLeafDeviceNode.getChildren().get(currentLeafDeviceNodeIndex + 1).getOffset();
}
if (endOffset - childIndex.getOffset() < Integer.MAX_VALUE) {
ByteBuffer nextBuffer = readData(childIndex.getOffset(), endOffset);
generateMetadataIndex(
childIndex,
nextBuffer,
deviceId,
currentLeafDeviceNode.getNodeType(),
nextValueMap,
needChunkMetadataForNonBlob,
needCHunkMetadataForBlob);
} else {
// when the buffer length is over than Integer.MAX_VALUE,
// using tsFileInput to get timeseriesMetadataList
generateMetadataIndexUsingTsFileInput(
childIndex,
childIndex.getOffset(),
endOffset,
deviceId,
currentLeafDeviceNode.getNodeType(),
nextValueMap,
needChunkMetadataForNonBlob,
needCHunkMetadataForBlob);
}
currentLeafDeviceNodeIndex++;
Entry<IDeviceID, List<TimeseriesMetadata>> entry = nextValueMap.entrySet().iterator().next();
nextValue = new Pair<>(entry.getKey(), entry.getValue());
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -171,4 +171,8 @@ public Statistics getChunkStatistic() {
public long getRetainedSizeInBytes() {
return INSTANCE_SIZE + sizeOfByteArray(chunkData.capacity());
}

public boolean isSinglePageChunk() {
return (getHeader().getChunkType() & 0x3F) == MetaMarker.ONLY_ONE_PAGE_CHUNK_HEADER;
}
}
Loading