TsFile API
TsFile is a file format of Time Series used in IoTDB. This session introduces the usage of this file format.
TsFile library Installation
There are two ways to use TsFile in your own project.
- Use as jars: Compile the source codes and build to jars
git clone https://github.com/apache/iotdb.git
cd tsfile/
mvn clean package -Dmaven.test.skip=true
Then, all the jars are in folder named target/
. Import target/tsfile-0.12.0-jar-with-dependencies.jar
to your project.
Use as a maven dependency:
Compile source codes and deploy to your local repository in three steps:
Get the source codes
git clone https://github.com/apache/iotdb.git
Compile the source codes and deploy
cd tsfile/ mvn clean install -Dmaven.test.skip=true
add dependencies into your project:
<dependency> <groupId>org.apache.iotdb</groupId> <artifactId>tsfile</artifactId> <version>1.0.0</version> </dependency>
Or, you can download the dependencies from official Maven repository:
First, find your maven
settings.xml
on path:${username}\.m2\settings.xml
, add this<profile>
to<profiles>
:<profile> <id>allow-snapshots</id> <activation><activeByDefault>true</activeByDefault></activation> <repositories> <repository> <id>apache.snapshots</id> <name>Apache Development Snapshot Repository</name> <url>https://repository.apache.org/content/repositories/snapshots/</url> <releases> <enabled>false</enabled> </releases> <snapshots> <enabled>true</enabled> </snapshots> </repository> </repositories> </profile>
Then add dependencies into your project:
<dependency> <groupId>org.apache.iotdb</groupId> <artifactId>tsfile</artifactId> <version>1.0.0</version> </dependency>
TsFile Usage
This section demonstrates the detailed usages of TsFile.
Time-series Data
Time-series data is considered as a sequence of quadruples. A quadruple is defined as (device, measurement, time, value).
measurement: A physical or formal measurement that a time-series data takes, e.g., the temperature of a city, the
sales number of some goods or the speed of a train at different times. As a traditional sensor (like a thermometer) also
takes a single measurement and produce a time-series, we will use measurement and sensor interchangeably below.device: A device refers to an entity that takes several measurements (producing multiple time-series), e.g.,
a running train monitors its speed, oil meter, miles it has run, current passengers each is conveyed to a time-series dataset.
One Line of Data: In many industrial applications, a device normally contains more than one sensor and these sensors
may have values at the same timestamp, which is called one line of data.
Formally, one line of data consists of a device_id
, a timestamp which indicates the milliseconds since January 1,
1970, 00:00:00, and several data pairs composed of measurement_id
and corresponding value
. All data pairs in one
line belong to this device_id
and have the same timestamp. If one of the measurements
does not have a value
in the timestamp
, use a space instead(Actually, TsFile does not store null values). Its format is shown as follow:
device_id, timestamp, <measurement_id, value>...
An example is illustrated as follow. In this example, the data type of two measurements are INT32
, FLOAT
respectively.
device_1, 1490860659000, m1, 10, m2, 12.12
Write TsFile
A TsFile is generated by the following three steps and the complete code is given in the section "Example for writing TsFile".
construct a
TsFileWriter
instance.Here are the available constructors:
- Without pre-defined schema
public TsFileWriter(File file) throws IOException
- With pre-defined schema
public TsFileWriter(File file, Schema schema) throws IOException
This one is for using the HDFS file system.
TsFileOutput
can be an instance of classHDFSOutput
.public TsFileWriter(TsFileOutput output, Schema schema) throws IOException
If you want to set some TSFile configuration on your own, you could use param
config
. For example:TSFileConfig conf = new TSFileConfig(); conf.setTSFileStorageFs("HDFS"); TsFileWriter tsFileWriter = new TsFileWriter(file, schema, conf);
In this example, data files will be stored in HDFS, instead of local file system. If you'd like to store data files in local file system, you can use
conf.setTSFileStorageFs("LOCAL")
, which is also the default config.You can also config the ip and rpc port of your HDFS by
config.setHdfsIp(...)
andconfig.setHdfsPort(...)
. The default ip islocalhost
and default rpc port is9000
.Parameters:
file : The TsFile to write
schema : The file schemas, will be introduced in next part.
config : The config of TsFile.
add measurements
Or you can make an instance of class
Schema
first and pass this to the constructor of classTsFileWriter
The class
Schema
contains a map whose key is the name of one measurement schema, and the value is the schema itself.Here are the interfaces:
// Create an empty Schema or from an existing map public Schema() public Schema(Map<String, MeasurementSchema> measurements) // Use this two interfaces to add measurements public void registerMeasurement(MeasurementSchema descriptor) public void registerMeasurements(Map<String, MeasurementSchema> measurements) // Some useful getter and checker public TSDataType getMeasurementDataType(String measurementId) public MeasurementSchema getMeasurementSchema(String measurementId) public Map<String, MeasurementSchema> getAllMeasurementSchema() public boolean hasMeasurement(String measurementId)
You can always use the following interface in
TsFileWriter
class to add additional measurements:public void addMeasurement(MeasurementSchema measurementSchema) throws WriteProcessException
The class
MeasurementSchema
contains the information of one measurement, there are several constructors:public MeasurementSchema(String measurementId, TSDataType type, TSEncoding encoding) public MeasurementSchema(String measurementId, TSDataType type, TSEncoding encoding, CompressionType compressionType) public MeasurementSchema(String measurementId, TSDataType type, TSEncoding encoding, CompressionType compressionType, Map<String, String> props)
Parameters:
measurementID: The name of this measurement, typically the name of the sensor.
type: The data type, now support six types:
BOOLEAN
,INT32
,INT64
,FLOAT
,DOUBLE
,TEXT
;encoding: The data encoding.
compression: The data compression.
props: Properties for special data types.Such as
max_point_number
forFLOAT
andDOUBLE
,max_string_length
forTEXT
. Use as string pairs into a map such as ("max_point_number", "3").
Notice: Although one measurement name can be used in multiple deltaObjects, the properties cannot be changed. I.e.
it's not allowed to add one measurement name for multiple times with different type or encoding.
Here is a bad example:// The measurement "sensor_1" is float type addMeasurement(new MeasurementSchema("sensor_1", TSDataType.FLOAT, TSEncoding.RLE)); // This call will throw a WriteProcessException exception
addMeasurement(new MeasurementSchema("sensor_1", TSDataType.INT32, TSEncoding.RLE));
3. insert and write data continually.
Use this interface to create a new `TSRecord`(a timestamp and device pair).
```java
public TSRecord(long timestamp, String deviceId)
```
Then create a `DataPoint`(a measurement and value pair), and use the addTuple method to add the DataPoint to the correct
TsRecord.
Use this method to write
```java
public void write(TSRecord record) throws IOException, WriteProcessException
4. call `close` to finish this writing process.
```java
public void close() throws IOException
```
We are also able to write data into a closed TsFile.
1. Use `ForceAppendTsFileWriter` to open a closed file.
```java
public ForceAppendTsFileWriter(File file) throws IOException
```
2. call `doTruncate` truncate the part of Metadata
3. Then use `ForceAppendTsFileWriter` to construct a new `TsFileWriter`
```java
public TsFileWriter(TsFileIOWriter fileWriter) throws IOException
Please note, we should redo the step of adding measurements before writing new data to the TsFile.
Example for writing a TsFile
You should install TsFile to your local maven repository.
mvn clean install -pl tsfile -am -DskipTests
You could write a TsFile by constructing TSRecord if you have the non-aligned (e.g. not all sensors contain values) time series data.
A more thorough example can be found at /example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileWriteWithTSRecord.java
You could write a TsFile by constructing Tablet if you have the aligned time series data.
A more thorough example can be found at /example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileWriteWithTablet.java
You could write data into a closed TsFile by using ForceAppendTsFileWriter.
A more thorough example can be found at /example/tsfile/src/main/java/org/apache/iotdb/tsfile/TsFileForceAppendWrite.java
Interface for Reading TsFile
- Definition of Path
A path is a dot-separated string which uniquely identifies a time-series in TsFile, e.g., "root.area_1.device_1.sensor_1".
The last section "sensor_1" is called "measurementId" while the remaining parts "root.area_1.device_1" is called deviceId.
As mentioned above, the same measurement in different devices has the same data type and encoding, and devices are also unique.
In read interfaces, The parameter paths
indicates the measurements to be selected.
Path instance can be easily constructed through the class Path
. For example:
Path p = new Path("device_1.sensor_1");
We will pass an ArrayList of paths for final query call to support multiple paths.
List<Path> paths = new ArrayList<Path>();
paths.add(new Path("device_1.sensor_1"));
paths.add(new Path("device_1.sensor_3"));
Notice: When constructing a Path, the format of the parameter should be a dot-separated string, the last part will
be recognized as measurementId while the remaining parts will be recognized as deviceId.
Definition of Filter
Usage Scenario
Filter is used in TsFile reading process to select data satisfying one or more given condition(s).IExpression
TheIExpression
is a filter expression interface and it will be passed to our final query call.
We create one or more filter expressions and may use binary filter operators to link them to our final expression.Create a Filter Expression
There are two types of filters.
TimeFilter: A filter for
time
in time-series data.IExpression timeFilterExpr = new GlobalTimeExpression(TimeFilter);
Use the following relationships to get a
TimeFilter
object (value is a long int variable).Relationship Description TimeFilter.eq(value) Choose the time equal to the value TimeFilter.lt(value) Choose the time less than the value TimeFilter.gt(value) Choose the time greater than the value TimeFilter.ltEq(value) Choose the time less than or equal to the value TimeFilter.gtEq(value) Choose the time greater than or equal to the value TimeFilter.notEq(value) Choose the time not equal to the value TimeFilter.not(TimeFilter) Choose the time not satisfy another TimeFilter ValueFilter: A filter for
value
in time-series data.IExpression valueFilterExpr = new SingleSeriesExpression(Path, ValueFilter);
The usage of
ValueFilter
is the same as usingTimeFilter
, just to make sure that the type of the value
equal to the measurement's(defined in the path).
Binary Filter Operators
Binary filter operators can be used to link two single expressions.
- BinaryExpression.and(Expression, Expression): Choose the value satisfy for both expressions.
- BinaryExpression.or(Expression, Expression): Choose the value satisfy for at least one expression.
Filter Expression Examples
TimeFilterExpression Examples
IExpression timeFilterExpr = new GlobalTimeExpression(TimeFilter.eq(15)); // series time = 15
```java
IExpression timeFilterExpr = new GlobalTimeExpression(TimeFilter.ltEq(15)); // series time <= 15
IExpression timeFilterExpr = new GlobalTimeExpression(TimeFilter.lt(15)); // series time < 15
```java
IExpression timeFilterExpr = new GlobalTimeExpression(TimeFilter.gtEq(15)); // series time >= 15java
IExpression timeFilterExpr = new GlobalTimeExpression(TimeFilter.notEq(15)); // series time != 15
```java
IExpression timeFilterExpr = BinaryExpression.and(
new GlobalTimeExpression(TimeFilter.gtEq(15L)),
new GlobalTimeExpression(TimeFilter.lt(25L))); // 15 <= series time < 25
```java
IExpression timeFilterExpr = BinaryExpression.or(
new GlobalTimeExpression(TimeFilter.gtEq(15L)),
new GlobalTimeExpression(TimeFilter.lt(25L))); // series time >= 15 or series time < 25
```
- Read Interface
First, we open the TsFile and get a ReadOnlyTsFile
instance from a file path string path
.
TsFileSequenceReader reader = new TsFileSequenceReader(path);
ReadOnlyTsFile readTsFile = new ReadOnlyTsFile(reader);
Next, we prepare the path array and query expression, then get final QueryExpression
object by this interface:
QueryExpression queryExpression = QueryExpression.create(paths, statement);
The ReadOnlyTsFile class has two query
method to perform a query.
Method 1
public QueryDataSet query(QueryExpression queryExpression) throws IOException
Method 2
public QueryDataSet query(QueryExpression queryExpression, long partitionStartOffset, long partitionEndOffset) throws IOException
This method is designed for advanced applications such as the TsFile-Spark Connector.
params : For method 2, two additional parameters are added to support partial query:
partitionStartOffset
: start offset for a TsFilepartitionEndOffset
: end offset for a TsFile
What is Partial Query ?
In some distributed file systems(e.g. HDFS), a file is split into severval parts which are called "Blocks" and stored in different nodes. Executing a query paralleled in each nodes involved makes better efficiency. Thus Partial Query is needed. Paritial Query only selects the results stored in the part split by
QueryConstant.PARTITION_START_OFFSET
andQueryConstant.PARTITION_END_OFFSET
for a TsFile.
QueryDataset Interface
The query performed above will return a QueryDataset
object.
Here's the useful interfaces for user.
bool hasNext();
Return true if this dataset still has elements.
List<Path> getPaths()
Get the paths in this data set.
List<TSDataType> getDataTypes();
Get the data types. The class TSDataType is an enum class, the value will be one of the following:
BOOLEAN,
INT32,
INT64,
FLOAT,
DOUBLE,
TEXT;
RowRecord next() throws IOException;
Get the next record.
The class
RowRecord
consists of along
timestamp and aList<Field>
for data in different sensors,
we can use two getter methods to get them.long getTimestamp(); List<Field> getFields();
To get data from one Field, use these methods:
TSDataType getDataType(); Object getObjectValue();
Example for reading an existing TsFile
You should install TsFile to your local maven repository.
A more thorough example with query statement can be found at/tsfile/example/src/main/java/org/apache/iotdb/tsfile/TsFileRead.java
package org.apache.iotdb.tsfile;
import java.io.IOException;
import java.util.ArrayList;
import org.apache.iotdb.tsfile.read.ReadOnlyTsFile;
import org.apache.iotdb.tsfile.read.TsFileSequenceReader;
import org.apache.iotdb.tsfile.read.common.Path;
import org.apache.iotdb.tsfile.read.expression.IExpression;
import org.apache.iotdb.tsfile.read.expression.QueryExpression;
import org.apache.iotdb.tsfile.read.expression.impl.BinaryExpression;
import org.apache.iotdb.tsfile.read.expression.impl.GlobalTimeExpression;
import org.apache.iotdb.tsfile.read.expression.impl.SingleSeriesExpression;
import org.apache.iotdb.tsfile.read.filter.TimeFilter;
import org.apache.iotdb.tsfile.read.filter.ValueFilter;
import org.apache.iotdb.tsfile.read.query.dataset.QueryDataSet;
/**
* The class is to show how to read TsFile file named "test.tsfile".
* The TsFile file "test.tsfile" is generated from class TsFileWrite.
* Run TsFileWrite to generate the test.tsfile first
*/
public class TsFileRead {
private static void queryAndPrint(ArrayList<Path> paths, ReadOnlyTsFile readTsFile, IExpression statement)
throws IOException {
QueryExpression queryExpression = QueryExpression.create(paths, statement);
QueryDataSet queryDataSet = readTsFile.query(queryExpression);
while (queryDataSet.hasNext()) {
System.out.println(queryDataSet.next());
}
System.out.println("------------");
}
public static void main(String[] args) throws IOException {
// file path
String path = "test.tsfile";
// create reader and get the readTsFile interface
TsFileSequenceReader reader = new TsFileSequenceReader(path);
ReadOnlyTsFile readTsFile = new ReadOnlyTsFile(reader);
// use these paths(all sensors) for all the queries
ArrayList<Path> paths = new ArrayList<>();
paths.add(new Path("device_1.sensor_1"));
paths.add(new Path("device_1.sensor_2"));
paths.add(new Path("device_1.sensor_3"));
// no query statement
queryAndPrint(paths, readTsFile, null);
//close the reader when you left
reader.close();
}
}
Change TsFile Configuration
TSFileConfig config = TSFileDescriptor.getInstance().getConfig();
config.setXXX();