You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iotdb.apache.org by li...@apache.org on 2021/06/01 08:29:20 UTC

[iotdb] branch master updated: [IOTDB-1403] Dictionary encoding for TEXT (#3218)

This is an automated email from the ASF dual-hosted git repository.

liudw pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/iotdb.git


The following commit(s) were added to refs/heads/master by this push:
     new 622a9c4  [IOTDB-1403] Dictionary encoding for TEXT  (#3218)
622a9c4 is described below

commit 622a9c412e51d45b28325ebf0b3d89b22ef7aaae
Author: J.J. Liu <li...@gmail.com>
AuthorDate: Tue Jun 1 16:28:40 2021 +0800

    [IOTDB-1403] Dictionary encoding for TEXT  (#3218)
---
 .../antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4   |   7 +-
 client-cpp/src/main/Session.h                      |   4 +-
 client-py/iotdb/utils/IoTDBConstants.py            |   3 +-
 docs/SystemDesign/TsFile/Format.md                 |   2 +-
 docs/UserGuide/Data-Concept/Encoding.md            |   7 +-
 docs/zh/SystemDesign/TsFile/Format.md              |   2 +-
 docs/zh/UserGuide/Data-Concept/Encoding.md         |   8 +-
 .../org/apache/iotdb/db/utils/SchemaUtils.java     |   1 +
 .../iotdb/db/integration/IoTDBEncodingIT.java      |  56 ++++++++++
 .../test/java/org/apache/iotdb/db/sql/Cases.java   |  25 +++++
 .../iotdb/tsfile/encoding/decoder/Decoder.java     |   2 +
 .../tsfile/encoding/decoder/DictionaryDecoder.java |  86 +++++++++++++++
 .../tsfile/encoding/encoder/DictionaryEncoder.java | 115 +++++++++++++++++++++
 .../tsfile/encoding/encoder/TSEncodingBuilder.java |  18 ++++
 .../tsfile/file/metadata/enums/TSEncoding.java     |   4 +-
 .../encoding/decoder/DictionaryDecoderTest.java    |  85 +++++++++++++++
 16 files changed, 409 insertions(+), 16 deletions(-)

diff --git a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4 b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
index 5ce6135..18539c7 100644
--- a/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
+++ b/antlr/src/main/antlr4/org/apache/iotdb/db/qp/sql/SqlBase.g4
@@ -638,7 +638,7 @@ dateExpression
     ;
 
 encoding
-    : PLAIN | PLAIN_DICTIONARY | RLE | DIFF | TS_2DIFF | GORILLA | REGULAR
+    : PLAIN | DICTIONARY | RLE | DIFF | TS_2DIFF | GORILLA | REGULAR
     ;
 
 realLiteral
@@ -864,8 +864,8 @@ PLAIN
     : P L A I N
     ;
 
-PLAIN_DICTIONARY
-    : P L A I N '_' D I C T I O N A R Y
+DICTIONARY
+    : D I C T I O N A R Y
     ;
 
 RLE
@@ -884,7 +884,6 @@ GORILLA
     : G O R I L L A
     ;
 
-
 REGULAR
     : R E G U L A R
     ;
diff --git a/client-cpp/src/main/Session.h b/client-cpp/src/main/Session.h
index 7e1a8c3..3cbd320 100644
--- a/client-cpp/src/main/Session.h
+++ b/client-cpp/src/main/Session.h
@@ -105,14 +105,14 @@ namespace TSDataType{
 namespace TSEncoding {
     enum TSEncoding {
         PLAIN = 0,
-        PLAIN_DICTIONARY = 1,
+        DICTIONARY = 1,
         RLE = 2,
         DIFF = 3,
         TS_2DIFF = 4,
         BITMAP = 5,
         GORILLA_V1 = 6,
         REGULAR = 7,
-        GORILLA = 8
+        GORILLA = 8,
     };
 }
 namespace TSStatusCode {
diff --git a/client-py/iotdb/utils/IoTDBConstants.py b/client-py/iotdb/utils/IoTDBConstants.py
index f053af7..ff92773 100644
--- a/client-py/iotdb/utils/IoTDBConstants.py
+++ b/client-py/iotdb/utils/IoTDBConstants.py
@@ -32,7 +32,7 @@ class TSDataType(Enum):
 @unique
 class TSEncoding(Enum):
     PLAIN = 0
-    PLAIN_DICTIONARY = 1
+    DICTIONARY = 1
     RLE = 2
     DIFF = 3
     TS_2DIFF = 4
@@ -41,7 +41,6 @@ class TSEncoding(Enum):
     REGULAR = 7
     GORILLA = 8
 
-
 @unique
 class Compressor(Enum):
     UNCOMPRESSED = 0
diff --git a/docs/SystemDesign/TsFile/Format.md b/docs/SystemDesign/TsFile/Format.md
index 41a3ca1..0725c39 100644
--- a/docs/SystemDesign/TsFile/Format.md
+++ b/docs/SystemDesign/TsFile/Format.md
@@ -45,7 +45,7 @@
   - 5: TEXT (`String`)
 - **Encoding Type Hardcode**
   - 0: PLAIN
-  - 1: PLAIN_DICTIONARY
+  - 1: DICTIONARY
   - 2: RLE
   - 3: DIFF
   - 4: TS_2DIFF
diff --git a/docs/UserGuide/Data-Concept/Encoding.md b/docs/UserGuide/Data-Concept/Encoding.md
index 7a12011..5b44184 100644
--- a/docs/UserGuide/Data-Concept/Encoding.md
+++ b/docs/UserGuide/Data-Concept/Encoding.md
@@ -53,9 +53,12 @@ Regular data encoding is more suitable for time encoding regular sequence increa
 
 Regular data encoding method is not suitable for the data with fluctuations (irregular data), and TS_2DIFF is recommended to deal with it.
 
+* DICTIONARY
+
+DICTIONARY encoding is lossless. It is suitable for TEXT data with low cardinality (i.e. low number of distinct values). It is not recommended to use it for high-cardinality data. 
 * Correspondence between data type and encoding
 
-The four encodings described in the previous sections are applicable to different data types. If the correspondence is wrong, the time series cannot be created correctly. The correspondence between the data type and its supported encodings is summarized in the Table below.
+The five encodings described in the previous sections are applicable to different data types. If the correspondence is wrong, the time series cannot be created correctly. The correspondence between the data type and its supported encodings is summarized in the Table below.
 
 <center> **The correspondence between the data type and its supported encodings**
 
@@ -66,6 +69,6 @@ The four encodings described in the previous sections are applicable to differen
 |INT64	|PLAIN, RLE, TS_2DIFF, GORILLA|
 |FLOAT	|PLAIN, RLE, TS_2DIFF, GORILLA|
 |DOUBLE	|PLAIN, RLE, TS_2DIFF, GORILLA|
-|TEXT	|PLAIN|
+|TEXT	|PLAIN, DICTIONARY|
 
 </center>
diff --git a/docs/zh/SystemDesign/TsFile/Format.md b/docs/zh/SystemDesign/TsFile/Format.md
index 31c0019..e8ecbbe 100644
--- a/docs/zh/SystemDesign/TsFile/Format.md
+++ b/docs/zh/SystemDesign/TsFile/Format.md
@@ -44,7 +44,7 @@
   - 5: TEXT (`String`)
 - **编码类型**
   - 0: PLAIN
-  - 1: PLAIN_DICTIONARY
+  - 1: DICTIONARY
   - 2: RLE
   - 3: DIFF
   - 4: TS_2DIFF
diff --git a/docs/zh/UserGuide/Data-Concept/Encoding.md b/docs/zh/UserGuide/Data-Concept/Encoding.md
index 6265256..787c833 100644
--- a/docs/zh/UserGuide/Data-Concept/Encoding.md
+++ b/docs/zh/UserGuide/Data-Concept/Encoding.md
@@ -53,9 +53,13 @@ GORILLA编码是一种无损编码,它比较适合编码前后值比较接近
 
 定频数据编码无法用于非定频数据,建议使用二阶差分编码(TS_2DIFF)进行处理。
 
+* 字典编码 (DICTIONARY)
+
+字典编码是一种无损编码。它适合编码基数小的数据(即数据去重后唯一值数量小)。不推荐用于基数大的数据。
+
 * 数据类型与编码的对应关系
 
-前文介绍的四种编码适用于不同的数据类型,若对应关系错误,则无法正确创建时间序列。数据类型与支持其编码的编码方式对应关系总结如表格2-3。
+前文介绍的五种编码适用于不同的数据类型,若对应关系错误,则无法正确创建时间序列。数据类型与支持其编码的编码方式对应关系总结如表格2-3。
 
 <div style="text-align: center;"> **表格2-3 数据类型与支持其编码的对应关系**
 
@@ -66,6 +70,6 @@ GORILLA编码是一种无损编码,它比较适合编码前后值比较接近
 |INT64	|PLAIN, RLE, TS_2DIFF, GORILLA|
 |FLOAT	|PLAIN, RLE, TS_2DIFF, GORILLA|
 |DOUBLE	|PLAIN, RLE, TS_2DIFF, GORILLA|
-|TEXT	|PLAIN|
+|TEXT	|PLAIN, DICTIONARY|
 
 </div>
diff --git a/server/src/main/java/org/apache/iotdb/db/utils/SchemaUtils.java b/server/src/main/java/org/apache/iotdb/db/utils/SchemaUtils.java
index 0f268c3..828c83b 100644
--- a/server/src/main/java/org/apache/iotdb/db/utils/SchemaUtils.java
+++ b/server/src/main/java/org/apache/iotdb/db/utils/SchemaUtils.java
@@ -80,6 +80,7 @@ public class SchemaUtils {
 
     Set<TSEncoding> textSet = new HashSet<>();
     textSet.add(TSEncoding.PLAIN);
+    textSet.add(TSEncoding.DICTIONARY);
     schemaChecker.put(TSDataType.TEXT, textSet);
   }
 
diff --git a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBEncodingIT.java b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBEncodingIT.java
index ee1d634..36ae2c3 100644
--- a/server/src/test/java/org/apache/iotdb/db/integration/IoTDBEncodingIT.java
+++ b/server/src/test/java/org/apache/iotdb/db/integration/IoTDBEncodingIT.java
@@ -241,6 +241,62 @@ public class IoTDBEncodingIT {
     }
   }
 
+  @Test
+  public void testSetTimeEncoderRegularAndValueEncoderDictionary() {
+    try (Connection connection =
+            DriverManager.getConnection(
+                Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+      statement.execute(
+          "CREATE TIMESERIES root.db_0.tab0.city WITH DATATYPE=TEXT,ENCODING=DICTIONARY");
+      statement.execute("insert into root.db_0.tab0(time,city) values(1,\"Nanjing\")");
+      statement.execute("insert into root.db_0.tab0(time,city) values(2,\"Nanjing\")");
+      statement.execute("insert into root.db_0.tab0(time,city) values(3,\"Beijing\")");
+      statement.execute("insert into root.db_0.tab0(time,city) values(4,\"Shanghai\")");
+      statement.execute("flush");
+
+      String[] result = new String[] {"Nanjing", "Nanjing", "Beijing", "Shanghai"};
+      try (ResultSet resultSet = statement.executeQuery("select * from root.db_0.tab0")) {
+        int index = 0;
+        while (resultSet.next()) {
+          String city = resultSet.getString("root.db_0.tab0.city");
+          assertEquals(result[index], city);
+          index++;
+        }
+      }
+    } catch (Exception e) {
+      e.printStackTrace();
+    }
+  }
+
+  @Test
+  public void testSetTimeEncoderRegularAndValueEncoderDictionaryOutOfOrder() {
+    try (Connection connection =
+            DriverManager.getConnection(
+                Config.IOTDB_URL_PREFIX + "127.0.0.1:6667/", "root", "root");
+        Statement statement = connection.createStatement()) {
+      statement.execute(
+          "CREATE TIMESERIES root.db_0.tab0.city WITH DATATYPE=TEXT,ENCODING=DICTIONARY");
+      statement.execute("insert into root.db_0.tab0(time,city) values(1,\"Nanjing\")");
+      statement.execute("insert into root.db_0.tab0(time,city) values(2,\"Nanjing\")");
+      statement.execute("insert into root.db_0.tab0(time,city) values(4,\"Beijing\")");
+      statement.execute("insert into root.db_0.tab0(time,city) values(3,\"Shanghai\")");
+      statement.execute("flush");
+
+      String[] result = new String[] {"Nanjing", "Nanjing", "Shanghai", "Beijing"};
+      try (ResultSet resultSet = statement.executeQuery("select * from root.db_0.tab0")) {
+        int index = 0;
+        while (resultSet.next()) {
+          String city = resultSet.getString("root.db_0.tab0.city");
+          assertEquals(result[index], city);
+          index++;
+        }
+      }
+    } catch (Exception e) {
+      e.printStackTrace();
+    }
+  }
+
   private static void insertData() throws ClassNotFoundException {
     List<String> sqls =
         new ArrayList<>(
diff --git a/testcontainer/src/test/java/org/apache/iotdb/db/sql/Cases.java b/testcontainer/src/test/java/org/apache/iotdb/db/sql/Cases.java
index 8b1441b..17e4f53 100644
--- a/testcontainer/src/test/java/org/apache/iotdb/db/sql/Cases.java
+++ b/testcontainer/src/test/java/org/apache/iotdb/db/sql/Cases.java
@@ -133,6 +133,31 @@ public abstract class Cases {
       Assert.assertEquals(25.0, last, 0.1);
       resultSet.close();
     }
+
+    // test dictionary encoding
+    writeStatement.execute(
+        "create timeseries root.ln.wf01.wt01.city WITH DATATYPE=TEXT, ENCODING=DICTIONARY");
+    initDataArray =
+        new String[] {
+          "INSERT INTO root.ln.wf01.wt01(timestamp, city) values(250, \"Nanjing\")",
+          "INSERT INTO root.ln.wf01.wt01(timestamp, city) values(300, \"Nanjing\")",
+          "INSERT INTO root.ln.wf01.wt01(timestamp, city) values(350, \"Singapore\")",
+          "INSERT INTO root.ln.wf01.wt01(timestamp, city) values(350, \"Shanghai\")"
+        };
+    for (String initData : initDataArray) {
+      writeStatement.execute(initData);
+    }
+
+    String[] results = new String[] {"Nanjing", "Nanjing", "Singapore", "Shanghai"};
+    for (Statement readStatement : readStatements) {
+      resultSet = readStatement.executeQuery("select * from root.ln.wf01.wt01");
+      int i = 0;
+      while (resultSet.next()) {
+        Assert.assertEquals(results[i++], resultSet.getString("root.ln.wf01.wt01.city"));
+      }
+      Assert.assertFalse(resultSet.next());
+      resultSet.close();
+    }
   }
 
   // test https://issues.apache.org/jira/browse/IOTDB-1266
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/Decoder.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/Decoder.java
index d5ac0cb..37c5ebe 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/Decoder.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/Decoder.java
@@ -110,6 +110,8 @@ public abstract class Decoder {
           default:
             throw new TsFileDecodingException(String.format(ERROR_MSG, encoding, dataType));
         }
+      case DICTIONARY:
+        return new DictionaryDecoder();
       default:
         throw new TsFileDecodingException(String.format(ERROR_MSG, encoding, dataType));
     }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/DictionaryDecoder.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/DictionaryDecoder.java
new file mode 100644
index 0000000..aff030a
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/decoder/DictionaryDecoder.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.tsfile.encoding.decoder;
+
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.utils.Binary;
+import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+public class DictionaryDecoder extends Decoder {
+  private static final Logger logger = LoggerFactory.getLogger(DictionaryDecoder.class);
+
+  private List<Binary> entryIndex;
+  private IntRleDecoder valueDecoder;
+
+  public DictionaryDecoder() {
+    super(TSEncoding.DICTIONARY);
+
+    valueDecoder = new IntRleDecoder();
+  }
+
+  @Override
+  public boolean hasNext(ByteBuffer buffer) {
+    if (entryIndex == null) {
+      initMap(buffer);
+    }
+
+    try {
+      return valueDecoder.hasNext(buffer);
+    } catch (IOException e) {
+      logger.error("tsfile-decoding DictionaryDecoder: error occurs when decoding", e);
+    }
+
+    return false;
+  }
+
+  @Override
+  public Binary readBinary(ByteBuffer buffer) {
+    if (entryIndex == null) {
+      initMap(buffer);
+    }
+    int code = valueDecoder.readInt(buffer);
+    return entryIndex.get(code);
+  }
+
+  private void initMap(ByteBuffer buffer) {
+    int length = ReadWriteForEncodingUtils.readVarInt(buffer);
+    entryIndex = new ArrayList<>(length);
+    for (int i = 0; i < length; i++) {
+      int binaryLength = ReadWriteForEncodingUtils.readVarInt(buffer);
+      byte[] buf = new byte[binaryLength];
+      buffer.get(buf, 0, binaryLength);
+      entryIndex.add(new Binary(buf));
+    }
+  }
+
+  @Override
+  public void reset() {
+    entryIndex = null;
+    valueDecoder.reset();
+  }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/DictionaryEncoder.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/DictionaryEncoder.java
new file mode 100644
index 0000000..b339c71
--- /dev/null
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/DictionaryEncoder.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.tsfile.encoding.encoder;
+
+import org.apache.iotdb.tsfile.file.metadata.enums.TSEncoding;
+import org.apache.iotdb.tsfile.utils.Binary;
+import org.apache.iotdb.tsfile.utils.ReadWriteForEncodingUtils;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+
+/**
+ * An encoder implementing dictionary encoding.
+ *
+ * <pre>Encoding format: {@code
+ * <map> <indexes>
+ * <map> := <map length> <map data>
+ * <map data> := [<entry size><entry data>]...
+ * <indexes> := [<index>]...
+ * }</pre>
+ */
+public class DictionaryEncoder extends Encoder {
+  private static final Logger logger = LoggerFactory.getLogger(DictionaryEncoder.class);
+
+  private HashMap<Binary, Integer> entryIndex;
+  private List<Binary> indexEntry;
+  private IntRleEncoder valuesEncoder;
+  private long mapSize;
+
+  public DictionaryEncoder() {
+    super(TSEncoding.DICTIONARY);
+
+    entryIndex = new HashMap<>();
+    indexEntry = new ArrayList<>();
+    valuesEncoder = new IntRleEncoder();
+    mapSize = 0;
+  }
+
+  @Override
+  public void encode(Binary value, ByteArrayOutputStream out) {
+    entryIndex.computeIfAbsent(
+        value,
+        (v) -> {
+          indexEntry.add(v);
+          mapSize += v.getLength();
+          return entryIndex.size();
+        });
+    valuesEncoder.encode(entryIndex.get(value), out);
+  }
+
+  @Override
+  public void flush(ByteArrayOutputStream out) {
+    try {
+      writeMap(out);
+      writeEncodedData(out);
+    } catch (IOException e) {
+      logger.error("tsfile-encoding DictionaryEncoder: error occurs when flushing", e);
+    }
+    reset();
+  }
+
+  @Override
+  public int getOneItemMaxSize() {
+    // map + one encoded value = (map size + map value) + one encoded value = (4 + 4) + 4
+    return 12;
+  }
+
+  @Override
+  public long getMaxByteSize() {
+    // has max size when when all points are unique
+    return 4 + mapSize + valuesEncoder.getMaxByteSize();
+  }
+
+  private void writeMap(ByteArrayOutputStream out) throws IOException {
+    ReadWriteForEncodingUtils.writeVarInt(indexEntry.size(), out);
+    for (Binary value : indexEntry) {
+      ReadWriteForEncodingUtils.writeVarInt(value.getLength(), out);
+      out.write(value.getValues());
+    }
+  }
+
+  private void writeEncodedData(ByteArrayOutputStream out) throws IOException {
+    valuesEncoder.flush(out);
+  }
+
+  private void reset() {
+    entryIndex.clear();
+    indexEntry.clear();
+    valuesEncoder.reset();
+    mapSize = 0;
+  }
+}
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/TSEncodingBuilder.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/TSEncodingBuilder.java
index e1d4fb6..74f9c3c 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/TSEncodingBuilder.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/encoding/encoder/TSEncodingBuilder.java
@@ -68,6 +68,8 @@ public abstract class TSEncodingBuilder {
         return new Regular();
       case GORILLA:
         return new GorillaV2();
+      case DICTIONARY:
+        return new Dictionary();
       default:
         throw new UnsupportedOperationException(type.toString());
     }
@@ -296,4 +298,20 @@ public abstract class TSEncodingBuilder {
       // allowed do nothing
     }
   }
+
+  public static class Dictionary extends TSEncodingBuilder {
+
+    @Override
+    public Encoder getEncoder(TSDataType type) {
+      if (type == TSDataType.TEXT) {
+        return new DictionaryEncoder();
+      }
+      throw new UnSupportedDataTypeException("DICTIONARY doesn't support data type: " + type);
+    }
+
+    @Override
+    public void initFromProps(Map<String, String> props) {
+      // do nothing
+    }
+  }
 }
diff --git a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/TSEncoding.java b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/TSEncoding.java
index 394f0d2..de95fac 100644
--- a/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/TSEncoding.java
+++ b/tsfile/src/main/java/org/apache/iotdb/tsfile/file/metadata/enums/TSEncoding.java
@@ -20,7 +20,7 @@ package org.apache.iotdb.tsfile.file.metadata.enums;
 
 public enum TSEncoding {
   PLAIN((byte) 0),
-  PLAIN_DICTIONARY((byte) 1),
+  DICTIONARY((byte) 1),
   RLE((byte) 2),
   DIFF((byte) 3),
   TS_2DIFF((byte) 4),
@@ -50,7 +50,7 @@ public enum TSEncoding {
       case 0:
         return TSEncoding.PLAIN;
       case 1:
-        return TSEncoding.PLAIN_DICTIONARY;
+        return TSEncoding.DICTIONARY;
       case 2:
         return TSEncoding.RLE;
       case 3:
diff --git a/tsfile/src/test/java/org/apache/iotdb/tsfile/encoding/decoder/DictionaryDecoderTest.java b/tsfile/src/test/java/org/apache/iotdb/tsfile/encoding/decoder/DictionaryDecoderTest.java
new file mode 100644
index 0000000..de3f6e7
--- /dev/null
+++ b/tsfile/src/test/java/org/apache/iotdb/tsfile/encoding/decoder/DictionaryDecoderTest.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iotdb.tsfile.encoding.decoder;
+
+import org.apache.iotdb.tsfile.encoding.encoder.DictionaryEncoder;
+import org.apache.iotdb.tsfile.utils.Binary;
+
+import org.junit.Test;
+
+import java.io.ByteArrayOutputStream;
+import java.nio.ByteBuffer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class DictionaryDecoderTest {
+  private DictionaryEncoder encoder = new DictionaryEncoder();
+  private DictionaryDecoder decoder = new DictionaryDecoder();
+  private ByteArrayOutputStream baos = new ByteArrayOutputStream();
+
+  @Test
+  public void testSingle() {
+    testAll("a");
+    testAll("b");
+    testAll("c");
+  }
+
+  @Test
+  public void testAllUnique() {
+    testAll("a", "b", "c");
+    testAll("x", "o", "q");
+    testAll(",", ".", "c", "b", "e");
+  }
+
+  @Test
+  public void testAllSame() {
+    testAll("a", "a", "a");
+    testAll("b", "b", "b");
+  }
+
+  @Test
+  public void testMixed() {
+    // all characters
+    String[] allChars = new String[256];
+    allChars[0] = "" + (char) ('a' + 1);
+    for (int i = 0; i < 256; i++) {
+      allChars[i] = "" + (char) (i) + (char) (i) + (char) (i);
+    }
+    testAll(allChars);
+  }
+
+  private void testAll(String... all) {
+    for (String s : all) {
+      encoder.encode(new Binary(s), baos);
+    }
+    encoder.flush(baos);
+
+    ByteBuffer out = ByteBuffer.wrap(baos.toByteArray());
+
+    for (String s : all) {
+      assertTrue(decoder.hasNext(out));
+      assertEquals(s, decoder.readBinary(out).getStringValue());
+    }
+
+    decoder.reset();
+    baos.reset();
+  }
+}