You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@iotdb.apache.org by "xiangdong Huang (JIRA)" <ji...@apache.org> on 2019/06/04 15:00:00 UTC

[jira] [Created] (IOTDB-112) Avoid long tail insertion which is caused by synchronized close-bufferwrite

xiangdong Huang created IOTDB-112:
-------------------------------------

             Summary: Avoid long tail insertion which is caused by synchronized close-bufferwrite
                 Key: IOTDB-112
                 URL: https://issues.apache.org/jira/browse/IOTDB-112
             Project: Apache IoTDB
          Issue Type: Improvement
            Reporter: xiangdong Huang


In our test, IoTDB has a good insertion performance, and the average latency can be ~200 ms in a given workload and hardware. 

However, when we draw the histogram of the latency, we find that 97.5% latencies are less than 200 ms, while 2.7% latencies are greater. The result shows that there are some long tail latency.

Then we find that some insertion latencies are about 30 seconds... (but the ratio is less than 0.5%). Indeed, for each connection, a long tail insertion appears per 1 or 2 minutes....

By reading source codes, I think it is because that in the insertion function,

`private void insertBufferWrite(FileNodeProcessor fileNodeProcessor, long timestamp,
 boolean isMonitor, TSRecord tsRecord, String deviceId)`,

if the corresponding TsFile is too large, the function is blocked until the memtable is flushed on disk and the TsFile is sealed (we call it as closing a TsFile). The latencies of the long tail insertions are very close to the time cost of flushing and sealing a TsFile. 

So, if we set the closing function using the async mode, we can avoid the long tail insertion.

However,  there are some side effects we have to fix:
 # At the same time, if a new insertion comes, then a new memtable should be assigned, and a new unsealed TsFile is created; 
 # That means that there are more than 1 unsealed TsFiles if the system is crashed before the closing function is finished. So, we have to modify the startup process to recover these files. 

Is there any other side effect that I have to pay attention to?

 

 

 

 

 

 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Re: Avoid long-tail insertion

Posted by Jialin Qiao <qj...@mails.tsinghua.edu.cn>.
Hi,

1. TVList is an evolution version of PrimitiveArrayList, as it has the ability to sort data. The data structure is still array of primitive types. By including sorting, it avoids using ArrayList and TimeValuePair, which cause a lot of temp objects (causes much overhead for GC in JVM). We add a test LongTVListTest to evaluate the performances of TVList and PrimitiveArrayList. The result shows that the total time cost of write+sort+read of TVList is about half of PrimitiveArrayList.

2. Version 2 is the new storage engine. We leave version 1 just for reference. Once the version 2 is stable, we will remove version 1.

3. Thanks, the change of PR109 is added in this branch.

Best,
--
Jialin Qiao
School of Software, Tsinghua University

乔嘉林
清华大学 软件学院

> -----原始邮件-----
> 发件人: "Xiangdong Huang" <sa...@gmail.com>
> 发送时间: 2019-06-27 22:28:20 (星期四)
> 收件人: dev@iotdb.apache.org
> 抄送: 
> 主题: Re: Re: Re: Avoid long-tail insertion
> 
> Hi,
> 
> I am reading the codes on the branch feature_async_close_tsfile... and
> there are some questions about your current work:
> 
> 1. Why you replace the PrimitiveArrayList with TVList?
>  As I know, PrimitiveArrayList is for avoiding auto-boxing, and it uses
> some tricks  to avoid writing too many similar codes.
> So, does TVList have better performance?
> 
> 2. Some classes have both version 1 and version 2, e.g., FileNodeManager
> and FileNodeManagerV2, will you retain both of them or just leave version 2?
> 
> 3. The behavior of the NativeRestorableIOWriter is changed in this
> branch... please notice the related PR (PR109)
> 
> I have to say, this branch contains bulk of modifications....
> 
> Best,
> -----------------------------------
> Xiangdong Huang
> School of Software, Tsinghua University
> 
>  黄向东
> 清华大学 软件学院
> 
> 
> Jialin Qiao <qj...@mails.tsinghua.edu.cn> 于2019年6月24日周一 下午8:24写道:
> 
> >
> > Yes, there are many changes. The branch I am working on is
> > feature_async_close_tsfile.
> > Anyone interested is welcome to join and discuss.
> >
> > Best,
> > --
> > Jialin Qiao
> > School of Software, Tsinghua University
> >
> > 乔嘉林
> > 清华大学 软件学院
> >
> > > -----原始邮件-----
> > > 发件人: "Xiangdong Huang" <sa...@gmail.com>
> > > 发送时间: 2019-06-23 10:59:29 (星期日)
> > > 收件人: dev@iotdb.apache.org
> > > 抄送:
> > > 主题: Re: Re: Avoid long-tail insertion
> > >
> > > Hi,
> > >
> > > Once your work branch is almost ready, let me know so I can help to
> > review.
> > > I think it is a HUGE PR...
> > >
> > > -----------------------------------
> > > Xiangdong Huang
> > > School of Software, Tsinghua University
> > >
> > >  黄向东
> > > 清华大学 软件学院
> > >
> > >
> > > Jialin Qiao <qj...@mails.tsinghua.edu.cn> 于2019年6月22日周六 下午9:57写道:
> > >
> > > > Hi Xiangdong,
> > > >
> > > > I will merge this patch. Let "Directories" manage the folders of both
> > > > sequence and unSequence files is good.
> > > >
> > > > However, the naming of "Directories" is not clear. It would be better
> > to
> > > > rename to "DirectoryManager"
> > > >
> > > > Best,
> > > > --
> > > > Jialin Qiao
> > > > School of Software, Tsinghua University
> > > >
> > > > 乔嘉林
> > > > 清华大学 软件学院
> > > >
> > > > > -----原始邮件-----
> > > > > 发件人: "Xiangdong Huang" <sa...@gmail.com>
> > > > > 发送时间: 2019-06-22 16:35:29 (星期六)
> > > > > 收件人: dev@iotdb.apache.org
> > > > > 抄送:
> > > > > 主题: Re: Avoid long-tail insertion
> > > > >
> > > > > Hi jialin,
> > > > >
> > > > > I submit some modifications for:
> > > > >
> > > > > * add the overflow data folder location setting in the
> > > > > iotdb-engine.properties;
> > > > > * let Directories.java to manage the above folder.
> > > > >
> > > > > If you need to refactor the overflow when you solving the long tail
> > > > issue,
> > > > > you can apply the patch from [1] first to simplify your work.
> > > > >
> > > > > [1]
> > > > >
> > > >
> > https://issues.apache.org/jira/secure/attachment/12972547/overflow-folder.patch
> > > > >
> > > > > Best,
> > > > > -----------------------------------
> > > > > Xiangdong Huang
> > > > > School of Software, Tsinghua University
> > > > >
> > > > >  黄向东
> > > > > 清华大学 软件学院
> > > > >
> > > > >
> > > > > Xiangdong Huang <sa...@gmail.com> 于2019年6月22日周六 下午3:19写道:
> > > > >
> > > > > > If you change the process like this, i.e., there are more than one
> > > > > > unsealed TsFiles for each storage group, then  you have to modify
> > the
> > > > WAL
> > > > > > module.. Because current WAL module only recognizes the last
> > unsealed
> > > > > > TsFile..
> > > > > >
> > > > > > By the way, "sealed" is better than "closed", I think..  A sealed
> > file
> > > > > > means the file which has the magic string at the head and the tail.
> > > > > >
> > > > > > Best,
> > > > > > -----------------------------------
> > > > > > Xiangdong Huang
> > > > > > School of Software, Tsinghua University
> > > > > >
> > > > > >  黄向东
> > > > > > 清华大学 软件学院
> > > > > >
> > > > > >
> > > > > > Jialin Qiao <qj...@mails.tsinghua.edu.cn> 于2019年6月22日周六 下午2:54写道:
> > > > > >
> > > > > >>
> > > > > >> Hi, I am solving the long-tail latency problem.
> > > > > >>
> > > > > >> There are some cases (blocking points) that blocking the
> > insertion.
> > > > For a
> > > > > >> better understanding of this problem, I first introduce the
> > writing
> > > > process
> > > > > >> of IoTDB:
> > > > > >>
> > > > > >> IoTDB maintains several independent engines (storage group) that
> > > > supports
> > > > > >> read and write. In the following, we focus on one engine. A engine
> > > > > >> maintains several closed data files and one unclosed data file
> > that
> > > > > >> receives appended data. In memory, there is only one working
> > memtable
> > > > (m1)
> > > > > >> that receives writes. There is also another memtable (m2) that
> > will
> > > > take
> > > > > >> place m1 when m1 is full and being flushed.
> > > > > >>
> > > > > >> When a data item is inserted:
> > > > > >>
> > > > > >> (1)We insert it into the working memtable.
> > > > > >> (2)We check the size of the memtable. If it reaches a threshold,
> > we
> > > > > >> submit a flush task “after the previous flush task is finished”
> > and
> > > > switch
> > > > > >> the two memtables.
> > > > > >> (3)We check the size of the unclosed file. If it reaches a
> > threshold,
> > > > we
> > > > > >> close it “after the previous flush task is finished”.
> > > > > >>
> > > > > >> In the above steps, all the "after the previous flush task is
> > > > finished"
> > > > > >> will block the insertion process. One solution is to make all
> > flush
> > > > and
> > > > > >> close task asynchronous. Some questions need to carefully
> > considered:
> > > > > >>
> > > > > >> (1) Many memtables may be flushed concurrently to an unclosed
> > file.
> > > > How
> > > > > >> to guarantee the order of serialization?
> > > > > >> (2) Once a close task is submitted, a new unclosed file will be
> > > > created
> > > > > >> and receives appended data. So there will exists many unclosed
> > files.
> > > > How
> > > > > >> the query and compaction process will be impacted?
> > > > > >>
> > > > > >> Thanks,
> > > > > >>
> > > > > >> Jialin Qiao
> > > > > >> School of Software, Tsinghua University
> > > > > >>
> > > > > >> 乔嘉林
> > > > > >> 清华大学 软件学院
> > > > > >>
> > > > > >> > -----原始邮件-----
> > > > > >> > 发件人: "Xiangdong Huang" <sa...@gmail.com>
> > > > > >> > 发送时间: 2019-06-04 23:08:34 (星期二)
> > > > > >> > 收件人: dev@iotdb.apache.org, "江天" <jt...@163.com>
> > > > > >> > 抄送:
> > > > > >> > 主题: Re: [jira] [Created] (IOTDB-112) Avoid long tail insertion
> > > > which is
> > > > > >> caused by synchronized close-bufferwrite
> > > > > >> >
> > > > > >> > I attached the histogram of the latency in the JIRA.
> > > > > >> >
> > > > > >> > The x-axis is the latency while the y-axis is the cumulative
> > > > > >> distribution.
> > > > > >> > We can see that about 30% insertion can be finished in 20ms,
> > and 60%
> > > > > >> > insertion can be finished in 40ms even though the IoTDB
> > instance is
> > > > > >> serving
> > > > > >> > for a heavy workload... So, eliminating the long tail insertion
> > can
> > > > make
> > > > > >> > the average latency far better.
> > > > > >> >
> > > > > >> > If someone is working on the refactor_overflow or
> > > > refactor_bufferwrite,
> > > > > >> > please pay attention to the code branch for this issue.
> > > > > >> >
> > > > > >> > Best,
> > > > > >> >
> > > > > >> > -----------------------------------
> > > > > >> > Xiangdong Huang
> > > > > >> > School of Software, Tsinghua University
> > > > > >> >
> > > > > >> >  黄向东
> > > > > >> > 清华大学 软件学院
> > > > > >> >
> > > > > >> >
> > > > > >> > xiangdong Huang (JIRA) <ji...@apache.org> 于2019年6月4日周二
> > 下午11:00写道:
> > > > > >> >
> > > > > >> > > xiangdong Huang created IOTDB-112:
> > > > > >> > > -------------------------------------
> > > > > >> > >
> > > > > >> > >              Summary: Avoid long tail insertion which is
> > caused by
> > > > > >> > > synchronized close-bufferwrite
> > > > > >> > >                  Key: IOTDB-112
> > > > > >> > >                  URL:
> > > > https://issues.apache.org/jira/browse/IOTDB-112
> > > > > >> > >              Project: Apache IoTDB
> > > > > >> > >           Issue Type: Improvement
> > > > > >> > >             Reporter: xiangdong Huang
> > > > > >> > >
> > > > > >> > >
> > > > > >> > > In our test, IoTDB has a good insertion performance, and the
> > > > average
> > > > > >> > > latency can be ~200 ms in a given workload and hardware.
> > > > > >> > >
> > > > > >> > > However, when we draw the histogram of the latency, we find
> > that
> > > > 97.5%
> > > > > >> > > latencies are less than 200 ms, while 2.7% latencies are
> > greater.
> > > > The
> > > > > >> > > result shows that there are some long tail latency.
> > > > > >> > >
> > > > > >> > > Then we find that some insertion latencies are about 30
> > seconds...
> > > > > >> (but
> > > > > >> > > the ratio is less than 0.5%). Indeed, for each connection, a
> > long
> > > > tail
> > > > > >> > > insertion appears per 1 or 2 minutes....
> > > > > >> > >
> > > > > >> > > By reading source codes, I think it is because that in the
> > > > insertion
> > > > > >> > > function,
> > > > > >> > >
> > > > > >> > > `private void insertBufferWrite(FileNodeProcessor
> > > > fileNodeProcessor,
> > > > > >> long
> > > > > >> > > timestamp,
> > > > > >> > >  boolean isMonitor, TSRecord tsRecord, String deviceId)`,
> > > > > >> > >
> > > > > >> > > if the corresponding TsFile is too large, the function is
> > blocked
> > > > > >> until
> > > > > >> > > the memtable is flushed on disk and the TsFile is sealed (we
> > call
> > > > it
> > > > > >> as
> > > > > >> > > closing a TsFile). The latencies of the long tail insertions
> > are
> > > > very
> > > > > >> close
> > > > > >> > > to the time cost of flushing and sealing a TsFile.
> > > > > >> > >
> > > > > >> > > So, if we set the closing function using the async mode, we
> > can
> > > > avoid
> > > > > >> the
> > > > > >> > > long tail insertion.
> > > > > >> > >
> > > > > >> > > However,  there are some side effects we have to fix:
> > > > > >> > >  # At the same time, if a new insertion comes, then a new
> > memtable
> > > > > >> should
> > > > > >> > > be assigned, and a new unsealed TsFile is created;
> > > > > >> > >  # That means that there are more than 1 unsealed TsFiles if
> > the
> > > > > >> system is
> > > > > >> > > crashed before the closing function is finished. So, we have
> > to
> > > > > >> modify the
> > > > > >> > > startup process to recover these files.
> > > > > >> > >
> > > > > >> > > Is there any other side effect that I have to pay attention
> > to?
> > > > > >> > >
> > > > > >> > >
> > > > > >> > >
> > > > > >> > > --
> > > > > >> > > This message was sent by Atlassian JIRA
> > > > > >> > > (v7.6.3#76005)
> > > > > >> > >
> > > > > >>
> > > > > >
> > > >
> >

Re: Re: Re: Avoid long-tail insertion

Posted by Xiangdong Huang <sa...@gmail.com>.
Hi,

I am reading the codes on the branch feature_async_close_tsfile... and
there are some questions about your current work:

1. Why you replace the PrimitiveArrayList with TVList?
 As I know, PrimitiveArrayList is for avoiding auto-boxing, and it uses
some tricks  to avoid writing too many similar codes.
So, does TVList have better performance?

2. Some classes have both version 1 and version 2, e.g., FileNodeManager
and FileNodeManagerV2, will you retain both of them or just leave version 2?

3. The behavior of the NativeRestorableIOWriter is changed in this
branch... please notice the related PR (PR109)

I have to say, this branch contains bulk of modifications....

Best,
-----------------------------------
Xiangdong Huang
School of Software, Tsinghua University

 黄向东
清华大学 软件学院


Jialin Qiao <qj...@mails.tsinghua.edu.cn> 于2019年6月24日周一 下午8:24写道:

>
> Yes, there are many changes. The branch I am working on is
> feature_async_close_tsfile.
> Anyone interested is welcome to join and discuss.
>
> Best,
> --
> Jialin Qiao
> School of Software, Tsinghua University
>
> 乔嘉林
> 清华大学 软件学院
>
> > -----原始邮件-----
> > 发件人: "Xiangdong Huang" <sa...@gmail.com>
> > 发送时间: 2019-06-23 10:59:29 (星期日)
> > 收件人: dev@iotdb.apache.org
> > 抄送:
> > 主题: Re: Re: Avoid long-tail insertion
> >
> > Hi,
> >
> > Once your work branch is almost ready, let me know so I can help to
> review.
> > I think it is a HUGE PR...
> >
> > -----------------------------------
> > Xiangdong Huang
> > School of Software, Tsinghua University
> >
> >  黄向东
> > 清华大学 软件学院
> >
> >
> > Jialin Qiao <qj...@mails.tsinghua.edu.cn> 于2019年6月22日周六 下午9:57写道:
> >
> > > Hi Xiangdong,
> > >
> > > I will merge this patch. Let "Directories" manage the folders of both
> > > sequence and unSequence files is good.
> > >
> > > However, the naming of "Directories" is not clear. It would be better
> to
> > > rename to "DirectoryManager"
> > >
> > > Best,
> > > --
> > > Jialin Qiao
> > > School of Software, Tsinghua University
> > >
> > > 乔嘉林
> > > 清华大学 软件学院
> > >
> > > > -----原始邮件-----
> > > > 发件人: "Xiangdong Huang" <sa...@gmail.com>
> > > > 发送时间: 2019-06-22 16:35:29 (星期六)
> > > > 收件人: dev@iotdb.apache.org
> > > > 抄送:
> > > > 主题: Re: Avoid long-tail insertion
> > > >
> > > > Hi jialin,
> > > >
> > > > I submit some modifications for:
> > > >
> > > > * add the overflow data folder location setting in the
> > > > iotdb-engine.properties;
> > > > * let Directories.java to manage the above folder.
> > > >
> > > > If you need to refactor the overflow when you solving the long tail
> > > issue,
> > > > you can apply the patch from [1] first to simplify your work.
> > > >
> > > > [1]
> > > >
> > >
> https://issues.apache.org/jira/secure/attachment/12972547/overflow-folder.patch
> > > >
> > > > Best,
> > > > -----------------------------------
> > > > Xiangdong Huang
> > > > School of Software, Tsinghua University
> > > >
> > > >  黄向东
> > > > 清华大学 软件学院
> > > >
> > > >
> > > > Xiangdong Huang <sa...@gmail.com> 于2019年6月22日周六 下午3:19写道:
> > > >
> > > > > If you change the process like this, i.e., there are more than one
> > > > > unsealed TsFiles for each storage group, then  you have to modify
> the
> > > WAL
> > > > > module.. Because current WAL module only recognizes the last
> unsealed
> > > > > TsFile..
> > > > >
> > > > > By the way, "sealed" is better than "closed", I think..  A sealed
> file
> > > > > means the file which has the magic string at the head and the tail.
> > > > >
> > > > > Best,
> > > > > -----------------------------------
> > > > > Xiangdong Huang
> > > > > School of Software, Tsinghua University
> > > > >
> > > > >  黄向东
> > > > > 清华大学 软件学院
> > > > >
> > > > >
> > > > > Jialin Qiao <qj...@mails.tsinghua.edu.cn> 于2019年6月22日周六 下午2:54写道:
> > > > >
> > > > >>
> > > > >> Hi, I am solving the long-tail latency problem.
> > > > >>
> > > > >> There are some cases (blocking points) that blocking the
> insertion.
> > > For a
> > > > >> better understanding of this problem, I first introduce the
> writing
> > > process
> > > > >> of IoTDB:
> > > > >>
> > > > >> IoTDB maintains several independent engines (storage group) that
> > > supports
> > > > >> read and write. In the following, we focus on one engine. A engine
> > > > >> maintains several closed data files and one unclosed data file
> that
> > > > >> receives appended data. In memory, there is only one working
> memtable
> > > (m1)
> > > > >> that receives writes. There is also another memtable (m2) that
> will
> > > take
> > > > >> place m1 when m1 is full and being flushed.
> > > > >>
> > > > >> When a data item is inserted:
> > > > >>
> > > > >> (1)We insert it into the working memtable.
> > > > >> (2)We check the size of the memtable. If it reaches a threshold,
> we
> > > > >> submit a flush task “after the previous flush task is finished”
> and
> > > switch
> > > > >> the two memtables.
> > > > >> (3)We check the size of the unclosed file. If it reaches a
> threshold,
> > > we
> > > > >> close it “after the previous flush task is finished”.
> > > > >>
> > > > >> In the above steps, all the "after the previous flush task is
> > > finished"
> > > > >> will block the insertion process. One solution is to make all
> flush
> > > and
> > > > >> close task asynchronous. Some questions need to carefully
> considered:
> > > > >>
> > > > >> (1) Many memtables may be flushed concurrently to an unclosed
> file.
> > > How
> > > > >> to guarantee the order of serialization?
> > > > >> (2) Once a close task is submitted, a new unclosed file will be
> > > created
> > > > >> and receives appended data. So there will exists many unclosed
> files.
> > > How
> > > > >> the query and compaction process will be impacted?
> > > > >>
> > > > >> Thanks,
> > > > >>
> > > > >> Jialin Qiao
> > > > >> School of Software, Tsinghua University
> > > > >>
> > > > >> 乔嘉林
> > > > >> 清华大学 软件学院
> > > > >>
> > > > >> > -----原始邮件-----
> > > > >> > 发件人: "Xiangdong Huang" <sa...@gmail.com>
> > > > >> > 发送时间: 2019-06-04 23:08:34 (星期二)
> > > > >> > 收件人: dev@iotdb.apache.org, "江天" <jt...@163.com>
> > > > >> > 抄送:
> > > > >> > 主题: Re: [jira] [Created] (IOTDB-112) Avoid long tail insertion
> > > which is
> > > > >> caused by synchronized close-bufferwrite
> > > > >> >
> > > > >> > I attached the histogram of the latency in the JIRA.
> > > > >> >
> > > > >> > The x-axis is the latency while the y-axis is the cumulative
> > > > >> distribution.
> > > > >> > We can see that about 30% insertion can be finished in 20ms,
> and 60%
> > > > >> > insertion can be finished in 40ms even though the IoTDB
> instance is
> > > > >> serving
> > > > >> > for a heavy workload... So, eliminating the long tail insertion
> can
> > > make
> > > > >> > the average latency far better.
> > > > >> >
> > > > >> > If someone is working on the refactor_overflow or
> > > refactor_bufferwrite,
> > > > >> > please pay attention to the code branch for this issue.
> > > > >> >
> > > > >> > Best,
> > > > >> >
> > > > >> > -----------------------------------
> > > > >> > Xiangdong Huang
> > > > >> > School of Software, Tsinghua University
> > > > >> >
> > > > >> >  黄向东
> > > > >> > 清华大学 软件学院
> > > > >> >
> > > > >> >
> > > > >> > xiangdong Huang (JIRA) <ji...@apache.org> 于2019年6月4日周二
> 下午11:00写道:
> > > > >> >
> > > > >> > > xiangdong Huang created IOTDB-112:
> > > > >> > > -------------------------------------
> > > > >> > >
> > > > >> > >              Summary: Avoid long tail insertion which is
> caused by
> > > > >> > > synchronized close-bufferwrite
> > > > >> > >                  Key: IOTDB-112
> > > > >> > >                  URL:
> > > https://issues.apache.org/jira/browse/IOTDB-112
> > > > >> > >              Project: Apache IoTDB
> > > > >> > >           Issue Type: Improvement
> > > > >> > >             Reporter: xiangdong Huang
> > > > >> > >
> > > > >> > >
> > > > >> > > In our test, IoTDB has a good insertion performance, and the
> > > average
> > > > >> > > latency can be ~200 ms in a given workload and hardware.
> > > > >> > >
> > > > >> > > However, when we draw the histogram of the latency, we find
> that
> > > 97.5%
> > > > >> > > latencies are less than 200 ms, while 2.7% latencies are
> greater.
> > > The
> > > > >> > > result shows that there are some long tail latency.
> > > > >> > >
> > > > >> > > Then we find that some insertion latencies are about 30
> seconds...
> > > > >> (but
> > > > >> > > the ratio is less than 0.5%). Indeed, for each connection, a
> long
> > > tail
> > > > >> > > insertion appears per 1 or 2 minutes....
> > > > >> > >
> > > > >> > > By reading source codes, I think it is because that in the
> > > insertion
> > > > >> > > function,
> > > > >> > >
> > > > >> > > `private void insertBufferWrite(FileNodeProcessor
> > > fileNodeProcessor,
> > > > >> long
> > > > >> > > timestamp,
> > > > >> > >  boolean isMonitor, TSRecord tsRecord, String deviceId)`,
> > > > >> > >
> > > > >> > > if the corresponding TsFile is too large, the function is
> blocked
> > > > >> until
> > > > >> > > the memtable is flushed on disk and the TsFile is sealed (we
> call
> > > it
> > > > >> as
> > > > >> > > closing a TsFile). The latencies of the long tail insertions
> are
> > > very
> > > > >> close
> > > > >> > > to the time cost of flushing and sealing a TsFile.
> > > > >> > >
> > > > >> > > So, if we set the closing function using the async mode, we
> can
> > > avoid
> > > > >> the
> > > > >> > > long tail insertion.
> > > > >> > >
> > > > >> > > However,  there are some side effects we have to fix:
> > > > >> > >  # At the same time, if a new insertion comes, then a new
> memtable
> > > > >> should
> > > > >> > > be assigned, and a new unsealed TsFile is created;
> > > > >> > >  # That means that there are more than 1 unsealed TsFiles if
> the
> > > > >> system is
> > > > >> > > crashed before the closing function is finished. So, we have
> to
> > > > >> modify the
> > > > >> > > startup process to recover these files.
> > > > >> > >
> > > > >> > > Is there any other side effect that I have to pay attention
> to?
> > > > >> > >
> > > > >> > >
> > > > >> > >
> > > > >> > > --
> > > > >> > > This message was sent by Atlassian JIRA
> > > > >> > > (v7.6.3#76005)
> > > > >> > >
> > > > >>
> > > > >
> > >
>

Re: Re: New directory structure

Posted by Jialin Qiao <qj...@mails.tsinghua.edu.cn>.
Hi,

Thanks for reminding, I will update the documents ASAP.

Best,
--
Jialin Qiao
School of Software, Tsinghua University

乔嘉林
清华大学 软件学院

> -----原始邮件-----
> 发件人: "Xiangdong Huang" <sa...@gmail.com>
> 发送时间: 2019-07-03 19:34:43 (星期三)
> 收件人: dev@iotdb.apache.org
> 抄送: 
> 主题: Re: New directory structure
> 
> Hi,
> 
> +1 for the new structure.
> 
> Jialin, as Christofer said, I think we need review all documents before
> your branch can be merged..
> 
> Best,
> -----------------------------------
> Xiangdong Huang
> School of Software, Tsinghua University
> 
>  黄向东
> 清华大学 软件学院
> 
> 
> Christofer Dutz <ch...@c-ware.de> 于2019年7月3日周三 下午7:28写道:
> 
> > Hi all,
> >
> > Please think of documenting this in any release notes as incompatible
> > change.
> >
> > Chris
> >
> > Holen Sie sichOutlook für Android<https://aka.ms/ghei36>
> >
> > ________________________________
> > From: Jialin Qiao <qj...@mails.tsinghua.edu.cn>
> > Sent: Wednesday, July 3, 2019 4:46:18 AM
> > To: dev@iotdb.apache.org
> > Subject: Re: New directory structure
> >
> > Hi,
> >
> > The following illustration is more clear:
> >
> > data  —————————————————————————(base directory)
> > |├── data   —————————————————————(data files)
> > │   ├── sequence  ——————————————————(folder of sequence data files in each
> > storage group)
> > │   │   └── storage_group_x ——————————————(storage group x)
> > │   │       ├── {systemtime-version}.tsfile ————————(data file)
> > │   │       ├── {name_of_TsFile}.resource  ————————(resource file of
> > corresponding tsfile)
> > │   └── unsequence —————————————————(folder of unsequence data files,
> > which the same structure with "sequence" directory)
> > |├── system
> > │   ├── roles  ————————————————————(information of roles)
> > │   ├── schema ———————————————————(system schema, including storage groups
> > and timeseries)
> > │   ├── storage_groups   ———————————————(version file of each storage
> > group)
> > │   └── users  ————————————————————(information of users)
> > |└── wal
> > |    ├── {name_of_storage_group}-{name_of_TsFile} ———(a folder for one
> > data file)
> > |      └── wal{version}  ————————————————(wal file for a memtable in
> > corresponding data file)
> >
> > Best,
> > --
> > Jialin Qiao
> > School of Software, Tsinghua University
> >
> > 乔嘉林
> > 清华大学 软件学院
> >
> > > -----原始邮件-----
> > > 发件人: "Jialin Qiao" <qj...@mails.tsinghua.edu.cn>
> > > 发送时间: 2019-07-03 10:24:43 (星期三)
> > > 收件人: dev@iotdb.apache.org
> > > 抄送:
> > > 主题: New directory structure
> > >
> > > Hi,
> > >
> > > The directory structure of IoTDB is in chaos, so I reorganize it in
> > PR[1].
> > >
> > > User configurations:
> > >
> > > base_dir: base directory of IoTDB ("data" by default)
> > > sys_dir: system data directory ("data/system" by default)
> > > data_dirs: data file directories, which supports multiple directories
> > ("data/data" by default, multiple directories are separated by ',")
> > > wal_dir: directory of write-ahead-log ("data/wal" by default)
> > >
> > > Under default configurations, the actual folders are like this:
> > >
> > >     data
> > >         data    —————————————————————(data files)
> > >             sequence  ——————————————————(folder of sequence data files)
> > >                 storage_group_x ——————————————(storage group x)
> > >                     {systemtime-version}.tsfile  ————————(data file)
> > >                     {name_of_TsFile}.resource   ————————(resource file
> > of corresponding tsfile)
> > >             unsequence —————————————————(folder of unsequence data
> > files, which the same structure with "sequence" directory)
> > >         system
> > >             roles  ————————————————————(information of roles)
> > >             users ————————————————————(information of users)
> > >             schema ———————————————————(system schema, including storage
> > groups and timeseries)
> > >             storage_groups   ———————————————(version file of each
> > storage group)
> > >         wal
> > >             {name_of_storage_group}-{name_of_TsFile} ———(a folder for
> > one data file)
> > >                 wal{version}  ————————————————(wal file for a memtable
> > in corresponding data file)
> > >
> > >
> > > [1] https://github.com/apache/incubator-iotdb/pull/217
> > >
> > > Best,
> > > --
> > > Jialin Qiao
> > > School of Software, Tsinghua University
> > >
> > > 乔嘉林
> > > 清华大学 软件学院
> >

Re: New directory structure

Posted by Xiangdong Huang <sa...@gmail.com>.
Hi,

+1 for the new structure.

Jialin, as Christofer said, I think we need review all documents before
your branch can be merged..

Best,
-----------------------------------
Xiangdong Huang
School of Software, Tsinghua University

 黄向东
清华大学 软件学院


Christofer Dutz <ch...@c-ware.de> 于2019年7月3日周三 下午7:28写道:

> Hi all,
>
> Please think of documenting this in any release notes as incompatible
> change.
>
> Chris
>
> Holen Sie sichOutlook für Android<https://aka.ms/ghei36>
>
> ________________________________
> From: Jialin Qiao <qj...@mails.tsinghua.edu.cn>
> Sent: Wednesday, July 3, 2019 4:46:18 AM
> To: dev@iotdb.apache.org
> Subject: Re: New directory structure
>
> Hi,
>
> The following illustration is more clear:
>
> data  —————————————————————————(base directory)
> |├── data   —————————————————————(data files)
> │   ├── sequence  ——————————————————(folder of sequence data files in each
> storage group)
> │   │   └── storage_group_x ——————————————(storage group x)
> │   │       ├── {systemtime-version}.tsfile ————————(data file)
> │   │       ├── {name_of_TsFile}.resource  ————————(resource file of
> corresponding tsfile)
> │   └── unsequence —————————————————(folder of unsequence data files,
> which the same structure with "sequence" directory)
> |├── system
> │   ├── roles  ————————————————————(information of roles)
> │   ├── schema ———————————————————(system schema, including storage groups
> and timeseries)
> │   ├── storage_groups   ———————————————(version file of each storage
> group)
> │   └── users  ————————————————————(information of users)
> |└── wal
> |    ├── {name_of_storage_group}-{name_of_TsFile} ———(a folder for one
> data file)
> |      └── wal{version}  ————————————————(wal file for a memtable in
> corresponding data file)
>
> Best,
> --
> Jialin Qiao
> School of Software, Tsinghua University
>
> 乔嘉林
> 清华大学 软件学院
>
> > -----原始邮件-----
> > 发件人: "Jialin Qiao" <qj...@mails.tsinghua.edu.cn>
> > 发送时间: 2019-07-03 10:24:43 (星期三)
> > 收件人: dev@iotdb.apache.org
> > 抄送:
> > 主题: New directory structure
> >
> > Hi,
> >
> > The directory structure of IoTDB is in chaos, so I reorganize it in
> PR[1].
> >
> > User configurations:
> >
> > base_dir: base directory of IoTDB ("data" by default)
> > sys_dir: system data directory ("data/system" by default)
> > data_dirs: data file directories, which supports multiple directories
> ("data/data" by default, multiple directories are separated by ',")
> > wal_dir: directory of write-ahead-log ("data/wal" by default)
> >
> > Under default configurations, the actual folders are like this:
> >
> >     data
> >         data    —————————————————————(data files)
> >             sequence  ——————————————————(folder of sequence data files)
> >                 storage_group_x ——————————————(storage group x)
> >                     {systemtime-version}.tsfile  ————————(data file)
> >                     {name_of_TsFile}.resource   ————————(resource file
> of corresponding tsfile)
> >             unsequence —————————————————(folder of unsequence data
> files, which the same structure with "sequence" directory)
> >         system
> >             roles  ————————————————————(information of roles)
> >             users ————————————————————(information of users)
> >             schema ———————————————————(system schema, including storage
> groups and timeseries)
> >             storage_groups   ———————————————(version file of each
> storage group)
> >         wal
> >             {name_of_storage_group}-{name_of_TsFile} ———(a folder for
> one data file)
> >                 wal{version}  ————————————————(wal file for a memtable
> in corresponding data file)
> >
> >
> > [1] https://github.com/apache/incubator-iotdb/pull/217
> >
> > Best,
> > --
> > Jialin Qiao
> > School of Software, Tsinghua University
> >
> > 乔嘉林
> > 清华大学 软件学院
>

Re: New directory structure

Posted by Christofer Dutz <ch...@c-ware.de>.
Hi all,

Please think of documenting this in any release notes as incompatible change.

Chris

Holen Sie sichOutlook für Android<https://aka.ms/ghei36>

________________________________
From: Jialin Qiao <qj...@mails.tsinghua.edu.cn>
Sent: Wednesday, July 3, 2019 4:46:18 AM
To: dev@iotdb.apache.org
Subject: Re: New directory structure

Hi,

The following illustration is more clear:

data  ―――――――――――――――――――――――――(base directory)
|├── data   ―――――――――――――――――――――(data files)
│   ├── sequence  ――――――――――――――――――(folder of sequence data files in each storage group)
│   │   └── storage_group_x ――――――――――――――(storage group x)
│   │       ├── {systemtime-version}.tsfile ――――――――(data file)
│   │       ├── {name_of_TsFile}.resource  ――――――――(resource file of corresponding tsfile)
│   └── unsequence ―――――――――――――――――(folder of unsequence data files, which the same structure with "sequence" directory)
|├── system
│   ├── roles  ――――――――――――――――――――(information of roles)
│   ├── schema ―――――――――――――――――――(system schema, including storage groups and timeseries)
│   ├── storage_groups   ―――――――――――――――(version file of each storage group)
│   └── users  ――――――――――――――――――――(information of users)
|└── wal
|    ├── {name_of_storage_group}-{name_of_TsFile} ―――(a folder for one data file)
|      └── wal{version}  ――――――――――――――――(wal file for a memtable in corresponding data file)

Best,
--
Jialin Qiao
School of Software, Tsinghua University

乔嘉林
清华大学 软件学院

> -----原始邮件-----
> 发件人: "Jialin Qiao" <qj...@mails.tsinghua.edu.cn>
> 发送时间: 2019-07-03 10:24:43 (星期三)
> 收件人: dev@iotdb.apache.org
> 抄送:
> 主题: New directory structure
>
> Hi,
>
> The directory structure of IoTDB is in chaos, so I reorganize it in PR[1].
>
> User configurations:
>
> base_dir: base directory of IoTDB ("data" by default)
> sys_dir: system data directory ("data/system" by default)
> data_dirs: data file directories, which supports multiple directories ("data/data" by default, multiple directories are separated by ',")
> wal_dir: directory of write-ahead-log ("data/wal" by default)
>
> Under default configurations, the actual folders are like this:
>
>     data
>         data    ―――――――――――――――――――――(data files)
>             sequence  ――――――――――――――――――(folder of sequence data files)
>                 storage_group_x ――――――――――――――(storage group x)
>                     {systemtime-version}.tsfile  ――――――――(data file)
>                     {name_of_TsFile}.resource   ――――――――(resource file of corresponding tsfile)
>             unsequence ―――――――――――――――――(folder of unsequence data files, which the same structure with "sequence" directory)
>         system
>             roles  ――――――――――――――――――――(information of roles)
>             users ――――――――――――――――――――(information of users)
>             schema ―――――――――――――――――――(system schema, including storage groups and timeseries)
>             storage_groups   ―――――――――――――――(version file of each storage group)
>         wal
>             {name_of_storage_group}-{name_of_TsFile} ―――(a folder for one data file)
>                 wal{version}  ――――――――――――――――(wal file for a memtable in corresponding data file)
>
>
> [1] https://github.com/apache/incubator-iotdb/pull/217
>
> Best,
> --
> Jialin Qiao
> School of Software, Tsinghua University
>
> 乔嘉林
> 清华大学 软件学院

Re: New directory structure

Posted by Jialin Qiao <qj...@mails.tsinghua.edu.cn>.
Hi,

The following illustration is more clear:

data  —————————————————————————(base directory)
|├── data   —————————————————————(data files)
│   ├── sequence  ——————————————————(folder of sequence data files in each storage group)
│   │   └── storage_group_x ——————————————(storage group x)
│   │       ├── {systemtime-version}.tsfile ————————(data file)
│   │       ├── {name_of_TsFile}.resource  ————————(resource file of corresponding tsfile)
│   └── unsequence —————————————————(folder of unsequence data files, which the same structure with "sequence" directory)
|├── system
│   ├── roles  ————————————————————(information of roles)
│   ├── schema ———————————————————(system schema, including storage groups and timeseries)
│   ├── storage_groups   ———————————————(version file of each storage group)
│   └── users  ————————————————————(information of users)
|└── wal
|    ├── {name_of_storage_group}-{name_of_TsFile} ———(a folder for one data file)
|      └── wal{version}  ————————————————(wal file for a memtable in corresponding data file)

Best,
--
Jialin Qiao
School of Software, Tsinghua University

乔嘉林
清华大学 软件学院

> -----原始邮件-----
> 发件人: "Jialin Qiao" <qj...@mails.tsinghua.edu.cn>
> 发送时间: 2019-07-03 10:24:43 (星期三)
> 收件人: dev@iotdb.apache.org
> 抄送: 
> 主题: New directory structure
> 
> Hi,
> 
> The directory structure of IoTDB is in chaos, so I reorganize it in PR[1].
> 
> User configurations:
> 
> base_dir: base directory of IoTDB ("data" by default)
> sys_dir: system data directory ("data/system" by default)
> data_dirs: data file directories, which supports multiple directories ("data/data" by default, multiple directories are separated by ',")
> wal_dir: directory of write-ahead-log ("data/wal" by default)
> 
> Under default configurations, the actual folders are like this:
> 
>     data
>         data    —————————————————————(data files)
>             sequence  ——————————————————(folder of sequence data files)
>                 storage_group_x ——————————————(storage group x)
>                     {systemtime-version}.tsfile  ————————(data file)
>                     {name_of_TsFile}.resource   ————————(resource file of corresponding tsfile)
>             unsequence —————————————————(folder of unsequence data files, which the same structure with "sequence" directory)
>         system
>             roles  ————————————————————(information of roles)
>             users ————————————————————(information of users)
>             schema ———————————————————(system schema, including storage groups and timeseries)
>             storage_groups   ———————————————(version file of each storage group)
>         wal
>             {name_of_storage_group}-{name_of_TsFile} ———(a folder for one data file)
>                 wal{version}  ————————————————(wal file for a memtable in corresponding data file)
> 
> 
> [1] https://github.com/apache/incubator-iotdb/pull/217
> 
> Best,
> --
> Jialin Qiao
> School of Software, Tsinghua University
> 
> 乔嘉林
> 清华大学 软件学院

New directory structure

Posted by Jialin Qiao <qj...@mails.tsinghua.edu.cn>.
Hi,

The directory structure of IoTDB is in chaos, so I reorganize it in PR[1].

User configurations:

base_dir: base directory of IoTDB ("data" by default)
sys_dir: system data directory ("data/system" by default)
data_dirs: data file directories, which supports multiple directories ("data/data" by default, multiple directories are separated by ',")
wal_dir: directory of write-ahead-log ("data/wal" by default)

Under default configurations, the actual folders are like this:

    data
        data    —————————————————————(data files)
            sequence  ——————————————————(folder of sequence data files)
                storage_group_x ——————————————(storage group x)
                    {systemtime-version}.tsfile  ————————(data file)
                    {name_of_TsFile}.resource   ————————(resource file of corresponding tsfile)
            unsequence —————————————————(folder of unsequence data files, which the same structure with "sequence" directory)
        system
            roles  ————————————————————(information of roles)
            users ————————————————————(information of users)
            schema ———————————————————(system schema, including storage groups and timeseries)
            storage_groups   ———————————————(version file of each storage group)
        wal
            {name_of_storage_group}-{name_of_TsFile} ———(a folder for one data file)
                wal{version}  ————————————————(wal file for a memtable in corresponding data file)


[1] https://github.com/apache/incubator-iotdb/pull/217

Best,
--
Jialin Qiao
School of Software, Tsinghua University

乔嘉林
清华大学 软件学院

Re: Re: Avoid long-tail insertion

Posted by 李天安 <lt...@mails.tsinghua.edu.cn>.
Hi, 
       Indeed, in this pr of new storage engine, data sync module is not included. After the PR of the new storage engine is merged into the master, I will add the data sync module and submit a pr of that.

Best Regards,
—————————————————
Tianan Li
School of Software, Tsinghua University 

> -----原始邮件-----
> 发件人: "Xiangdong Huang" <sa...@gmail.com>
> 发送时间: 2019-07-02 19:11:16 (星期二)
> 收件人: dev@iotdb.apache.org
> 抄送: 
> 主题: Re: Avoid long-tail insertion
> 
> Hi,
> 
> I have reviewed the codes. The new structure is far better than previous
> version.
> 
> A question is that,  it seems that the merge process and and data sync
> module are not finished in your branch.
> 
> If we merge this PR into the master branch, we need to finish the two
> modules ASAP..
> 
> Best,
> -----------------------------------
> Xiangdong Huang
> School of Software, Tsinghua University
> 
>  黄向东
> 清华大学 软件学院
> 
> 
> Jialin Qiao <qj...@mails.tsinghua.edu.cn> 于2019年7月2日周二 上午8:38写道:
> 
> > Hi,
> >
> > After several days' hard work, the new storage engine and corresponding
> > query engine is finally stable and behaves well...
> >
> > I have submitted a PR[1]. The new engine reduces the long-tail latency
> > from 50+s to under 5s. Detailed performance evaluation result will be
> > presented later.
> >
> > In this version, there are several new modules and strategies:
> >
> > Modules:
> >
> > (1) TVListPool: A cache of TVList, which is designed for reusing the
> > TVList and reducing the GC.
> > (2) PrimitiveArrayPool: A cache of arrays of primitive data type in java,
> > such as int[], long[], double[].
> > (3) TsFileResource: Each data file is denoted by a TsFileResource, which
> > is used for query.
> >
> > Strategies:
> >
> > (1) Asynchronously flushing memtable to disk.
> > (2) Asynchronously appending metadata and closing data file.
> >
> >
> > [1]https://github.com/apache/incubator-iotdb/pull/217
> >
> > Best,
> > --
> > Jialin Qiao
> > School of Software, Tsinghua University
> >
> > 乔嘉林
> > 清华大学 软件学院
> >
> > > -----原始邮件-----
> > > 发件人: "Xiangdong Huang" <sa...@gmail.com>
> > > 发送时间: 2019-06-28 20:45:44 (星期五)
> > > 收件人: dev@iotdb.apache.org
> > > 抄送:
> > > 主题: Re: Re: Avoid long-tail insertion
> > >
> > > I see. The following sentence is very important to guarantee the
> > > correctness...
> > >
> > > >  At the same time, resubmitting UFP after the end of the task in each
> > > Flush thread ensures that all tasks can be executed.
> > >
> > > I think it should be highlight listed, rather than be drown in the last
> > > paragraph.
> > >
> > > Anyway, now I understand what you want to do.
> > >
> > > Best,
> > > -----------------------------------
> > > Xiangdong Huang
> > > School of Software, Tsinghua University
> > >
> > >  黄向东
> > > 清华大学 软件学院
> > >
> > >
> > > Xiangdong Huang <sa...@gmail.com> 于2019年6月28日周五 下午8:41写道:
> > >
> > > > Hi Tianan,
> > > >
> > > > > the flush tasks in each UFP(There is a flushing queue in UFP) need
> > to be
> > > > executed sequentially.
> > > > > Flush thread polls the first UFP from UFP queue in FlushManager ->
> > polls
> > > > the first flush task in UFP -> completes the flush task -> set
> > > > ‘managedByFlushManager’  of the UFP to false.
> > > >
> > > > It indicates that there are more than one flush tasks in a UFP, but the
> > > > FlushManager  just spends one task from the UFP and then it mark the
> > UFP as
> > > > managedByFlushManager=false and poll it out of the queue? (So, when to
> > > > flush the rest tasks??)
> > > >
> > > > > (1) UFP is not managed by Flush Manager, i.e.'managedByFlushManager'
> > is
> > > > false
> > > >
> > > > If suppose the UFP has one flush task and enqueued the queue. Then the
> > UFP
> > > > has the second flush task. However, it can not be added into the queue
> > > > because managedByFlushManager == true. Using your above logic, you will
> > > > ignore the second flush task....
> > > >
> > > >
> > > > > Flush Manager first determines whether UFP meets the criteria for
> > > > submission
> > > >
> > > > Do you want to say both the two criteria should be satisfied? (If so,
> > the
> > > > above  hypothetical situation will occur).
> > > >
> > > > > Through the above design, we can ensure that at the same time for
> > each
> > > > UFP, Flush Manager will only manage at most once and execute at most
> > one
> > > > flush task, while there is no restriction between different UFPs.
> > > >
> > > > Using your design, we can ensure that at a certain time, given a UFP,
> > the
> > > > Flush Manager will only manage a UFP at most once and execute at most
> > one
> > > > flush task from the UFP, but how to avoid the above hypothetical
> > situation?
> > > >
> > > > Best,
> > > > -----------------------------------
> > > > Xiangdong Huang
> > > > School of Software, Tsinghua University
> > > >
> > > >  黄向东
> > > > 清华大学 软件学院
> > > >
> > > >
> > > > 李天安 <lt...@mails.tsinghua.edu.cn> 于2019年6月28日周五 上午11:29写道:
> > > >
> > > >> Hi,
> > > >>  I'm also involved in the design of the new storage engine. Let me
> > > >> complement the new design of the flush task.
> > > >>
> > > >>  To improve system performance, we changed flush tasks from
> > synchronous
> > > >> to asynchronous. We introduced a Flush Manager to manage all flush
> > tasks.
> > > >> The tricky problem is that each Unsealed TsFile Processor (UFP)
> > corresponds
> > > >> to a data file on a disk, so the flush tasks in each UFP(There is a
> > > >> flushing queue in UFP) need to be executed sequentially. However,
> > flush
> > > >> tasks in different UFPs have no sequential requirements. How to
> > design them
> > > >> to meet the above requirements?
> > > >>
> > > >>  We introduce a UFP FIFO queue in Flush Manager, and add a boolean
> > > >> attribute ‘managedByFlushManager’ to each UFP to indicate whether it
> > is
> > > >> managed by Flush Manager. Flush Manager maintains a Flush thread pool
> > to
> > > >> perform Flush tasks, so the lifecycle of a Flush task is
> > > >> 1. UFP are submitted to FlushManager,FlushManager add UFP to its queue
> > > >> and set ‘managedByFlushManager’  of the UFP to true.
> > > >> 2. The Flush Pool in FlushManager start a flush thread to execute
> > task.
> > > >> 3. Flush thread polls the first UFP from UFP queue in FlushManager ->
> > > >> polls the first flush task in UFP -> completes the flush task -> set
> > > >> ‘managedByFlushManager’  of the UFP to false.
> > > >>
> > > >> There are two ways to submit a UFP to FlushManager:
> > > >> 1. UFP, whenever a MemTable reaches a certain size or forcibly
> > triggers a
> > > >> flush task, it submits itself to Flush Manager (because the queue in
> > Flush
> > > >> Manager is UFP). Flush Manager first determines whether UFP meets the
> > > >> criteria for submission:
> > > >> (1) UFP is not managed by Flush Manager, i.e.'managedByFlushManager'
> > is
> > > >> false
> > > >> (2) The Flush task queue in UFP is not empty, that is, there are at
> > least
> > > >> one flush task to be executed.
> > > >>
> > > >> 2. When the Flush thread completes the flush task, it sets
> > > >> ‘managedByFlushManager’ to false and resubmits the UFP of the
> > completed
> > > >> flush task to the FlushManager.
> > > >>
> > > >> Through the above design, we can ensure that at the same time for each
> > > >> UFP, Flush Manager will only manage at most once and execute at most
> > one
> > > >> flush task, while there is no restriction between different UFPs. At
> > the
> > > >> same time, resubmitting UFP after the end of the task in each Flush
> > thread
> > > >> ensures that all tasks can be executed. Therefore, we solve the above
> > > >> problem and the design meets the requirements of Flush Manager.
> > > >>
> > > >> Best Regards,
> > > >> -------------------------------------
> > > >> Tianan Li
> > > >> School of Software, Tsinghua University
> > > >>
> > > >> > -----原始邮件-----
> > > >> > 发件人: "Jialin Qiao" <qj...@mails.tsinghua.edu.cn>
> > > >> > 发送时间: 2019-06-27 11:27:24 (星期四)
> > > >> > 收件人: dev@iotdb.apache.org
> > > >> > 抄送:
> > > >> > 主题: Re: Avoid long-tail insertion
> > > >> >
> > > >> > Hi,
> > > >> >
> > > >> > The new storage engine is designed to have the following components:
> > > >> >
> > > >> > (1) MemTable: A memory structure, which stores all inserted data in
> > > >> memory.
> > > >> >
> > > >> > (2) MemtablePool: Manages all memtables. All memtables are gotten
> > from
> > > >> this pool. The total number of memtables is fixed
> > > >> > in the system. Once the pool do not has available memtables, the
> > > >> getMemtable() operation will wait or directly return.
> > > >> >
> > > >> > (3) UnsealedTsFileProcessor (UFP): A writer for one data file. It
> > > >> always has one working memtable that receives writes and a
> > > >> > list (flushing list) of memtables that for flush. Once the working
> > > >> memtable reaches a threshold, it will be moved to the
> > > >> > flushing list and the working memtable is set null. When a new write
> > > >> arrives, if the working memtable is null, UFP will
> > > >> > call getMemtable() of the MemtablePool to get one as the working
> > > >> memtable.
> > > >> >
> > > >> > (4) StorageGroupProcessor (SGP): Each SGP is responsible for all
> > writes
> > > >> and reads in one storage group. It always has one
> > > >> > working UFP that receives write and a list (closing list) of UFPs
> > that
> > > >> for close. Once the file size of the working UFP reaches
> > > >> > a threshold, the UFP is moved to the closing list and the working
> > UFP
> > > >> is set null. When a new write arrives, if the working UFP
> > > >> > is null, a new UFP is generated as working UFP and receives write.
> > > >> >
> > > >> > (5) StorageGroupManager (SGM): A manager of all SGPs in IoTDB. It is
> > > >> only responsible for routing read and write operations
> > > >> > to its corresponding SGP according to the deviceId of the operation.
> > > >> >
> > > >> > (6) Flush thread: The flush thread poll a memtable from the flushing
> > > >> list in UFP and flush a memtable to disk. After flushing,
> > > >> > the memtable is returned to the MemtablePool.
> > > >> >
> > > >> > These are only the main components of the new storage engine. Some
> > > >> things may be lost. It would be great if someone could
> > > >> > give some advices or supplementations.
> > > >> >
> > > >> > Best,
> > > >> > --
> > > >> > Jialin Qiao
> > > >> > School of Software, Tsinghua University
> > > >> >
> > > >> > 乔嘉林
> > > >> > 清华大学 软件学院
> > > >> >
> > > >> > > -----原始邮件-----
> > > >> > > 发件人: "Jialin Qiao" <qj...@mails.tsinghua.edu.cn>
> > > >> > > 发送时间: 2019-06-24 20:24:05 (星期一)
> > > >> > > 收件人: dev@iotdb.apache.org
> > > >> > > 抄送:
> > > >> > > 主题: Re: Re: Re: Avoid long-tail insertion
> > > >> > >
> > > >> > >
> > > >> > > Yes, there are many changes. The branch I am working on is
> > > >> feature_async_close_tsfile.
> > > >> > > Anyone interested is welcome to join and discuss.
> > > >> > >
> > > >> > > Best,
> > > >> > > --
> > > >> > > Jialin Qiao
> > > >> > > School of Software, Tsinghua University
> > > >> > >
> > > >> > > 乔嘉林
> > > >> > > 清华大学 软件学院
> > > >> > >
> > > >> > > > -----原始邮件-----
> > > >> > > > 发件人: "Xiangdong Huang" <sa...@gmail.com>
> > > >> > > > 发送时间: 2019-06-23 10:59:29 (星期日)
> > > >> > > > 收件人: dev@iotdb.apache.org
> > > >> > > > 抄送:
> > > >> > > > 主题: Re: Re: Avoid long-tail insertion
> > > >> > > >
> > > >> > > > Hi,
> > > >> > > >
> > > >> > > > Once your work branch is almost ready, let me know so I can
> > help to
> > > >> review.
> > > >> > > > I think it is a HUGE PR...
> > > >> > > >
> > > >> > > > -----------------------------------
> > > >> > > > Xiangdong Huang
> > > >> > > > School of Software, Tsinghua University
> > > >> > > >
> > > >> > > >  黄向东
> > > >> > > > 清华大学 软件学院
> > > >> > > >
> > > >> > > >
> > > >> > > > Jialin Qiao <qj...@mails.tsinghua.edu.cn> 于2019年6月22日周六
> > 下午9:57写道:
> > > >> > > >
> > > >> > > > > Hi Xiangdong,
> > > >> > > > >
> > > >> > > > > I will merge this patch. Let "Directories" manage the folders
> > of
> > > >> both
> > > >> > > > > sequence and unSequence files is good.
> > > >> > > > >
> > > >> > > > > However, the naming of "Directories" is not clear. It would be
> > > >> better to
> > > >> > > > > rename to "DirectoryManager"
> > > >> > > > >
> > > >> > > > > Best,
> > > >> > > > > --
> > > >> > > > > Jialin Qiao
> > > >> > > > > School of Software, Tsinghua University
> > > >> > > > >
> > > >> > > > > 乔嘉林
> > > >> > > > > 清华大学 软件学院
> > > >> > > > >
> > > >> > > > > > -----原始邮件-----
> > > >> > > > > > 发件人: "Xiangdong Huang" <sa...@gmail.com>
> > > >> > > > > > 发送时间: 2019-06-22 16:35:29 (星期六)
> > > >> > > > > > 收件人: dev@iotdb.apache.org
> > > >> > > > > > 抄送:
> > > >> > > > > > 主题: Re: Avoid long-tail insertion
> > > >> > > > > >
> > > >> > > > > > Hi jialin,
> > > >> > > > > >
> > > >> > > > > > I submit some modifications for:
> > > >> > > > > >
> > > >> > > > > > * add the overflow data folder location setting in the
> > > >> > > > > > iotdb-engine.properties;
> > > >> > > > > > * let Directories.java to manage the above folder.
> > > >> > > > > >
> > > >> > > > > > If you need to refactor the overflow when you solving the
> > long
> > > >> tail
> > > >> > > > > issue,
> > > >> > > > > > you can apply the patch from [1] first to simplify your
> > work.
> > > >> > > > > >
> > > >> > > > > > [1]
> > > >> > > > > >
> > > >> > > > >
> > > >>
> > https://issues.apache.org/jira/secure/attachment/12972547/overflow-folder.patch
> > > >> > > > > >
> > > >> > > > > > Best,
> > > >> > > > > > -----------------------------------
> > > >> > > > > > Xiangdong Huang
> > > >> > > > > > School of Software, Tsinghua University
> > > >> > > > > >
> > > >> > > > > >  黄向东
> > > >> > > > > > 清华大学 软件学院
> > > >> > > > > >
> > > >> > > > > >
> > > >> > > > > > Xiangdong Huang <sa...@gmail.com> 于2019年6月22日周六
> > 下午3:19写道:
> > > >> > > > > >
> > > >> > > > > > > If you change the process like this, i.e., there are more
> > > >> than one
> > > >> > > > > > > unsealed TsFiles for each storage group, then  you have to
> > > >> modify the
> > > >> > > > > WAL
> > > >> > > > > > > module.. Because current WAL module only recognizes the
> > last
> > > >> unsealed
> > > >> > > > > > > TsFile..
> > > >> > > > > > >
> > > >> > > > > > > By the way, "sealed" is better than "closed", I think..  A
> > > >> sealed file
> > > >> > > > > > > means the file which has the magic string at the head and
> > the
> > > >> tail.
> > > >> > > > > > >
> > > >> > > > > > > Best,
> > > >> > > > > > > -----------------------------------
> > > >> > > > > > > Xiangdong Huang
> > > >> > > > > > > School of Software, Tsinghua University
> > > >> > > > > > >
> > > >> > > > > > >  黄向东
> > > >> > > > > > > 清华大学 软件学院
> > > >> > > > > > >
> > > >> > > > > > >
> > > >> > > > > > > Jialin Qiao <qj...@mails.tsinghua.edu.cn> 于2019年6月22日周六
> > > >> 下午2:54写道:
> > > >> > > > > > >
> > > >> > > > > > >>
> > > >> > > > > > >> Hi, I am solving the long-tail latency problem.
> > > >> > > > > > >>
> > > >> > > > > > >> There are some cases (blocking points) that blocking the
> > > >> insertion.
> > > >> > > > > For a
> > > >> > > > > > >> better understanding of this problem, I first introduce
> > the
> > > >> writing
> > > >> > > > > process
> > > >> > > > > > >> of IoTDB:
> > > >> > > > > > >>
> > > >> > > > > > >> IoTDB maintains several independent engines (storage
> > group)
> > > >> that
> > > >> > > > > supports
> > > >> > > > > > >> read and write. In the following, we focus on one
> > engine. A
> > > >> engine
> > > >> > > > > > >> maintains several closed data files and one unclosed data
> > > >> file that
> > > >> > > > > > >> receives appended data. In memory, there is only one
> > working
> > > >> memtable
> > > >> > > > > (m1)
> > > >> > > > > > >> that receives writes. There is also another memtable (m2)
> > > >> that will
> > > >> > > > > take
> > > >> > > > > > >> place m1 when m1 is full and being flushed.
> > > >> > > > > > >>
> > > >> > > > > > >> When a data item is inserted:
> > > >> > > > > > >>
> > > >> > > > > > >> (1)We insert it into the working memtable.
> > > >> > > > > > >> (2)We check the size of the memtable. If it reaches a
> > > >> threshold, we
> > > >> > > > > > >> submit a flush task “after the previous flush task is
> > > >> finished” and
> > > >> > > > > switch
> > > >> > > > > > >> the two memtables.
> > > >> > > > > > >> (3)We check the size of the unclosed file. If it reaches
> > a
> > > >> threshold,
> > > >> > > > > we
> > > >> > > > > > >> close it “after the previous flush task is finished”.
> > > >> > > > > > >>
> > > >> > > > > > >> In the above steps, all the "after the previous flush
> > task is
> > > >> > > > > finished"
> > > >> > > > > > >> will block the insertion process. One solution is to make
> > > >> all flush
> > > >> > > > > and
> > > >> > > > > > >> close task asynchronous. Some questions need to carefully
> > > >> considered:
> > > >> > > > > > >>
> > > >> > > > > > >> (1) Many memtables may be flushed concurrently to an
> > > >> unclosed file.
> > > >> > > > > How
> > > >> > > > > > >> to guarantee the order of serialization?
> > > >> > > > > > >> (2) Once a close task is submitted, a new unclosed file
> > will
> > > >> be
> > > >> > > > > created
> > > >> > > > > > >> and receives appended data. So there will exists many
> > > >> unclosed files.
> > > >> > > > > How
> > > >> > > > > > >> the query and compaction process will be impacted?
> > > >> > > > > > >>
> > > >> > > > > > >> Thanks,
> > > >> > > > > > >>
> > > >> > > > > > >> Jialin Qiao
> > > >> > > > > > >> School of Software, Tsinghua University
> > > >> > > > > > >>
> > > >> > > > > > >> 乔嘉林
> > > >> > > > > > >> 清华大学 软件学院
> > > >> > > > > > >>
> > > >> > > > > > >> > -----原始邮件-----
> > > >> > > > > > >> > 发件人: "Xiangdong Huang" <sa...@gmail.com>
> > > >> > > > > > >> > 发送时间: 2019-06-04 23:08:34 (星期二)
> > > >> > > > > > >> > 收件人: dev@iotdb.apache.org, "江天" <jt...@163.com>
> > > >> > > > > > >> > 抄送:
> > > >> > > > > > >> > 主题: Re: [jira] [Created] (IOTDB-112) Avoid long tail
> > > >> insertion
> > > >> > > > > which is
> > > >> > > > > > >> caused by synchronized close-bufferwrite
> > > >> > > > > > >> >
> > > >> > > > > > >> > I attached the histogram of the latency in the JIRA.
> > > >> > > > > > >> >
> > > >> > > > > > >> > The x-axis is the latency while the y-axis is the
> > > >> cumulative
> > > >> > > > > > >> distribution.
> > > >> > > > > > >> > We can see that about 30% insertion can be finished in
> > > >> 20ms, and 60%
> > > >> > > > > > >> > insertion can be finished in 40ms even though the IoTDB
> > > >> instance is
> > > >> > > > > > >> serving
> > > >> > > > > > >> > for a heavy workload... So, eliminating the long tail
> > > >> insertion can
> > > >> > > > > make
> > > >> > > > > > >> > the average latency far better.
> > > >> > > > > > >> >
> > > >> > > > > > >> > If someone is working on the refactor_overflow or
> > > >> > > > > refactor_bufferwrite,
> > > >> > > > > > >> > please pay attention to the code branch for this issue.
> > > >> > > > > > >> >
> > > >> > > > > > >> > Best,
> > > >> > > > > > >> >
> > > >> > > > > > >> > -----------------------------------
> > > >> > > > > > >> > Xiangdong Huang
> > > >> > > > > > >> > School of Software, Tsinghua University
> > > >> > > > > > >> >
> > > >> > > > > > >> >  黄向东
> > > >> > > > > > >> > 清华大学 软件学院
> > > >> > > > > > >> >
> > > >> > > > > > >> >
> > > >> > > > > > >> > xiangdong Huang (JIRA) <ji...@apache.org> 于2019年6月4日周二
> > > >> 下午11:00写道:
> > > >> > > > > > >> >
> > > >> > > > > > >> > > xiangdong Huang created IOTDB-112:
> > > >> > > > > > >> > > -------------------------------------
> > > >> > > > > > >> > >
> > > >> > > > > > >> > >              Summary: Avoid long tail insertion
> > which is
> > > >> caused by
> > > >> > > > > > >> > > synchronized close-bufferwrite
> > > >> > > > > > >> > >                  Key: IOTDB-112
> > > >> > > > > > >> > >                  URL:
> > > >> > > > > https://issues.apache.org/jira/browse/IOTDB-112
> > > >> > > > > > >> > >              Project: Apache IoTDB
> > > >> > > > > > >> > >           Issue Type: Improvement
> > > >> > > > > > >> > >             Reporter: xiangdong Huang
> > > >> > > > > > >> > >
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > In our test, IoTDB has a good insertion performance,
> > and
> > > >> the
> > > >> > > > > average
> > > >> > > > > > >> > > latency can be ~200 ms in a given workload and
> > hardware.
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > However, when we draw the histogram of the latency,
> > we
> > > >> find that
> > > >> > > > > 97.5%
> > > >> > > > > > >> > > latencies are less than 200 ms, while 2.7% latencies
> > are
> > > >> greater.
> > > >> > > > > The
> > > >> > > > > > >> > > result shows that there are some long tail latency.
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > Then we find that some insertion latencies are about
> > 30
> > > >> seconds...
> > > >> > > > > > >> (but
> > > >> > > > > > >> > > the ratio is less than 0.5%). Indeed, for each
> > > >> connection, a long
> > > >> > > > > tail
> > > >> > > > > > >> > > insertion appears per 1 or 2 minutes....
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > By reading source codes, I think it is because that
> > in
> > > >> the
> > > >> > > > > insertion
> > > >> > > > > > >> > > function,
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > `private void insertBufferWrite(FileNodeProcessor
> > > >> > > > > fileNodeProcessor,
> > > >> > > > > > >> long
> > > >> > > > > > >> > > timestamp,
> > > >> > > > > > >> > >  boolean isMonitor, TSRecord tsRecord, String
> > deviceId)`,
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > if the corresponding TsFile is too large, the
> > function
> > > >> is blocked
> > > >> > > > > > >> until
> > > >> > > > > > >> > > the memtable is flushed on disk and the TsFile is
> > sealed
> > > >> (we call
> > > >> > > > > it
> > > >> > > > > > >> as
> > > >> > > > > > >> > > closing a TsFile). The latencies of the long tail
> > > >> insertions are
> > > >> > > > > very
> > > >> > > > > > >> close
> > > >> > > > > > >> > > to the time cost of flushing and sealing a TsFile.
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > So, if we set the closing function using the async
> > mode,
> > > >> we can
> > > >> > > > > avoid
> > > >> > > > > > >> the
> > > >> > > > > > >> > > long tail insertion.
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > However,  there are some side effects we have to fix:
> > > >> > > > > > >> > >  # At the same time, if a new insertion comes, then a
> > > >> new memtable
> > > >> > > > > > >> should
> > > >> > > > > > >> > > be assigned, and a new unsealed TsFile is created;
> > > >> > > > > > >> > >  # That means that there are more than 1 unsealed
> > > >> TsFiles if the
> > > >> > > > > > >> system is
> > > >> > > > > > >> > > crashed before the closing function is finished. So,
> > we
> > > >> have to
> > > >> > > > > > >> modify the
> > > >> > > > > > >> > > startup process to recover these files.
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > Is there any other side effect that I have to pay
> > > >> attention to?
> > > >> > > > > > >> > >
> > > >> > > > > > >> > >
> > > >> > > > > > >> > >
> > > >> > > > > > >> > > --
> > > >> > > > > > >> > > This message was sent by Atlassian JIRA
> > > >> > > > > > >> > > (v7.6.3#76005)
> > > >> > > > > > >> > >
> > > >> > > > > > >>
> > > >> > > > > > >
> > > >> > > > >
> > > >>
> > > >
> >

Re: Avoid long-tail insertion

Posted by Xiangdong Huang <sa...@gmail.com>.
Hi,

I have reviewed the codes. The new structure is far better than previous
version.

A question is that,  it seems that the merge process and and data sync
module are not finished in your branch.

If we merge this PR into the master branch, we need to finish the two
modules ASAP..

Best,
-----------------------------------
Xiangdong Huang
School of Software, Tsinghua University

 黄向东
清华大学 软件学院


Jialin Qiao <qj...@mails.tsinghua.edu.cn> 于2019年7月2日周二 上午8:38写道:

> Hi,
>
> After several days' hard work, the new storage engine and corresponding
> query engine is finally stable and behaves well...
>
> I have submitted a PR[1]. The new engine reduces the long-tail latency
> from 50+s to under 5s. Detailed performance evaluation result will be
> presented later.
>
> In this version, there are several new modules and strategies:
>
> Modules:
>
> (1) TVListPool: A cache of TVList, which is designed for reusing the
> TVList and reducing the GC.
> (2) PrimitiveArrayPool: A cache of arrays of primitive data type in java,
> such as int[], long[], double[].
> (3) TsFileResource: Each data file is denoted by a TsFileResource, which
> is used for query.
>
> Strategies:
>
> (1) Asynchronously flushing memtable to disk.
> (2) Asynchronously appending metadata and closing data file.
>
>
> [1]https://github.com/apache/incubator-iotdb/pull/217
>
> Best,
> --
> Jialin Qiao
> School of Software, Tsinghua University
>
> 乔嘉林
> 清华大学 软件学院
>
> > -----原始邮件-----
> > 发件人: "Xiangdong Huang" <sa...@gmail.com>
> > 发送时间: 2019-06-28 20:45:44 (星期五)
> > 收件人: dev@iotdb.apache.org
> > 抄送:
> > 主题: Re: Re: Avoid long-tail insertion
> >
> > I see. The following sentence is very important to guarantee the
> > correctness...
> >
> > >  At the same time, resubmitting UFP after the end of the task in each
> > Flush thread ensures that all tasks can be executed.
> >
> > I think it should be highlight listed, rather than be drown in the last
> > paragraph.
> >
> > Anyway, now I understand what you want to do.
> >
> > Best,
> > -----------------------------------
> > Xiangdong Huang
> > School of Software, Tsinghua University
> >
> >  黄向东
> > 清华大学 软件学院
> >
> >
> > Xiangdong Huang <sa...@gmail.com> 于2019年6月28日周五 下午8:41写道:
> >
> > > Hi Tianan,
> > >
> > > > the flush tasks in each UFP(There is a flushing queue in UFP) need
> to be
> > > executed sequentially.
> > > > Flush thread polls the first UFP from UFP queue in FlushManager ->
> polls
> > > the first flush task in UFP -> completes the flush task -> set
> > > ‘managedByFlushManager’  of the UFP to false.
> > >
> > > It indicates that there are more than one flush tasks in a UFP, but the
> > > FlushManager  just spends one task from the UFP and then it mark the
> UFP as
> > > managedByFlushManager=false and poll it out of the queue? (So, when to
> > > flush the rest tasks??)
> > >
> > > > (1) UFP is not managed by Flush Manager, i.e.'managedByFlushManager'
> is
> > > false
> > >
> > > If suppose the UFP has one flush task and enqueued the queue. Then the
> UFP
> > > has the second flush task. However, it can not be added into the queue
> > > because managedByFlushManager == true. Using your above logic, you will
> > > ignore the second flush task....
> > >
> > >
> > > > Flush Manager first determines whether UFP meets the criteria for
> > > submission
> > >
> > > Do you want to say both the two criteria should be satisfied? (If so,
> the
> > > above  hypothetical situation will occur).
> > >
> > > > Through the above design, we can ensure that at the same time for
> each
> > > UFP, Flush Manager will only manage at most once and execute at most
> one
> > > flush task, while there is no restriction between different UFPs.
> > >
> > > Using your design, we can ensure that at a certain time, given a UFP,
> the
> > > Flush Manager will only manage a UFP at most once and execute at most
> one
> > > flush task from the UFP, but how to avoid the above hypothetical
> situation?
> > >
> > > Best,
> > > -----------------------------------
> > > Xiangdong Huang
> > > School of Software, Tsinghua University
> > >
> > >  黄向东
> > > 清华大学 软件学院
> > >
> > >
> > > 李天安 <lt...@mails.tsinghua.edu.cn> 于2019年6月28日周五 上午11:29写道:
> > >
> > >> Hi,
> > >>  I'm also involved in the design of the new storage engine. Let me
> > >> complement the new design of the flush task.
> > >>
> > >>  To improve system performance, we changed flush tasks from
> synchronous
> > >> to asynchronous. We introduced a Flush Manager to manage all flush
> tasks.
> > >> The tricky problem is that each Unsealed TsFile Processor (UFP)
> corresponds
> > >> to a data file on a disk, so the flush tasks in each UFP(There is a
> > >> flushing queue in UFP) need to be executed sequentially. However,
> flush
> > >> tasks in different UFPs have no sequential requirements. How to
> design them
> > >> to meet the above requirements?
> > >>
> > >>  We introduce a UFP FIFO queue in Flush Manager, and add a boolean
> > >> attribute ‘managedByFlushManager’ to each UFP to indicate whether it
> is
> > >> managed by Flush Manager. Flush Manager maintains a Flush thread pool
> to
> > >> perform Flush tasks, so the lifecycle of a Flush task is
> > >> 1. UFP are submitted to FlushManager,FlushManager add UFP to its queue
> > >> and set ‘managedByFlushManager’  of the UFP to true.
> > >> 2. The Flush Pool in FlushManager start a flush thread to execute
> task.
> > >> 3. Flush thread polls the first UFP from UFP queue in FlushManager ->
> > >> polls the first flush task in UFP -> completes the flush task -> set
> > >> ‘managedByFlushManager’  of the UFP to false.
> > >>
> > >> There are two ways to submit a UFP to FlushManager:
> > >> 1. UFP, whenever a MemTable reaches a certain size or forcibly
> triggers a
> > >> flush task, it submits itself to Flush Manager (because the queue in
> Flush
> > >> Manager is UFP). Flush Manager first determines whether UFP meets the
> > >> criteria for submission:
> > >> (1) UFP is not managed by Flush Manager, i.e.'managedByFlushManager'
> is
> > >> false
> > >> (2) The Flush task queue in UFP is not empty, that is, there are at
> least
> > >> one flush task to be executed.
> > >>
> > >> 2. When the Flush thread completes the flush task, it sets
> > >> ‘managedByFlushManager’ to false and resubmits the UFP of the
> completed
> > >> flush task to the FlushManager.
> > >>
> > >> Through the above design, we can ensure that at the same time for each
> > >> UFP, Flush Manager will only manage at most once and execute at most
> one
> > >> flush task, while there is no restriction between different UFPs. At
> the
> > >> same time, resubmitting UFP after the end of the task in each Flush
> thread
> > >> ensures that all tasks can be executed. Therefore, we solve the above
> > >> problem and the design meets the requirements of Flush Manager.
> > >>
> > >> Best Regards,
> > >> -------------------------------------
> > >> Tianan Li
> > >> School of Software, Tsinghua University
> > >>
> > >> > -----原始邮件-----
> > >> > 发件人: "Jialin Qiao" <qj...@mails.tsinghua.edu.cn>
> > >> > 发送时间: 2019-06-27 11:27:24 (星期四)
> > >> > 收件人: dev@iotdb.apache.org
> > >> > 抄送:
> > >> > 主题: Re: Avoid long-tail insertion
> > >> >
> > >> > Hi,
> > >> >
> > >> > The new storage engine is designed to have the following components:
> > >> >
> > >> > (1) MemTable: A memory structure, which stores all inserted data in
> > >> memory.
> > >> >
> > >> > (2) MemtablePool: Manages all memtables. All memtables are gotten
> from
> > >> this pool. The total number of memtables is fixed
> > >> > in the system. Once the pool do not has available memtables, the
> > >> getMemtable() operation will wait or directly return.
> > >> >
> > >> > (3) UnsealedTsFileProcessor (UFP): A writer for one data file. It
> > >> always has one working memtable that receives writes and a
> > >> > list (flushing list) of memtables that for flush. Once the working
> > >> memtable reaches a threshold, it will be moved to the
> > >> > flushing list and the working memtable is set null. When a new write
> > >> arrives, if the working memtable is null, UFP will
> > >> > call getMemtable() of the MemtablePool to get one as the working
> > >> memtable.
> > >> >
> > >> > (4) StorageGroupProcessor (SGP): Each SGP is responsible for all
> writes
> > >> and reads in one storage group. It always has one
> > >> > working UFP that receives write and a list (closing list) of UFPs
> that
> > >> for close. Once the file size of the working UFP reaches
> > >> > a threshold, the UFP is moved to the closing list and the working
> UFP
> > >> is set null. When a new write arrives, if the working UFP
> > >> > is null, a new UFP is generated as working UFP and receives write.
> > >> >
> > >> > (5) StorageGroupManager (SGM): A manager of all SGPs in IoTDB. It is
> > >> only responsible for routing read and write operations
> > >> > to its corresponding SGP according to the deviceId of the operation.
> > >> >
> > >> > (6) Flush thread: The flush thread poll a memtable from the flushing
> > >> list in UFP and flush a memtable to disk. After flushing,
> > >> > the memtable is returned to the MemtablePool.
> > >> >
> > >> > These are only the main components of the new storage engine. Some
> > >> things may be lost. It would be great if someone could
> > >> > give some advices or supplementations.
> > >> >
> > >> > Best,
> > >> > --
> > >> > Jialin Qiao
> > >> > School of Software, Tsinghua University
> > >> >
> > >> > 乔嘉林
> > >> > 清华大学 软件学院
> > >> >
> > >> > > -----原始邮件-----
> > >> > > 发件人: "Jialin Qiao" <qj...@mails.tsinghua.edu.cn>
> > >> > > 发送时间: 2019-06-24 20:24:05 (星期一)
> > >> > > 收件人: dev@iotdb.apache.org
> > >> > > 抄送:
> > >> > > 主题: Re: Re: Re: Avoid long-tail insertion
> > >> > >
> > >> > >
> > >> > > Yes, there are many changes. The branch I am working on is
> > >> feature_async_close_tsfile.
> > >> > > Anyone interested is welcome to join and discuss.
> > >> > >
> > >> > > Best,
> > >> > > --
> > >> > > Jialin Qiao
> > >> > > School of Software, Tsinghua University
> > >> > >
> > >> > > 乔嘉林
> > >> > > 清华大学 软件学院
> > >> > >
> > >> > > > -----原始邮件-----
> > >> > > > 发件人: "Xiangdong Huang" <sa...@gmail.com>
> > >> > > > 发送时间: 2019-06-23 10:59:29 (星期日)
> > >> > > > 收件人: dev@iotdb.apache.org
> > >> > > > 抄送:
> > >> > > > 主题: Re: Re: Avoid long-tail insertion
> > >> > > >
> > >> > > > Hi,
> > >> > > >
> > >> > > > Once your work branch is almost ready, let me know so I can
> help to
> > >> review.
> > >> > > > I think it is a HUGE PR...
> > >> > > >
> > >> > > > -----------------------------------
> > >> > > > Xiangdong Huang
> > >> > > > School of Software, Tsinghua University
> > >> > > >
> > >> > > >  黄向东
> > >> > > > 清华大学 软件学院
> > >> > > >
> > >> > > >
> > >> > > > Jialin Qiao <qj...@mails.tsinghua.edu.cn> 于2019年6月22日周六
> 下午9:57写道:
> > >> > > >
> > >> > > > > Hi Xiangdong,
> > >> > > > >
> > >> > > > > I will merge this patch. Let "Directories" manage the folders
> of
> > >> both
> > >> > > > > sequence and unSequence files is good.
> > >> > > > >
> > >> > > > > However, the naming of "Directories" is not clear. It would be
> > >> better to
> > >> > > > > rename to "DirectoryManager"
> > >> > > > >
> > >> > > > > Best,
> > >> > > > > --
> > >> > > > > Jialin Qiao
> > >> > > > > School of Software, Tsinghua University
> > >> > > > >
> > >> > > > > 乔嘉林
> > >> > > > > 清华大学 软件学院
> > >> > > > >
> > >> > > > > > -----原始邮件-----
> > >> > > > > > 发件人: "Xiangdong Huang" <sa...@gmail.com>
> > >> > > > > > 发送时间: 2019-06-22 16:35:29 (星期六)
> > >> > > > > > 收件人: dev@iotdb.apache.org
> > >> > > > > > 抄送:
> > >> > > > > > 主题: Re: Avoid long-tail insertion
> > >> > > > > >
> > >> > > > > > Hi jialin,
> > >> > > > > >
> > >> > > > > > I submit some modifications for:
> > >> > > > > >
> > >> > > > > > * add the overflow data folder location setting in the
> > >> > > > > > iotdb-engine.properties;
> > >> > > > > > * let Directories.java to manage the above folder.
> > >> > > > > >
> > >> > > > > > If you need to refactor the overflow when you solving the
> long
> > >> tail
> > >> > > > > issue,
> > >> > > > > > you can apply the patch from [1] first to simplify your
> work.
> > >> > > > > >
> > >> > > > > > [1]
> > >> > > > > >
> > >> > > > >
> > >>
> https://issues.apache.org/jira/secure/attachment/12972547/overflow-folder.patch
> > >> > > > > >
> > >> > > > > > Best,
> > >> > > > > > -----------------------------------
> > >> > > > > > Xiangdong Huang
> > >> > > > > > School of Software, Tsinghua University
> > >> > > > > >
> > >> > > > > >  黄向东
> > >> > > > > > 清华大学 软件学院
> > >> > > > > >
> > >> > > > > >
> > >> > > > > > Xiangdong Huang <sa...@gmail.com> 于2019年6月22日周六
> 下午3:19写道:
> > >> > > > > >
> > >> > > > > > > If you change the process like this, i.e., there are more
> > >> than one
> > >> > > > > > > unsealed TsFiles for each storage group, then  you have to
> > >> modify the
> > >> > > > > WAL
> > >> > > > > > > module.. Because current WAL module only recognizes the
> last
> > >> unsealed
> > >> > > > > > > TsFile..
> > >> > > > > > >
> > >> > > > > > > By the way, "sealed" is better than "closed", I think..  A
> > >> sealed file
> > >> > > > > > > means the file which has the magic string at the head and
> the
> > >> tail.
> > >> > > > > > >
> > >> > > > > > > Best,
> > >> > > > > > > -----------------------------------
> > >> > > > > > > Xiangdong Huang
> > >> > > > > > > School of Software, Tsinghua University
> > >> > > > > > >
> > >> > > > > > >  黄向东
> > >> > > > > > > 清华大学 软件学院
> > >> > > > > > >
> > >> > > > > > >
> > >> > > > > > > Jialin Qiao <qj...@mails.tsinghua.edu.cn> 于2019年6月22日周六
> > >> 下午2:54写道:
> > >> > > > > > >
> > >> > > > > > >>
> > >> > > > > > >> Hi, I am solving the long-tail latency problem.
> > >> > > > > > >>
> > >> > > > > > >> There are some cases (blocking points) that blocking the
> > >> insertion.
> > >> > > > > For a
> > >> > > > > > >> better understanding of this problem, I first introduce
> the
> > >> writing
> > >> > > > > process
> > >> > > > > > >> of IoTDB:
> > >> > > > > > >>
> > >> > > > > > >> IoTDB maintains several independent engines (storage
> group)
> > >> that
> > >> > > > > supports
> > >> > > > > > >> read and write. In the following, we focus on one
> engine. A
> > >> engine
> > >> > > > > > >> maintains several closed data files and one unclosed data
> > >> file that
> > >> > > > > > >> receives appended data. In memory, there is only one
> working
> > >> memtable
> > >> > > > > (m1)
> > >> > > > > > >> that receives writes. There is also another memtable (m2)
> > >> that will
> > >> > > > > take
> > >> > > > > > >> place m1 when m1 is full and being flushed.
> > >> > > > > > >>
> > >> > > > > > >> When a data item is inserted:
> > >> > > > > > >>
> > >> > > > > > >> (1)We insert it into the working memtable.
> > >> > > > > > >> (2)We check the size of the memtable. If it reaches a
> > >> threshold, we
> > >> > > > > > >> submit a flush task “after the previous flush task is
> > >> finished” and
> > >> > > > > switch
> > >> > > > > > >> the two memtables.
> > >> > > > > > >> (3)We check the size of the unclosed file. If it reaches
> a
> > >> threshold,
> > >> > > > > we
> > >> > > > > > >> close it “after the previous flush task is finished”.
> > >> > > > > > >>
> > >> > > > > > >> In the above steps, all the "after the previous flush
> task is
> > >> > > > > finished"
> > >> > > > > > >> will block the insertion process. One solution is to make
> > >> all flush
> > >> > > > > and
> > >> > > > > > >> close task asynchronous. Some questions need to carefully
> > >> considered:
> > >> > > > > > >>
> > >> > > > > > >> (1) Many memtables may be flushed concurrently to an
> > >> unclosed file.
> > >> > > > > How
> > >> > > > > > >> to guarantee the order of serialization?
> > >> > > > > > >> (2) Once a close task is submitted, a new unclosed file
> will
> > >> be
> > >> > > > > created
> > >> > > > > > >> and receives appended data. So there will exists many
> > >> unclosed files.
> > >> > > > > How
> > >> > > > > > >> the query and compaction process will be impacted?
> > >> > > > > > >>
> > >> > > > > > >> Thanks,
> > >> > > > > > >>
> > >> > > > > > >> Jialin Qiao
> > >> > > > > > >> School of Software, Tsinghua University
> > >> > > > > > >>
> > >> > > > > > >> 乔嘉林
> > >> > > > > > >> 清华大学 软件学院
> > >> > > > > > >>
> > >> > > > > > >> > -----原始邮件-----
> > >> > > > > > >> > 发件人: "Xiangdong Huang" <sa...@gmail.com>
> > >> > > > > > >> > 发送时间: 2019-06-04 23:08:34 (星期二)
> > >> > > > > > >> > 收件人: dev@iotdb.apache.org, "江天" <jt...@163.com>
> > >> > > > > > >> > 抄送:
> > >> > > > > > >> > 主题: Re: [jira] [Created] (IOTDB-112) Avoid long tail
> > >> insertion
> > >> > > > > which is
> > >> > > > > > >> caused by synchronized close-bufferwrite
> > >> > > > > > >> >
> > >> > > > > > >> > I attached the histogram of the latency in the JIRA.
> > >> > > > > > >> >
> > >> > > > > > >> > The x-axis is the latency while the y-axis is the
> > >> cumulative
> > >> > > > > > >> distribution.
> > >> > > > > > >> > We can see that about 30% insertion can be finished in
> > >> 20ms, and 60%
> > >> > > > > > >> > insertion can be finished in 40ms even though the IoTDB
> > >> instance is
> > >> > > > > > >> serving
> > >> > > > > > >> > for a heavy workload... So, eliminating the long tail
> > >> insertion can
> > >> > > > > make
> > >> > > > > > >> > the average latency far better.
> > >> > > > > > >> >
> > >> > > > > > >> > If someone is working on the refactor_overflow or
> > >> > > > > refactor_bufferwrite,
> > >> > > > > > >> > please pay attention to the code branch for this issue.
> > >> > > > > > >> >
> > >> > > > > > >> > Best,
> > >> > > > > > >> >
> > >> > > > > > >> > -----------------------------------
> > >> > > > > > >> > Xiangdong Huang
> > >> > > > > > >> > School of Software, Tsinghua University
> > >> > > > > > >> >
> > >> > > > > > >> >  黄向东
> > >> > > > > > >> > 清华大学 软件学院
> > >> > > > > > >> >
> > >> > > > > > >> >
> > >> > > > > > >> > xiangdong Huang (JIRA) <ji...@apache.org> 于2019年6月4日周二
> > >> 下午11:00写道:
> > >> > > > > > >> >
> > >> > > > > > >> > > xiangdong Huang created IOTDB-112:
> > >> > > > > > >> > > -------------------------------------
> > >> > > > > > >> > >
> > >> > > > > > >> > >              Summary: Avoid long tail insertion
> which is
> > >> caused by
> > >> > > > > > >> > > synchronized close-bufferwrite
> > >> > > > > > >> > >                  Key: IOTDB-112
> > >> > > > > > >> > >                  URL:
> > >> > > > > https://issues.apache.org/jira/browse/IOTDB-112
> > >> > > > > > >> > >              Project: Apache IoTDB
> > >> > > > > > >> > >           Issue Type: Improvement
> > >> > > > > > >> > >             Reporter: xiangdong Huang
> > >> > > > > > >> > >
> > >> > > > > > >> > >
> > >> > > > > > >> > > In our test, IoTDB has a good insertion performance,
> and
> > >> the
> > >> > > > > average
> > >> > > > > > >> > > latency can be ~200 ms in a given workload and
> hardware.
> > >> > > > > > >> > >
> > >> > > > > > >> > > However, when we draw the histogram of the latency,
> we
> > >> find that
> > >> > > > > 97.5%
> > >> > > > > > >> > > latencies are less than 200 ms, while 2.7% latencies
> are
> > >> greater.
> > >> > > > > The
> > >> > > > > > >> > > result shows that there are some long tail latency.
> > >> > > > > > >> > >
> > >> > > > > > >> > > Then we find that some insertion latencies are about
> 30
> > >> seconds...
> > >> > > > > > >> (but
> > >> > > > > > >> > > the ratio is less than 0.5%). Indeed, for each
> > >> connection, a long
> > >> > > > > tail
> > >> > > > > > >> > > insertion appears per 1 or 2 minutes....
> > >> > > > > > >> > >
> > >> > > > > > >> > > By reading source codes, I think it is because that
> in
> > >> the
> > >> > > > > insertion
> > >> > > > > > >> > > function,
> > >> > > > > > >> > >
> > >> > > > > > >> > > `private void insertBufferWrite(FileNodeProcessor
> > >> > > > > fileNodeProcessor,
> > >> > > > > > >> long
> > >> > > > > > >> > > timestamp,
> > >> > > > > > >> > >  boolean isMonitor, TSRecord tsRecord, String
> deviceId)`,
> > >> > > > > > >> > >
> > >> > > > > > >> > > if the corresponding TsFile is too large, the
> function
> > >> is blocked
> > >> > > > > > >> until
> > >> > > > > > >> > > the memtable is flushed on disk and the TsFile is
> sealed
> > >> (we call
> > >> > > > > it
> > >> > > > > > >> as
> > >> > > > > > >> > > closing a TsFile). The latencies of the long tail
> > >> insertions are
> > >> > > > > very
> > >> > > > > > >> close
> > >> > > > > > >> > > to the time cost of flushing and sealing a TsFile.
> > >> > > > > > >> > >
> > >> > > > > > >> > > So, if we set the closing function using the async
> mode,
> > >> we can
> > >> > > > > avoid
> > >> > > > > > >> the
> > >> > > > > > >> > > long tail insertion.
> > >> > > > > > >> > >
> > >> > > > > > >> > > However,  there are some side effects we have to fix:
> > >> > > > > > >> > >  # At the same time, if a new insertion comes, then a
> > >> new memtable
> > >> > > > > > >> should
> > >> > > > > > >> > > be assigned, and a new unsealed TsFile is created;
> > >> > > > > > >> > >  # That means that there are more than 1 unsealed
> > >> TsFiles if the
> > >> > > > > > >> system is
> > >> > > > > > >> > > crashed before the closing function is finished. So,
> we
> > >> have to
> > >> > > > > > >> modify the
> > >> > > > > > >> > > startup process to recover these files.
> > >> > > > > > >> > >
> > >> > > > > > >> > > Is there any other side effect that I have to pay
> > >> attention to?
> > >> > > > > > >> > >
> > >> > > > > > >> > >
> > >> > > > > > >> > >
> > >> > > > > > >> > > --
> > >> > > > > > >> > > This message was sent by Atlassian JIRA
> > >> > > > > > >> > > (v7.6.3#76005)
> > >> > > > > > >> > >
> > >> > > > > > >>
> > >> > > > > > >
> > >> > > > >
> > >>
> > >
>

Re: Avoid long-tail insertion

Posted by Jialin Qiao <qj...@mails.tsinghua.edu.cn>.
Hi,

After several days' hard work, the new storage engine and corresponding query engine is finally stable and behaves well...

I have submitted a PR[1]. The new engine reduces the long-tail latency from 50+s to under 5s. Detailed performance evaluation result will be presented later.

In this version, there are several new modules and strategies:

Modules:

(1) TVListPool: A cache of TVList, which is designed for reusing the TVList and reducing the GC.
(2) PrimitiveArrayPool: A cache of arrays of primitive data type in java, such as int[], long[], double[].
(3) TsFileResource: Each data file is denoted by a TsFileResource, which is used for query.

Strategies:

(1) Asynchronously flushing memtable to disk.
(2) Asynchronously appending metadata and closing data file.


[1]https://github.com/apache/incubator-iotdb/pull/217

Best,
--
Jialin Qiao
School of Software, Tsinghua University

乔嘉林
清华大学 软件学院

> -----原始邮件-----
> 发件人: "Xiangdong Huang" <sa...@gmail.com>
> 发送时间: 2019-06-28 20:45:44 (星期五)
> 收件人: dev@iotdb.apache.org
> 抄送: 
> 主题: Re: Re: Avoid long-tail insertion
> 
> I see. The following sentence is very important to guarantee the
> correctness...
> 
> >  At the same time, resubmitting UFP after the end of the task in each
> Flush thread ensures that all tasks can be executed.
> 
> I think it should be highlight listed, rather than be drown in the last
> paragraph.
> 
> Anyway, now I understand what you want to do.
> 
> Best,
> -----------------------------------
> Xiangdong Huang
> School of Software, Tsinghua University
> 
>  黄向东
> 清华大学 软件学院
> 
> 
> Xiangdong Huang <sa...@gmail.com> 于2019年6月28日周五 下午8:41写道:
> 
> > Hi Tianan,
> >
> > > the flush tasks in each UFP(There is a flushing queue in UFP) need to be
> > executed sequentially.
> > > Flush thread polls the first UFP from UFP queue in FlushManager -> polls
> > the first flush task in UFP -> completes the flush task -> set
> > ‘managedByFlushManager’  of the UFP to false.
> >
> > It indicates that there are more than one flush tasks in a UFP, but the
> > FlushManager  just spends one task from the UFP and then it mark the UFP as
> > managedByFlushManager=false and poll it out of the queue? (So, when to
> > flush the rest tasks??)
> >
> > > (1) UFP is not managed by Flush Manager, i.e.'managedByFlushManager' is
> > false
> >
> > If suppose the UFP has one flush task and enqueued the queue. Then the UFP
> > has the second flush task. However, it can not be added into the queue
> > because managedByFlushManager == true. Using your above logic, you will
> > ignore the second flush task....
> >
> >
> > > Flush Manager first determines whether UFP meets the criteria for
> > submission
> >
> > Do you want to say both the two criteria should be satisfied? (If so, the
> > above  hypothetical situation will occur).
> >
> > > Through the above design, we can ensure that at the same time for each
> > UFP, Flush Manager will only manage at most once and execute at most one
> > flush task, while there is no restriction between different UFPs.
> >
> > Using your design, we can ensure that at a certain time, given a UFP, the
> > Flush Manager will only manage a UFP at most once and execute at most one
> > flush task from the UFP, but how to avoid the above hypothetical situation?
> >
> > Best,
> > -----------------------------------
> > Xiangdong Huang
> > School of Software, Tsinghua University
> >
> >  黄向东
> > 清华大学 软件学院
> >
> >
> > 李天安 <lt...@mails.tsinghua.edu.cn> 于2019年6月28日周五 上午11:29写道:
> >
> >> Hi,
> >>  I'm also involved in the design of the new storage engine. Let me
> >> complement the new design of the flush task.
> >>
> >>  To improve system performance, we changed flush tasks from synchronous
> >> to asynchronous. We introduced a Flush Manager to manage all flush tasks.
> >> The tricky problem is that each Unsealed TsFile Processor (UFP) corresponds
> >> to a data file on a disk, so the flush tasks in each UFP(There is a
> >> flushing queue in UFP) need to be executed sequentially. However, flush
> >> tasks in different UFPs have no sequential requirements. How to design them
> >> to meet the above requirements?
> >>
> >>  We introduce a UFP FIFO queue in Flush Manager, and add a boolean
> >> attribute ‘managedByFlushManager’ to each UFP to indicate whether it is
> >> managed by Flush Manager. Flush Manager maintains a Flush thread pool to
> >> perform Flush tasks, so the lifecycle of a Flush task is
> >> 1. UFP are submitted to FlushManager,FlushManager add UFP to its queue
> >> and set ‘managedByFlushManager’  of the UFP to true.
> >> 2. The Flush Pool in FlushManager start a flush thread to execute task.
> >> 3. Flush thread polls the first UFP from UFP queue in FlushManager ->
> >> polls the first flush task in UFP -> completes the flush task -> set
> >> ‘managedByFlushManager’  of the UFP to false.
> >>
> >> There are two ways to submit a UFP to FlushManager:
> >> 1. UFP, whenever a MemTable reaches a certain size or forcibly triggers a
> >> flush task, it submits itself to Flush Manager (because the queue in Flush
> >> Manager is UFP). Flush Manager first determines whether UFP meets the
> >> criteria for submission:
> >> (1) UFP is not managed by Flush Manager, i.e.'managedByFlushManager' is
> >> false
> >> (2) The Flush task queue in UFP is not empty, that is, there are at least
> >> one flush task to be executed.
> >>
> >> 2. When the Flush thread completes the flush task, it sets
> >> ‘managedByFlushManager’ to false and resubmits the UFP of the completed
> >> flush task to the FlushManager.
> >>
> >> Through the above design, we can ensure that at the same time for each
> >> UFP, Flush Manager will only manage at most once and execute at most one
> >> flush task, while there is no restriction between different UFPs. At the
> >> same time, resubmitting UFP after the end of the task in each Flush thread
> >> ensures that all tasks can be executed. Therefore, we solve the above
> >> problem and the design meets the requirements of Flush Manager.
> >>
> >> Best Regards,
> >> -------------------------------------
> >> Tianan Li
> >> School of Software, Tsinghua University
> >>
> >> > -----原始邮件-----
> >> > 发件人: "Jialin Qiao" <qj...@mails.tsinghua.edu.cn>
> >> > 发送时间: 2019-06-27 11:27:24 (星期四)
> >> > 收件人: dev@iotdb.apache.org
> >> > 抄送:
> >> > 主题: Re: Avoid long-tail insertion
> >> >
> >> > Hi,
> >> >
> >> > The new storage engine is designed to have the following components:
> >> >
> >> > (1) MemTable: A memory structure, which stores all inserted data in
> >> memory.
> >> >
> >> > (2) MemtablePool: Manages all memtables. All memtables are gotten from
> >> this pool. The total number of memtables is fixed
> >> > in the system. Once the pool do not has available memtables, the
> >> getMemtable() operation will wait or directly return.
> >> >
> >> > (3) UnsealedTsFileProcessor (UFP): A writer for one data file. It
> >> always has one working memtable that receives writes and a
> >> > list (flushing list) of memtables that for flush. Once the working
> >> memtable reaches a threshold, it will be moved to the
> >> > flushing list and the working memtable is set null. When a new write
> >> arrives, if the working memtable is null, UFP will
> >> > call getMemtable() of the MemtablePool to get one as the working
> >> memtable.
> >> >
> >> > (4) StorageGroupProcessor (SGP): Each SGP is responsible for all writes
> >> and reads in one storage group. It always has one
> >> > working UFP that receives write and a list (closing list) of UFPs that
> >> for close. Once the file size of the working UFP reaches
> >> > a threshold, the UFP is moved to the closing list and the working UFP
> >> is set null. When a new write arrives, if the working UFP
> >> > is null, a new UFP is generated as working UFP and receives write.
> >> >
> >> > (5) StorageGroupManager (SGM): A manager of all SGPs in IoTDB. It is
> >> only responsible for routing read and write operations
> >> > to its corresponding SGP according to the deviceId of the operation.
> >> >
> >> > (6) Flush thread: The flush thread poll a memtable from the flushing
> >> list in UFP and flush a memtable to disk. After flushing,
> >> > the memtable is returned to the MemtablePool.
> >> >
> >> > These are only the main components of the new storage engine. Some
> >> things may be lost. It would be great if someone could
> >> > give some advices or supplementations.
> >> >
> >> > Best,
> >> > --
> >> > Jialin Qiao
> >> > School of Software, Tsinghua University
> >> >
> >> > 乔嘉林
> >> > 清华大学 软件学院
> >> >
> >> > > -----原始邮件-----
> >> > > 发件人: "Jialin Qiao" <qj...@mails.tsinghua.edu.cn>
> >> > > 发送时间: 2019-06-24 20:24:05 (星期一)
> >> > > 收件人: dev@iotdb.apache.org
> >> > > 抄送:
> >> > > 主题: Re: Re: Re: Avoid long-tail insertion
> >> > >
> >> > >
> >> > > Yes, there are many changes. The branch I am working on is
> >> feature_async_close_tsfile.
> >> > > Anyone interested is welcome to join and discuss.
> >> > >
> >> > > Best,
> >> > > --
> >> > > Jialin Qiao
> >> > > School of Software, Tsinghua University
> >> > >
> >> > > 乔嘉林
> >> > > 清华大学 软件学院
> >> > >
> >> > > > -----原始邮件-----
> >> > > > 发件人: "Xiangdong Huang" <sa...@gmail.com>
> >> > > > 发送时间: 2019-06-23 10:59:29 (星期日)
> >> > > > 收件人: dev@iotdb.apache.org
> >> > > > 抄送:
> >> > > > 主题: Re: Re: Avoid long-tail insertion
> >> > > >
> >> > > > Hi,
> >> > > >
> >> > > > Once your work branch is almost ready, let me know so I can help to
> >> review.
> >> > > > I think it is a HUGE PR...
> >> > > >
> >> > > > -----------------------------------
> >> > > > Xiangdong Huang
> >> > > > School of Software, Tsinghua University
> >> > > >
> >> > > >  黄向东
> >> > > > 清华大学 软件学院
> >> > > >
> >> > > >
> >> > > > Jialin Qiao <qj...@mails.tsinghua.edu.cn> 于2019年6月22日周六 下午9:57写道:
> >> > > >
> >> > > > > Hi Xiangdong,
> >> > > > >
> >> > > > > I will merge this patch. Let "Directories" manage the folders of
> >> both
> >> > > > > sequence and unSequence files is good.
> >> > > > >
> >> > > > > However, the naming of "Directories" is not clear. It would be
> >> better to
> >> > > > > rename to "DirectoryManager"
> >> > > > >
> >> > > > > Best,
> >> > > > > --
> >> > > > > Jialin Qiao
> >> > > > > School of Software, Tsinghua University
> >> > > > >
> >> > > > > 乔嘉林
> >> > > > > 清华大学 软件学院
> >> > > > >
> >> > > > > > -----原始邮件-----
> >> > > > > > 发件人: "Xiangdong Huang" <sa...@gmail.com>
> >> > > > > > 发送时间: 2019-06-22 16:35:29 (星期六)
> >> > > > > > 收件人: dev@iotdb.apache.org
> >> > > > > > 抄送:
> >> > > > > > 主题: Re: Avoid long-tail insertion
> >> > > > > >
> >> > > > > > Hi jialin,
> >> > > > > >
> >> > > > > > I submit some modifications for:
> >> > > > > >
> >> > > > > > * add the overflow data folder location setting in the
> >> > > > > > iotdb-engine.properties;
> >> > > > > > * let Directories.java to manage the above folder.
> >> > > > > >
> >> > > > > > If you need to refactor the overflow when you solving the long
> >> tail
> >> > > > > issue,
> >> > > > > > you can apply the patch from [1] first to simplify your work.
> >> > > > > >
> >> > > > > > [1]
> >> > > > > >
> >> > > > >
> >> https://issues.apache.org/jira/secure/attachment/12972547/overflow-folder.patch
> >> > > > > >
> >> > > > > > Best,
> >> > > > > > -----------------------------------
> >> > > > > > Xiangdong Huang
> >> > > > > > School of Software, Tsinghua University
> >> > > > > >
> >> > > > > >  黄向东
> >> > > > > > 清华大学 软件学院
> >> > > > > >
> >> > > > > >
> >> > > > > > Xiangdong Huang <sa...@gmail.com> 于2019年6月22日周六 下午3:19写道:
> >> > > > > >
> >> > > > > > > If you change the process like this, i.e., there are more
> >> than one
> >> > > > > > > unsealed TsFiles for each storage group, then  you have to
> >> modify the
> >> > > > > WAL
> >> > > > > > > module.. Because current WAL module only recognizes the last
> >> unsealed
> >> > > > > > > TsFile..
> >> > > > > > >
> >> > > > > > > By the way, "sealed" is better than "closed", I think..  A
> >> sealed file
> >> > > > > > > means the file which has the magic string at the head and the
> >> tail.
> >> > > > > > >
> >> > > > > > > Best,
> >> > > > > > > -----------------------------------
> >> > > > > > > Xiangdong Huang
> >> > > > > > > School of Software, Tsinghua University
> >> > > > > > >
> >> > > > > > >  黄向东
> >> > > > > > > 清华大学 软件学院
> >> > > > > > >
> >> > > > > > >
> >> > > > > > > Jialin Qiao <qj...@mails.tsinghua.edu.cn> 于2019年6月22日周六
> >> 下午2:54写道:
> >> > > > > > >
> >> > > > > > >>
> >> > > > > > >> Hi, I am solving the long-tail latency problem.
> >> > > > > > >>
> >> > > > > > >> There are some cases (blocking points) that blocking the
> >> insertion.
> >> > > > > For a
> >> > > > > > >> better understanding of this problem, I first introduce the
> >> writing
> >> > > > > process
> >> > > > > > >> of IoTDB:
> >> > > > > > >>
> >> > > > > > >> IoTDB maintains several independent engines (storage group)
> >> that
> >> > > > > supports
> >> > > > > > >> read and write. In the following, we focus on one engine. A
> >> engine
> >> > > > > > >> maintains several closed data files and one unclosed data
> >> file that
> >> > > > > > >> receives appended data. In memory, there is only one working
> >> memtable
> >> > > > > (m1)
> >> > > > > > >> that receives writes. There is also another memtable (m2)
> >> that will
> >> > > > > take
> >> > > > > > >> place m1 when m1 is full and being flushed.
> >> > > > > > >>
> >> > > > > > >> When a data item is inserted:
> >> > > > > > >>
> >> > > > > > >> (1)We insert it into the working memtable.
> >> > > > > > >> (2)We check the size of the memtable. If it reaches a
> >> threshold, we
> >> > > > > > >> submit a flush task “after the previous flush task is
> >> finished” and
> >> > > > > switch
> >> > > > > > >> the two memtables.
> >> > > > > > >> (3)We check the size of the unclosed file. If it reaches a
> >> threshold,
> >> > > > > we
> >> > > > > > >> close it “after the previous flush task is finished”.
> >> > > > > > >>
> >> > > > > > >> In the above steps, all the "after the previous flush task is
> >> > > > > finished"
> >> > > > > > >> will block the insertion process. One solution is to make
> >> all flush
> >> > > > > and
> >> > > > > > >> close task asynchronous. Some questions need to carefully
> >> considered:
> >> > > > > > >>
> >> > > > > > >> (1) Many memtables may be flushed concurrently to an
> >> unclosed file.
> >> > > > > How
> >> > > > > > >> to guarantee the order of serialization?
> >> > > > > > >> (2) Once a close task is submitted, a new unclosed file will
> >> be
> >> > > > > created
> >> > > > > > >> and receives appended data. So there will exists many
> >> unclosed files.
> >> > > > > How
> >> > > > > > >> the query and compaction process will be impacted?
> >> > > > > > >>
> >> > > > > > >> Thanks,
> >> > > > > > >>
> >> > > > > > >> Jialin Qiao
> >> > > > > > >> School of Software, Tsinghua University
> >> > > > > > >>
> >> > > > > > >> 乔嘉林
> >> > > > > > >> 清华大学 软件学院
> >> > > > > > >>
> >> > > > > > >> > -----原始邮件-----
> >> > > > > > >> > 发件人: "Xiangdong Huang" <sa...@gmail.com>
> >> > > > > > >> > 发送时间: 2019-06-04 23:08:34 (星期二)
> >> > > > > > >> > 收件人: dev@iotdb.apache.org, "江天" <jt...@163.com>
> >> > > > > > >> > 抄送:
> >> > > > > > >> > 主题: Re: [jira] [Created] (IOTDB-112) Avoid long tail
> >> insertion
> >> > > > > which is
> >> > > > > > >> caused by synchronized close-bufferwrite
> >> > > > > > >> >
> >> > > > > > >> > I attached the histogram of the latency in the JIRA.
> >> > > > > > >> >
> >> > > > > > >> > The x-axis is the latency while the y-axis is the
> >> cumulative
> >> > > > > > >> distribution.
> >> > > > > > >> > We can see that about 30% insertion can be finished in
> >> 20ms, and 60%
> >> > > > > > >> > insertion can be finished in 40ms even though the IoTDB
> >> instance is
> >> > > > > > >> serving
> >> > > > > > >> > for a heavy workload... So, eliminating the long tail
> >> insertion can
> >> > > > > make
> >> > > > > > >> > the average latency far better.
> >> > > > > > >> >
> >> > > > > > >> > If someone is working on the refactor_overflow or
> >> > > > > refactor_bufferwrite,
> >> > > > > > >> > please pay attention to the code branch for this issue.
> >> > > > > > >> >
> >> > > > > > >> > Best,
> >> > > > > > >> >
> >> > > > > > >> > -----------------------------------
> >> > > > > > >> > Xiangdong Huang
> >> > > > > > >> > School of Software, Tsinghua University
> >> > > > > > >> >
> >> > > > > > >> >  黄向东
> >> > > > > > >> > 清华大学 软件学院
> >> > > > > > >> >
> >> > > > > > >> >
> >> > > > > > >> > xiangdong Huang (JIRA) <ji...@apache.org> 于2019年6月4日周二
> >> 下午11:00写道:
> >> > > > > > >> >
> >> > > > > > >> > > xiangdong Huang created IOTDB-112:
> >> > > > > > >> > > -------------------------------------
> >> > > > > > >> > >
> >> > > > > > >> > >              Summary: Avoid long tail insertion which is
> >> caused by
> >> > > > > > >> > > synchronized close-bufferwrite
> >> > > > > > >> > >                  Key: IOTDB-112
> >> > > > > > >> > >                  URL:
> >> > > > > https://issues.apache.org/jira/browse/IOTDB-112
> >> > > > > > >> > >              Project: Apache IoTDB
> >> > > > > > >> > >           Issue Type: Improvement
> >> > > > > > >> > >             Reporter: xiangdong Huang
> >> > > > > > >> > >
> >> > > > > > >> > >
> >> > > > > > >> > > In our test, IoTDB has a good insertion performance, and
> >> the
> >> > > > > average
> >> > > > > > >> > > latency can be ~200 ms in a given workload and hardware.
> >> > > > > > >> > >
> >> > > > > > >> > > However, when we draw the histogram of the latency, we
> >> find that
> >> > > > > 97.5%
> >> > > > > > >> > > latencies are less than 200 ms, while 2.7% latencies are
> >> greater.
> >> > > > > The
> >> > > > > > >> > > result shows that there are some long tail latency.
> >> > > > > > >> > >
> >> > > > > > >> > > Then we find that some insertion latencies are about 30
> >> seconds...
> >> > > > > > >> (but
> >> > > > > > >> > > the ratio is less than 0.5%). Indeed, for each
> >> connection, a long
> >> > > > > tail
> >> > > > > > >> > > insertion appears per 1 or 2 minutes....
> >> > > > > > >> > >
> >> > > > > > >> > > By reading source codes, I think it is because that in
> >> the
> >> > > > > insertion
> >> > > > > > >> > > function,
> >> > > > > > >> > >
> >> > > > > > >> > > `private void insertBufferWrite(FileNodeProcessor
> >> > > > > fileNodeProcessor,
> >> > > > > > >> long
> >> > > > > > >> > > timestamp,
> >> > > > > > >> > >  boolean isMonitor, TSRecord tsRecord, String deviceId)`,
> >> > > > > > >> > >
> >> > > > > > >> > > if the corresponding TsFile is too large, the function
> >> is blocked
> >> > > > > > >> until
> >> > > > > > >> > > the memtable is flushed on disk and the TsFile is sealed
> >> (we call
> >> > > > > it
> >> > > > > > >> as
> >> > > > > > >> > > closing a TsFile). The latencies of the long tail
> >> insertions are
> >> > > > > very
> >> > > > > > >> close
> >> > > > > > >> > > to the time cost of flushing and sealing a TsFile.
> >> > > > > > >> > >
> >> > > > > > >> > > So, if we set the closing function using the async mode,
> >> we can
> >> > > > > avoid
> >> > > > > > >> the
> >> > > > > > >> > > long tail insertion.
> >> > > > > > >> > >
> >> > > > > > >> > > However,  there are some side effects we have to fix:
> >> > > > > > >> > >  # At the same time, if a new insertion comes, then a
> >> new memtable
> >> > > > > > >> should
> >> > > > > > >> > > be assigned, and a new unsealed TsFile is created;
> >> > > > > > >> > >  # That means that there are more than 1 unsealed
> >> TsFiles if the
> >> > > > > > >> system is
> >> > > > > > >> > > crashed before the closing function is finished. So, we
> >> have to
> >> > > > > > >> modify the
> >> > > > > > >> > > startup process to recover these files.
> >> > > > > > >> > >
> >> > > > > > >> > > Is there any other side effect that I have to pay
> >> attention to?
> >> > > > > > >> > >
> >> > > > > > >> > >
> >> > > > > > >> > >
> >> > > > > > >> > > --
> >> > > > > > >> > > This message was sent by Atlassian JIRA
> >> > > > > > >> > > (v7.6.3#76005)
> >> > > > > > >> > >
> >> > > > > > >>
> >> > > > > > >
> >> > > > >
> >>
> >

Re: Re: Avoid long-tail insertion

Posted by Xiangdong Huang <sa...@gmail.com>.
I see. The following sentence is very important to guarantee the
correctness...

>  At the same time, resubmitting UFP after the end of the task in each
Flush thread ensures that all tasks can be executed.

I think it should be highlight listed, rather than be drown in the last
paragraph.

Anyway, now I understand what you want to do.

Best,
-----------------------------------
Xiangdong Huang
School of Software, Tsinghua University

 黄向东
清华大学 软件学院


Xiangdong Huang <sa...@gmail.com> 于2019年6月28日周五 下午8:41写道:

> Hi Tianan,
>
> > the flush tasks in each UFP(There is a flushing queue in UFP) need to be
> executed sequentially.
> > Flush thread polls the first UFP from UFP queue in FlushManager -> polls
> the first flush task in UFP -> completes the flush task -> set
> ‘managedByFlushManager’  of the UFP to false.
>
> It indicates that there are more than one flush tasks in a UFP, but the
> FlushManager  just spends one task from the UFP and then it mark the UFP as
> managedByFlushManager=false and poll it out of the queue? (So, when to
> flush the rest tasks??)
>
> > (1) UFP is not managed by Flush Manager, i.e.'managedByFlushManager' is
> false
>
> If suppose the UFP has one flush task and enqueued the queue. Then the UFP
> has the second flush task. However, it can not be added into the queue
> because managedByFlushManager == true. Using your above logic, you will
> ignore the second flush task....
>
>
> > Flush Manager first determines whether UFP meets the criteria for
> submission
>
> Do you want to say both the two criteria should be satisfied? (If so, the
> above  hypothetical situation will occur).
>
> > Through the above design, we can ensure that at the same time for each
> UFP, Flush Manager will only manage at most once and execute at most one
> flush task, while there is no restriction between different UFPs.
>
> Using your design, we can ensure that at a certain time, given a UFP, the
> Flush Manager will only manage a UFP at most once and execute at most one
> flush task from the UFP, but how to avoid the above hypothetical situation?
>
> Best,
> -----------------------------------
> Xiangdong Huang
> School of Software, Tsinghua University
>
>  黄向东
> 清华大学 软件学院
>
>
> 李天安 <lt...@mails.tsinghua.edu.cn> 于2019年6月28日周五 上午11:29写道:
>
>> Hi,
>>  I'm also involved in the design of the new storage engine. Let me
>> complement the new design of the flush task.
>>
>>  To improve system performance, we changed flush tasks from synchronous
>> to asynchronous. We introduced a Flush Manager to manage all flush tasks.
>> The tricky problem is that each Unsealed TsFile Processor (UFP) corresponds
>> to a data file on a disk, so the flush tasks in each UFP(There is a
>> flushing queue in UFP) need to be executed sequentially. However, flush
>> tasks in different UFPs have no sequential requirements. How to design them
>> to meet the above requirements?
>>
>>  We introduce a UFP FIFO queue in Flush Manager, and add a boolean
>> attribute ‘managedByFlushManager’ to each UFP to indicate whether it is
>> managed by Flush Manager. Flush Manager maintains a Flush thread pool to
>> perform Flush tasks, so the lifecycle of a Flush task is
>> 1. UFP are submitted to FlushManager,FlushManager add UFP to its queue
>> and set ‘managedByFlushManager’  of the UFP to true.
>> 2. The Flush Pool in FlushManager start a flush thread to execute task.
>> 3. Flush thread polls the first UFP from UFP queue in FlushManager ->
>> polls the first flush task in UFP -> completes the flush task -> set
>> ‘managedByFlushManager’  of the UFP to false.
>>
>> There are two ways to submit a UFP to FlushManager:
>> 1. UFP, whenever a MemTable reaches a certain size or forcibly triggers a
>> flush task, it submits itself to Flush Manager (because the queue in Flush
>> Manager is UFP). Flush Manager first determines whether UFP meets the
>> criteria for submission:
>> (1) UFP is not managed by Flush Manager, i.e.'managedByFlushManager' is
>> false
>> (2) The Flush task queue in UFP is not empty, that is, there are at least
>> one flush task to be executed.
>>
>> 2. When the Flush thread completes the flush task, it sets
>> ‘managedByFlushManager’ to false and resubmits the UFP of the completed
>> flush task to the FlushManager.
>>
>> Through the above design, we can ensure that at the same time for each
>> UFP, Flush Manager will only manage at most once and execute at most one
>> flush task, while there is no restriction between different UFPs. At the
>> same time, resubmitting UFP after the end of the task in each Flush thread
>> ensures that all tasks can be executed. Therefore, we solve the above
>> problem and the design meets the requirements of Flush Manager.
>>
>> Best Regards,
>> -------------------------------------
>> Tianan Li
>> School of Software, Tsinghua University
>>
>> > -----原始邮件-----
>> > 发件人: "Jialin Qiao" <qj...@mails.tsinghua.edu.cn>
>> > 发送时间: 2019-06-27 11:27:24 (星期四)
>> > 收件人: dev@iotdb.apache.org
>> > 抄送:
>> > 主题: Re: Avoid long-tail insertion
>> >
>> > Hi,
>> >
>> > The new storage engine is designed to have the following components:
>> >
>> > (1) MemTable: A memory structure, which stores all inserted data in
>> memory.
>> >
>> > (2) MemtablePool: Manages all memtables. All memtables are gotten from
>> this pool. The total number of memtables is fixed
>> > in the system. Once the pool do not has available memtables, the
>> getMemtable() operation will wait or directly return.
>> >
>> > (3) UnsealedTsFileProcessor (UFP): A writer for one data file. It
>> always has one working memtable that receives writes and a
>> > list (flushing list) of memtables that for flush. Once the working
>> memtable reaches a threshold, it will be moved to the
>> > flushing list and the working memtable is set null. When a new write
>> arrives, if the working memtable is null, UFP will
>> > call getMemtable() of the MemtablePool to get one as the working
>> memtable.
>> >
>> > (4) StorageGroupProcessor (SGP): Each SGP is responsible for all writes
>> and reads in one storage group. It always has one
>> > working UFP that receives write and a list (closing list) of UFPs that
>> for close. Once the file size of the working UFP reaches
>> > a threshold, the UFP is moved to the closing list and the working UFP
>> is set null. When a new write arrives, if the working UFP
>> > is null, a new UFP is generated as working UFP and receives write.
>> >
>> > (5) StorageGroupManager (SGM): A manager of all SGPs in IoTDB. It is
>> only responsible for routing read and write operations
>> > to its corresponding SGP according to the deviceId of the operation.
>> >
>> > (6) Flush thread: The flush thread poll a memtable from the flushing
>> list in UFP and flush a memtable to disk. After flushing,
>> > the memtable is returned to the MemtablePool.
>> >
>> > These are only the main components of the new storage engine. Some
>> things may be lost. It would be great if someone could
>> > give some advices or supplementations.
>> >
>> > Best,
>> > --
>> > Jialin Qiao
>> > School of Software, Tsinghua University
>> >
>> > 乔嘉林
>> > 清华大学 软件学院
>> >
>> > > -----原始邮件-----
>> > > 发件人: "Jialin Qiao" <qj...@mails.tsinghua.edu.cn>
>> > > 发送时间: 2019-06-24 20:24:05 (星期一)
>> > > 收件人: dev@iotdb.apache.org
>> > > 抄送:
>> > > 主题: Re: Re: Re: Avoid long-tail insertion
>> > >
>> > >
>> > > Yes, there are many changes. The branch I am working on is
>> feature_async_close_tsfile.
>> > > Anyone interested is welcome to join and discuss.
>> > >
>> > > Best,
>> > > --
>> > > Jialin Qiao
>> > > School of Software, Tsinghua University
>> > >
>> > > 乔嘉林
>> > > 清华大学 软件学院
>> > >
>> > > > -----原始邮件-----
>> > > > 发件人: "Xiangdong Huang" <sa...@gmail.com>
>> > > > 发送时间: 2019-06-23 10:59:29 (星期日)
>> > > > 收件人: dev@iotdb.apache.org
>> > > > 抄送:
>> > > > 主题: Re: Re: Avoid long-tail insertion
>> > > >
>> > > > Hi,
>> > > >
>> > > > Once your work branch is almost ready, let me know so I can help to
>> review.
>> > > > I think it is a HUGE PR...
>> > > >
>> > > > -----------------------------------
>> > > > Xiangdong Huang
>> > > > School of Software, Tsinghua University
>> > > >
>> > > >  黄向东
>> > > > 清华大学 软件学院
>> > > >
>> > > >
>> > > > Jialin Qiao <qj...@mails.tsinghua.edu.cn> 于2019年6月22日周六 下午9:57写道:
>> > > >
>> > > > > Hi Xiangdong,
>> > > > >
>> > > > > I will merge this patch. Let "Directories" manage the folders of
>> both
>> > > > > sequence and unSequence files is good.
>> > > > >
>> > > > > However, the naming of "Directories" is not clear. It would be
>> better to
>> > > > > rename to "DirectoryManager"
>> > > > >
>> > > > > Best,
>> > > > > --
>> > > > > Jialin Qiao
>> > > > > School of Software, Tsinghua University
>> > > > >
>> > > > > 乔嘉林
>> > > > > 清华大学 软件学院
>> > > > >
>> > > > > > -----原始邮件-----
>> > > > > > 发件人: "Xiangdong Huang" <sa...@gmail.com>
>> > > > > > 发送时间: 2019-06-22 16:35:29 (星期六)
>> > > > > > 收件人: dev@iotdb.apache.org
>> > > > > > 抄送:
>> > > > > > 主题: Re: Avoid long-tail insertion
>> > > > > >
>> > > > > > Hi jialin,
>> > > > > >
>> > > > > > I submit some modifications for:
>> > > > > >
>> > > > > > * add the overflow data folder location setting in the
>> > > > > > iotdb-engine.properties;
>> > > > > > * let Directories.java to manage the above folder.
>> > > > > >
>> > > > > > If you need to refactor the overflow when you solving the long
>> tail
>> > > > > issue,
>> > > > > > you can apply the patch from [1] first to simplify your work.
>> > > > > >
>> > > > > > [1]
>> > > > > >
>> > > > >
>> https://issues.apache.org/jira/secure/attachment/12972547/overflow-folder.patch
>> > > > > >
>> > > > > > Best,
>> > > > > > -----------------------------------
>> > > > > > Xiangdong Huang
>> > > > > > School of Software, Tsinghua University
>> > > > > >
>> > > > > >  黄向东
>> > > > > > 清华大学 软件学院
>> > > > > >
>> > > > > >
>> > > > > > Xiangdong Huang <sa...@gmail.com> 于2019年6月22日周六 下午3:19写道:
>> > > > > >
>> > > > > > > If you change the process like this, i.e., there are more
>> than one
>> > > > > > > unsealed TsFiles for each storage group, then  you have to
>> modify the
>> > > > > WAL
>> > > > > > > module.. Because current WAL module only recognizes the last
>> unsealed
>> > > > > > > TsFile..
>> > > > > > >
>> > > > > > > By the way, "sealed" is better than "closed", I think..  A
>> sealed file
>> > > > > > > means the file which has the magic string at the head and the
>> tail.
>> > > > > > >
>> > > > > > > Best,
>> > > > > > > -----------------------------------
>> > > > > > > Xiangdong Huang
>> > > > > > > School of Software, Tsinghua University
>> > > > > > >
>> > > > > > >  黄向东
>> > > > > > > 清华大学 软件学院
>> > > > > > >
>> > > > > > >
>> > > > > > > Jialin Qiao <qj...@mails.tsinghua.edu.cn> 于2019年6月22日周六
>> 下午2:54写道:
>> > > > > > >
>> > > > > > >>
>> > > > > > >> Hi, I am solving the long-tail latency problem.
>> > > > > > >>
>> > > > > > >> There are some cases (blocking points) that blocking the
>> insertion.
>> > > > > For a
>> > > > > > >> better understanding of this problem, I first introduce the
>> writing
>> > > > > process
>> > > > > > >> of IoTDB:
>> > > > > > >>
>> > > > > > >> IoTDB maintains several independent engines (storage group)
>> that
>> > > > > supports
>> > > > > > >> read and write. In the following, we focus on one engine. A
>> engine
>> > > > > > >> maintains several closed data files and one unclosed data
>> file that
>> > > > > > >> receives appended data. In memory, there is only one working
>> memtable
>> > > > > (m1)
>> > > > > > >> that receives writes. There is also another memtable (m2)
>> that will
>> > > > > take
>> > > > > > >> place m1 when m1 is full and being flushed.
>> > > > > > >>
>> > > > > > >> When a data item is inserted:
>> > > > > > >>
>> > > > > > >> (1)We insert it into the working memtable.
>> > > > > > >> (2)We check the size of the memtable. If it reaches a
>> threshold, we
>> > > > > > >> submit a flush task “after the previous flush task is
>> finished” and
>> > > > > switch
>> > > > > > >> the two memtables.
>> > > > > > >> (3)We check the size of the unclosed file. If it reaches a
>> threshold,
>> > > > > we
>> > > > > > >> close it “after the previous flush task is finished”.
>> > > > > > >>
>> > > > > > >> In the above steps, all the "after the previous flush task is
>> > > > > finished"
>> > > > > > >> will block the insertion process. One solution is to make
>> all flush
>> > > > > and
>> > > > > > >> close task asynchronous. Some questions need to carefully
>> considered:
>> > > > > > >>
>> > > > > > >> (1) Many memtables may be flushed concurrently to an
>> unclosed file.
>> > > > > How
>> > > > > > >> to guarantee the order of serialization?
>> > > > > > >> (2) Once a close task is submitted, a new unclosed file will
>> be
>> > > > > created
>> > > > > > >> and receives appended data. So there will exists many
>> unclosed files.
>> > > > > How
>> > > > > > >> the query and compaction process will be impacted?
>> > > > > > >>
>> > > > > > >> Thanks,
>> > > > > > >>
>> > > > > > >> Jialin Qiao
>> > > > > > >> School of Software, Tsinghua University
>> > > > > > >>
>> > > > > > >> 乔嘉林
>> > > > > > >> 清华大学 软件学院
>> > > > > > >>
>> > > > > > >> > -----原始邮件-----
>> > > > > > >> > 发件人: "Xiangdong Huang" <sa...@gmail.com>
>> > > > > > >> > 发送时间: 2019-06-04 23:08:34 (星期二)
>> > > > > > >> > 收件人: dev@iotdb.apache.org, "江天" <jt...@163.com>
>> > > > > > >> > 抄送:
>> > > > > > >> > 主题: Re: [jira] [Created] (IOTDB-112) Avoid long tail
>> insertion
>> > > > > which is
>> > > > > > >> caused by synchronized close-bufferwrite
>> > > > > > >> >
>> > > > > > >> > I attached the histogram of the latency in the JIRA.
>> > > > > > >> >
>> > > > > > >> > The x-axis is the latency while the y-axis is the
>> cumulative
>> > > > > > >> distribution.
>> > > > > > >> > We can see that about 30% insertion can be finished in
>> 20ms, and 60%
>> > > > > > >> > insertion can be finished in 40ms even though the IoTDB
>> instance is
>> > > > > > >> serving
>> > > > > > >> > for a heavy workload... So, eliminating the long tail
>> insertion can
>> > > > > make
>> > > > > > >> > the average latency far better.
>> > > > > > >> >
>> > > > > > >> > If someone is working on the refactor_overflow or
>> > > > > refactor_bufferwrite,
>> > > > > > >> > please pay attention to the code branch for this issue.
>> > > > > > >> >
>> > > > > > >> > Best,
>> > > > > > >> >
>> > > > > > >> > -----------------------------------
>> > > > > > >> > Xiangdong Huang
>> > > > > > >> > School of Software, Tsinghua University
>> > > > > > >> >
>> > > > > > >> >  黄向东
>> > > > > > >> > 清华大学 软件学院
>> > > > > > >> >
>> > > > > > >> >
>> > > > > > >> > xiangdong Huang (JIRA) <ji...@apache.org> 于2019年6月4日周二
>> 下午11:00写道:
>> > > > > > >> >
>> > > > > > >> > > xiangdong Huang created IOTDB-112:
>> > > > > > >> > > -------------------------------------
>> > > > > > >> > >
>> > > > > > >> > >              Summary: Avoid long tail insertion which is
>> caused by
>> > > > > > >> > > synchronized close-bufferwrite
>> > > > > > >> > >                  Key: IOTDB-112
>> > > > > > >> > >                  URL:
>> > > > > https://issues.apache.org/jira/browse/IOTDB-112
>> > > > > > >> > >              Project: Apache IoTDB
>> > > > > > >> > >           Issue Type: Improvement
>> > > > > > >> > >             Reporter: xiangdong Huang
>> > > > > > >> > >
>> > > > > > >> > >
>> > > > > > >> > > In our test, IoTDB has a good insertion performance, and
>> the
>> > > > > average
>> > > > > > >> > > latency can be ~200 ms in a given workload and hardware.
>> > > > > > >> > >
>> > > > > > >> > > However, when we draw the histogram of the latency, we
>> find that
>> > > > > 97.5%
>> > > > > > >> > > latencies are less than 200 ms, while 2.7% latencies are
>> greater.
>> > > > > The
>> > > > > > >> > > result shows that there are some long tail latency.
>> > > > > > >> > >
>> > > > > > >> > > Then we find that some insertion latencies are about 30
>> seconds...
>> > > > > > >> (but
>> > > > > > >> > > the ratio is less than 0.5%). Indeed, for each
>> connection, a long
>> > > > > tail
>> > > > > > >> > > insertion appears per 1 or 2 minutes....
>> > > > > > >> > >
>> > > > > > >> > > By reading source codes, I think it is because that in
>> the
>> > > > > insertion
>> > > > > > >> > > function,
>> > > > > > >> > >
>> > > > > > >> > > `private void insertBufferWrite(FileNodeProcessor
>> > > > > fileNodeProcessor,
>> > > > > > >> long
>> > > > > > >> > > timestamp,
>> > > > > > >> > >  boolean isMonitor, TSRecord tsRecord, String deviceId)`,
>> > > > > > >> > >
>> > > > > > >> > > if the corresponding TsFile is too large, the function
>> is blocked
>> > > > > > >> until
>> > > > > > >> > > the memtable is flushed on disk and the TsFile is sealed
>> (we call
>> > > > > it
>> > > > > > >> as
>> > > > > > >> > > closing a TsFile). The latencies of the long tail
>> insertions are
>> > > > > very
>> > > > > > >> close
>> > > > > > >> > > to the time cost of flushing and sealing a TsFile.
>> > > > > > >> > >
>> > > > > > >> > > So, if we set the closing function using the async mode,
>> we can
>> > > > > avoid
>> > > > > > >> the
>> > > > > > >> > > long tail insertion.
>> > > > > > >> > >
>> > > > > > >> > > However,  there are some side effects we have to fix:
>> > > > > > >> > >  # At the same time, if a new insertion comes, then a
>> new memtable
>> > > > > > >> should
>> > > > > > >> > > be assigned, and a new unsealed TsFile is created;
>> > > > > > >> > >  # That means that there are more than 1 unsealed
>> TsFiles if the
>> > > > > > >> system is
>> > > > > > >> > > crashed before the closing function is finished. So, we
>> have to
>> > > > > > >> modify the
>> > > > > > >> > > startup process to recover these files.
>> > > > > > >> > >
>> > > > > > >> > > Is there any other side effect that I have to pay
>> attention to?
>> > > > > > >> > >
>> > > > > > >> > >
>> > > > > > >> > >
>> > > > > > >> > > --
>> > > > > > >> > > This message was sent by Atlassian JIRA
>> > > > > > >> > > (v7.6.3#76005)
>> > > > > > >> > >
>> > > > > > >>
>> > > > > > >
>> > > > >
>>
>

Re: Re: Avoid long-tail insertion

Posted by Xiangdong Huang <sa...@gmail.com>.
Hi Tianan,

> the flush tasks in each UFP(There is a flushing queue in UFP) need to be
executed sequentially.
> Flush thread polls the first UFP from UFP queue in FlushManager -> polls
the first flush task in UFP -> completes the flush task -> set
‘managedByFlushManager’  of the UFP to false.

It indicates that there are more than one flush tasks in a UFP, but the
FlushManager  just spends one task from the UFP and then it mark the UFP as
managedByFlushManager=false and poll it out of the queue? (So, when to
flush the rest tasks??)

> (1) UFP is not managed by Flush Manager, i.e.'managedByFlushManager' is
false

If suppose the UFP has one flush task and enqueued the queue. Then the UFP
has the second flush task. However, it can not be added into the queue
because managedByFlushManager == true. Using your above logic, you will
ignore the second flush task....


> Flush Manager first determines whether UFP meets the criteria for
submission

Do you want to say both the two criteria should be satisfied? (If so, the
above  hypothetical situation will occur).

> Through the above design, we can ensure that at the same time for each
UFP, Flush Manager will only manage at most once and execute at most one
flush task, while there is no restriction between different UFPs.

Using your design, we can ensure that at a certain time, given a UFP, the
Flush Manager will only manage a UFP at most once and execute at most one
flush task from the UFP, but how to avoid the above hypothetical situation?

Best,
-----------------------------------
Xiangdong Huang
School of Software, Tsinghua University

 黄向东
清华大学 软件学院


李天安 <lt...@mails.tsinghua.edu.cn> 于2019年6月28日周五 上午11:29写道:

> Hi,
>  I'm also involved in the design of the new storage engine. Let me
> complement the new design of the flush task.
>
>  To improve system performance, we changed flush tasks from synchronous to
> asynchronous. We introduced a Flush Manager to manage all flush tasks. The
> tricky problem is that each Unsealed TsFile Processor (UFP) corresponds to
> a data file on a disk, so the flush tasks in each UFP(There is a flushing
> queue in UFP) need to be executed sequentially. However, flush tasks in
> different UFPs have no sequential requirements. How to design them to meet
> the above requirements?
>
>  We introduce a UFP FIFO queue in Flush Manager, and add a boolean
> attribute ‘managedByFlushManager’ to each UFP to indicate whether it is
> managed by Flush Manager. Flush Manager maintains a Flush thread pool to
> perform Flush tasks, so the lifecycle of a Flush task is
> 1. UFP are submitted to FlushManager,FlushManager add UFP to its queue and
> set ‘managedByFlushManager’  of the UFP to true.
> 2. The Flush Pool in FlushManager start a flush thread to execute task.
> 3. Flush thread polls the first UFP from UFP queue in FlushManager ->
> polls the first flush task in UFP -> completes the flush task -> set
> ‘managedByFlushManager’  of the UFP to false.
>
> There are two ways to submit a UFP to FlushManager:
> 1. UFP, whenever a MemTable reaches a certain size or forcibly triggers a
> flush task, it submits itself to Flush Manager (because the queue in Flush
> Manager is UFP). Flush Manager first determines whether UFP meets the
> criteria for submission:
> (1) UFP is not managed by Flush Manager, i.e.'managedByFlushManager' is
> false
> (2) The Flush task queue in UFP is not empty, that is, there are at least
> one flush task to be executed.
>
> 2. When the Flush thread completes the flush task, it sets
> ‘managedByFlushManager’ to false and resubmits the UFP of the completed
> flush task to the FlushManager.
>
> Through the above design, we can ensure that at the same time for each
> UFP, Flush Manager will only manage at most once and execute at most one
> flush task, while there is no restriction between different UFPs. At the
> same time, resubmitting UFP after the end of the task in each Flush thread
> ensures that all tasks can be executed. Therefore, we solve the above
> problem and the design meets the requirements of Flush Manager.
>
> Best Regards,
> -------------------------------------
> Tianan Li
> School of Software, Tsinghua University
>
> > -----原始邮件-----
> > 发件人: "Jialin Qiao" <qj...@mails.tsinghua.edu.cn>
> > 发送时间: 2019-06-27 11:27:24 (星期四)
> > 收件人: dev@iotdb.apache.org
> > 抄送:
> > 主题: Re: Avoid long-tail insertion
> >
> > Hi,
> >
> > The new storage engine is designed to have the following components:
> >
> > (1) MemTable: A memory structure, which stores all inserted data in
> memory.
> >
> > (2) MemtablePool: Manages all memtables. All memtables are gotten from
> this pool. The total number of memtables is fixed
> > in the system. Once the pool do not has available memtables, the
> getMemtable() operation will wait or directly return.
> >
> > (3) UnsealedTsFileProcessor (UFP): A writer for one data file. It always
> has one working memtable that receives writes and a
> > list (flushing list) of memtables that for flush. Once the working
> memtable reaches a threshold, it will be moved to the
> > flushing list and the working memtable is set null. When a new write
> arrives, if the working memtable is null, UFP will
> > call getMemtable() of the MemtablePool to get one as the working
> memtable.
> >
> > (4) StorageGroupProcessor (SGP): Each SGP is responsible for all writes
> and reads in one storage group. It always has one
> > working UFP that receives write and a list (closing list) of UFPs that
> for close. Once the file size of the working UFP reaches
> > a threshold, the UFP is moved to the closing list and the working UFP is
> set null. When a new write arrives, if the working UFP
> > is null, a new UFP is generated as working UFP and receives write.
> >
> > (5) StorageGroupManager (SGM): A manager of all SGPs in IoTDB. It is
> only responsible for routing read and write operations
> > to its corresponding SGP according to the deviceId of the operation.
> >
> > (6) Flush thread: The flush thread poll a memtable from the flushing
> list in UFP and flush a memtable to disk. After flushing,
> > the memtable is returned to the MemtablePool.
> >
> > These are only the main components of the new storage engine. Some
> things may be lost. It would be great if someone could
> > give some advices or supplementations.
> >
> > Best,
> > --
> > Jialin Qiao
> > School of Software, Tsinghua University
> >
> > 乔嘉林
> > 清华大学 软件学院
> >
> > > -----原始邮件-----
> > > 发件人: "Jialin Qiao" <qj...@mails.tsinghua.edu.cn>
> > > 发送时间: 2019-06-24 20:24:05 (星期一)
> > > 收件人: dev@iotdb.apache.org
> > > 抄送:
> > > 主题: Re: Re: Re: Avoid long-tail insertion
> > >
> > >
> > > Yes, there are many changes. The branch I am working on is
> feature_async_close_tsfile.
> > > Anyone interested is welcome to join and discuss.
> > >
> > > Best,
> > > --
> > > Jialin Qiao
> > > School of Software, Tsinghua University
> > >
> > > 乔嘉林
> > > 清华大学 软件学院
> > >
> > > > -----原始邮件-----
> > > > 发件人: "Xiangdong Huang" <sa...@gmail.com>
> > > > 发送时间: 2019-06-23 10:59:29 (星期日)
> > > > 收件人: dev@iotdb.apache.org
> > > > 抄送:
> > > > 主题: Re: Re: Avoid long-tail insertion
> > > >
> > > > Hi,
> > > >
> > > > Once your work branch is almost ready, let me know so I can help to
> review.
> > > > I think it is a HUGE PR...
> > > >
> > > > -----------------------------------
> > > > Xiangdong Huang
> > > > School of Software, Tsinghua University
> > > >
> > > >  黄向东
> > > > 清华大学 软件学院
> > > >
> > > >
> > > > Jialin Qiao <qj...@mails.tsinghua.edu.cn> 于2019年6月22日周六 下午9:57写道:
> > > >
> > > > > Hi Xiangdong,
> > > > >
> > > > > I will merge this patch. Let "Directories" manage the folders of
> both
> > > > > sequence and unSequence files is good.
> > > > >
> > > > > However, the naming of "Directories" is not clear. It would be
> better to
> > > > > rename to "DirectoryManager"
> > > > >
> > > > > Best,
> > > > > --
> > > > > Jialin Qiao
> > > > > School of Software, Tsinghua University
> > > > >
> > > > > 乔嘉林
> > > > > 清华大学 软件学院
> > > > >
> > > > > > -----原始邮件-----
> > > > > > 发件人: "Xiangdong Huang" <sa...@gmail.com>
> > > > > > 发送时间: 2019-06-22 16:35:29 (星期六)
> > > > > > 收件人: dev@iotdb.apache.org
> > > > > > 抄送:
> > > > > > 主题: Re: Avoid long-tail insertion
> > > > > >
> > > > > > Hi jialin,
> > > > > >
> > > > > > I submit some modifications for:
> > > > > >
> > > > > > * add the overflow data folder location setting in the
> > > > > > iotdb-engine.properties;
> > > > > > * let Directories.java to manage the above folder.
> > > > > >
> > > > > > If you need to refactor the overflow when you solving the long
> tail
> > > > > issue,
> > > > > > you can apply the patch from [1] first to simplify your work.
> > > > > >
> > > > > > [1]
> > > > > >
> > > > >
> https://issues.apache.org/jira/secure/attachment/12972547/overflow-folder.patch
> > > > > >
> > > > > > Best,
> > > > > > -----------------------------------
> > > > > > Xiangdong Huang
> > > > > > School of Software, Tsinghua University
> > > > > >
> > > > > >  黄向东
> > > > > > 清华大学 软件学院
> > > > > >
> > > > > >
> > > > > > Xiangdong Huang <sa...@gmail.com> 于2019年6月22日周六 下午3:19写道:
> > > > > >
> > > > > > > If you change the process like this, i.e., there are more than
> one
> > > > > > > unsealed TsFiles for each storage group, then  you have to
> modify the
> > > > > WAL
> > > > > > > module.. Because current WAL module only recognizes the last
> unsealed
> > > > > > > TsFile..
> > > > > > >
> > > > > > > By the way, "sealed" is better than "closed", I think..  A
> sealed file
> > > > > > > means the file which has the magic string at the head and the
> tail.
> > > > > > >
> > > > > > > Best,
> > > > > > > -----------------------------------
> > > > > > > Xiangdong Huang
> > > > > > > School of Software, Tsinghua University
> > > > > > >
> > > > > > >  黄向东
> > > > > > > 清华大学 软件学院
> > > > > > >
> > > > > > >
> > > > > > > Jialin Qiao <qj...@mails.tsinghua.edu.cn> 于2019年6月22日周六
> 下午2:54写道:
> > > > > > >
> > > > > > >>
> > > > > > >> Hi, I am solving the long-tail latency problem.
> > > > > > >>
> > > > > > >> There are some cases (blocking points) that blocking the
> insertion.
> > > > > For a
> > > > > > >> better understanding of this problem, I first introduce the
> writing
> > > > > process
> > > > > > >> of IoTDB:
> > > > > > >>
> > > > > > >> IoTDB maintains several independent engines (storage group)
> that
> > > > > supports
> > > > > > >> read and write. In the following, we focus on one engine. A
> engine
> > > > > > >> maintains several closed data files and one unclosed data
> file that
> > > > > > >> receives appended data. In memory, there is only one working
> memtable
> > > > > (m1)
> > > > > > >> that receives writes. There is also another memtable (m2)
> that will
> > > > > take
> > > > > > >> place m1 when m1 is full and being flushed.
> > > > > > >>
> > > > > > >> When a data item is inserted:
> > > > > > >>
> > > > > > >> (1)We insert it into the working memtable.
> > > > > > >> (2)We check the size of the memtable. If it reaches a
> threshold, we
> > > > > > >> submit a flush task “after the previous flush task is
> finished” and
> > > > > switch
> > > > > > >> the two memtables.
> > > > > > >> (3)We check the size of the unclosed file. If it reaches a
> threshold,
> > > > > we
> > > > > > >> close it “after the previous flush task is finished”.
> > > > > > >>
> > > > > > >> In the above steps, all the "after the previous flush task is
> > > > > finished"
> > > > > > >> will block the insertion process. One solution is to make all
> flush
> > > > > and
> > > > > > >> close task asynchronous. Some questions need to carefully
> considered:
> > > > > > >>
> > > > > > >> (1) Many memtables may be flushed concurrently to an unclosed
> file.
> > > > > How
> > > > > > >> to guarantee the order of serialization?
> > > > > > >> (2) Once a close task is submitted, a new unclosed file will
> be
> > > > > created
> > > > > > >> and receives appended data. So there will exists many
> unclosed files.
> > > > > How
> > > > > > >> the query and compaction process will be impacted?
> > > > > > >>
> > > > > > >> Thanks,
> > > > > > >>
> > > > > > >> Jialin Qiao
> > > > > > >> School of Software, Tsinghua University
> > > > > > >>
> > > > > > >> 乔嘉林
> > > > > > >> 清华大学 软件学院
> > > > > > >>
> > > > > > >> > -----原始邮件-----
> > > > > > >> > 发件人: "Xiangdong Huang" <sa...@gmail.com>
> > > > > > >> > 发送时间: 2019-06-04 23:08:34 (星期二)
> > > > > > >> > 收件人: dev@iotdb.apache.org, "江天" <jt...@163.com>
> > > > > > >> > 抄送:
> > > > > > >> > 主题: Re: [jira] [Created] (IOTDB-112) Avoid long tail
> insertion
> > > > > which is
> > > > > > >> caused by synchronized close-bufferwrite
> > > > > > >> >
> > > > > > >> > I attached the histogram of the latency in the JIRA.
> > > > > > >> >
> > > > > > >> > The x-axis is the latency while the y-axis is the cumulative
> > > > > > >> distribution.
> > > > > > >> > We can see that about 30% insertion can be finished in
> 20ms, and 60%
> > > > > > >> > insertion can be finished in 40ms even though the IoTDB
> instance is
> > > > > > >> serving
> > > > > > >> > for a heavy workload... So, eliminating the long tail
> insertion can
> > > > > make
> > > > > > >> > the average latency far better.
> > > > > > >> >
> > > > > > >> > If someone is working on the refactor_overflow or
> > > > > refactor_bufferwrite,
> > > > > > >> > please pay attention to the code branch for this issue.
> > > > > > >> >
> > > > > > >> > Best,
> > > > > > >> >
> > > > > > >> > -----------------------------------
> > > > > > >> > Xiangdong Huang
> > > > > > >> > School of Software, Tsinghua University
> > > > > > >> >
> > > > > > >> >  黄向东
> > > > > > >> > 清华大学 软件学院
> > > > > > >> >
> > > > > > >> >
> > > > > > >> > xiangdong Huang (JIRA) <ji...@apache.org> 于2019年6月4日周二
> 下午11:00写道:
> > > > > > >> >
> > > > > > >> > > xiangdong Huang created IOTDB-112:
> > > > > > >> > > -------------------------------------
> > > > > > >> > >
> > > > > > >> > >              Summary: Avoid long tail insertion which is
> caused by
> > > > > > >> > > synchronized close-bufferwrite
> > > > > > >> > >                  Key: IOTDB-112
> > > > > > >> > >                  URL:
> > > > > https://issues.apache.org/jira/browse/IOTDB-112
> > > > > > >> > >              Project: Apache IoTDB
> > > > > > >> > >           Issue Type: Improvement
> > > > > > >> > >             Reporter: xiangdong Huang
> > > > > > >> > >
> > > > > > >> > >
> > > > > > >> > > In our test, IoTDB has a good insertion performance, and
> the
> > > > > average
> > > > > > >> > > latency can be ~200 ms in a given workload and hardware.
> > > > > > >> > >
> > > > > > >> > > However, when we draw the histogram of the latency, we
> find that
> > > > > 97.5%
> > > > > > >> > > latencies are less than 200 ms, while 2.7% latencies are
> greater.
> > > > > The
> > > > > > >> > > result shows that there are some long tail latency.
> > > > > > >> > >
> > > > > > >> > > Then we find that some insertion latencies are about 30
> seconds...
> > > > > > >> (but
> > > > > > >> > > the ratio is less than 0.5%). Indeed, for each
> connection, a long
> > > > > tail
> > > > > > >> > > insertion appears per 1 or 2 minutes....
> > > > > > >> > >
> > > > > > >> > > By reading source codes, I think it is because that in the
> > > > > insertion
> > > > > > >> > > function,
> > > > > > >> > >
> > > > > > >> > > `private void insertBufferWrite(FileNodeProcessor
> > > > > fileNodeProcessor,
> > > > > > >> long
> > > > > > >> > > timestamp,
> > > > > > >> > >  boolean isMonitor, TSRecord tsRecord, String deviceId)`,
> > > > > > >> > >
> > > > > > >> > > if the corresponding TsFile is too large, the function is
> blocked
> > > > > > >> until
> > > > > > >> > > the memtable is flushed on disk and the TsFile is sealed
> (we call
> > > > > it
> > > > > > >> as
> > > > > > >> > > closing a TsFile). The latencies of the long tail
> insertions are
> > > > > very
> > > > > > >> close
> > > > > > >> > > to the time cost of flushing and sealing a TsFile.
> > > > > > >> > >
> > > > > > >> > > So, if we set the closing function using the async mode,
> we can
> > > > > avoid
> > > > > > >> the
> > > > > > >> > > long tail insertion.
> > > > > > >> > >
> > > > > > >> > > However,  there are some side effects we have to fix:
> > > > > > >> > >  # At the same time, if a new insertion comes, then a new
> memtable
> > > > > > >> should
> > > > > > >> > > be assigned, and a new unsealed TsFile is created;
> > > > > > >> > >  # That means that there are more than 1 unsealed TsFiles
> if the
> > > > > > >> system is
> > > > > > >> > > crashed before the closing function is finished. So, we
> have to
> > > > > > >> modify the
> > > > > > >> > > startup process to recover these files.
> > > > > > >> > >
> > > > > > >> > > Is there any other side effect that I have to pay
> attention to?
> > > > > > >> > >
> > > > > > >> > >
> > > > > > >> > >
> > > > > > >> > > --
> > > > > > >> > > This message was sent by Atlassian JIRA
> > > > > > >> > > (v7.6.3#76005)
> > > > > > >> > >
> > > > > > >>
> > > > > > >
> > > > >
>

Re: Re: Avoid long-tail insertion

Posted by 李天安 <lt...@mails.tsinghua.edu.cn>.
Hi,
 I'm also involved in the design of the new storage engine. Let me complement the new design of the flush task.  

 To improve system performance, we changed flush tasks from synchronous to asynchronous. We introduced a Flush Manager to manage all flush tasks. The tricky problem is that each Unsealed TsFile Processor (UFP) corresponds to a data file on a disk, so the flush tasks in each UFP(There is a flushing queue in UFP) need to be executed sequentially. However, flush tasks in different UFPs have no sequential requirements. How to design them to meet the above requirements?

 We introduce a UFP FIFO queue in Flush Manager, and add a boolean attribute ‘managedByFlushManager’ to each UFP to indicate whether it is managed by Flush Manager. Flush Manager maintains a Flush thread pool to perform Flush tasks, so the lifecycle of a Flush task is
1. UFP are submitted to FlushManager,FlushManager add UFP to its queue and set ‘managedByFlushManager’  of the UFP to true.
2. The Flush Pool in FlushManager start a flush thread to execute task.
3. Flush thread polls the first UFP from UFP queue in FlushManager -> polls the first flush task in UFP -> completes the flush task -> set ‘managedByFlushManager’  of the UFP to false. 

There are two ways to submit a UFP to FlushManager:
1. UFP, whenever a MemTable reaches a certain size or forcibly triggers a flush task, it submits itself to Flush Manager (because the queue in Flush Manager is UFP). Flush Manager first determines whether UFP meets the criteria for submission:
(1) UFP is not managed by Flush Manager, i.e.'managedByFlushManager' is false
(2) The Flush task queue in UFP is not empty, that is, there are at least one flush task to be executed.

2. When the Flush thread completes the flush task, it sets ‘managedByFlushManager’ to false and resubmits the UFP of the completed flush task to the FlushManager.

Through the above design, we can ensure that at the same time for each UFP, Flush Manager will only manage at most once and execute at most one flush task, while there is no restriction between different UFPs. At the same time, resubmitting UFP after the end of the task in each Flush thread ensures that all tasks can be executed. Therefore, we solve the above problem and the design meets the requirements of Flush Manager.

Best Regards,
-------------------------------------
Tianan Li
School of Software, Tsinghua University

> -----原始邮件-----
> 发件人: "Jialin Qiao" <qj...@mails.tsinghua.edu.cn>
> 发送时间: 2019-06-27 11:27:24 (星期四)
> 收件人: dev@iotdb.apache.org
> 抄送: 
> 主题: Re: Avoid long-tail insertion
> 
> Hi,
> 
> The new storage engine is designed to have the following components: 
> 
> (1) MemTable: A memory structure, which stores all inserted data in memory. 
> 
> (2) MemtablePool: Manages all memtables. All memtables are gotten from this pool. The total number of memtables is fixed 
> in the system. Once the pool do not has available memtables, the getMemtable() operation will wait or directly return.
> 
> (3) UnsealedTsFileProcessor (UFP): A writer for one data file. It always has one working memtable that receives writes and a 
> list (flushing list) of memtables that for flush. Once the working memtable reaches a threshold, it will be moved to the 
> flushing list and the working memtable is set null. When a new write arrives, if the working memtable is null, UFP will 
> call getMemtable() of the MemtablePool to get one as the working memtable.
> 
> (4) StorageGroupProcessor (SGP): Each SGP is responsible for all writes and reads in one storage group. It always has one 
> working UFP that receives write and a list (closing list) of UFPs that for close. Once the file size of the working UFP reaches 
> a threshold, the UFP is moved to the closing list and the working UFP is set null. When a new write arrives, if the working UFP 
> is null, a new UFP is generated as working UFP and receives write. 
> 
> (5) StorageGroupManager (SGM): A manager of all SGPs in IoTDB. It is only responsible for routing read and write operations 
> to its corresponding SGP according to the deviceId of the operation.
> 
> (6) Flush thread: The flush thread poll a memtable from the flushing list in UFP and flush a memtable to disk. After flushing, 
> the memtable is returned to the MemtablePool.
> 
> These are only the main components of the new storage engine. Some things may be lost. It would be great if someone could 
> give some advices or supplementations.
> 
> Best,
> --
> Jialin Qiao
> School of Software, Tsinghua University
> 
> 乔嘉林
> 清华大学 软件学院
> 
> > -----原始邮件-----
> > 发件人: "Jialin Qiao" <qj...@mails.tsinghua.edu.cn>
> > 发送时间: 2019-06-24 20:24:05 (星期一)
> > 收件人: dev@iotdb.apache.org
> > 抄送: 
> > 主题: Re: Re: Re: Avoid long-tail insertion
> > 
> > 
> > Yes, there are many changes. The branch I am working on is feature_async_close_tsfile. 
> > Anyone interested is welcome to join and discuss.
> > 
> > Best,
> > --
> > Jialin Qiao
> > School of Software, Tsinghua University
> > 
> > 乔嘉林
> > 清华大学 软件学院
> > 
> > > -----原始邮件-----
> > > 发件人: "Xiangdong Huang" <sa...@gmail.com>
> > > 发送时间: 2019-06-23 10:59:29 (星期日)
> > > 收件人: dev@iotdb.apache.org
> > > 抄送: 
> > > 主题: Re: Re: Avoid long-tail insertion
> > > 
> > > Hi,
> > > 
> > > Once your work branch is almost ready, let me know so I can help to review.
> > > I think it is a HUGE PR...
> > > 
> > > -----------------------------------
> > > Xiangdong Huang
> > > School of Software, Tsinghua University
> > > 
> > >  黄向东
> > > 清华大学 软件学院
> > > 
> > > 
> > > Jialin Qiao <qj...@mails.tsinghua.edu.cn> 于2019年6月22日周六 下午9:57写道:
> > > 
> > > > Hi Xiangdong,
> > > >
> > > > I will merge this patch. Let "Directories" manage the folders of both
> > > > sequence and unSequence files is good.
> > > >
> > > > However, the naming of "Directories" is not clear. It would be better to
> > > > rename to "DirectoryManager"
> > > >
> > > > Best,
> > > > --
> > > > Jialin Qiao
> > > > School of Software, Tsinghua University
> > > >
> > > > 乔嘉林
> > > > 清华大学 软件学院
> > > >
> > > > > -----原始邮件-----
> > > > > 发件人: "Xiangdong Huang" <sa...@gmail.com>
> > > > > 发送时间: 2019-06-22 16:35:29 (星期六)
> > > > > 收件人: dev@iotdb.apache.org
> > > > > 抄送:
> > > > > 主题: Re: Avoid long-tail insertion
> > > > >
> > > > > Hi jialin,
> > > > >
> > > > > I submit some modifications for:
> > > > >
> > > > > * add the overflow data folder location setting in the
> > > > > iotdb-engine.properties;
> > > > > * let Directories.java to manage the above folder.
> > > > >
> > > > > If you need to refactor the overflow when you solving the long tail
> > > > issue,
> > > > > you can apply the patch from [1] first to simplify your work.
> > > > >
> > > > > [1]
> > > > >
> > > > https://issues.apache.org/jira/secure/attachment/12972547/overflow-folder.patch
> > > > >
> > > > > Best,
> > > > > -----------------------------------
> > > > > Xiangdong Huang
> > > > > School of Software, Tsinghua University
> > > > >
> > > > >  黄向东
> > > > > 清华大学 软件学院
> > > > >
> > > > >
> > > > > Xiangdong Huang <sa...@gmail.com> 于2019年6月22日周六 下午3:19写道:
> > > > >
> > > > > > If you change the process like this, i.e., there are more than one
> > > > > > unsealed TsFiles for each storage group, then  you have to modify the
> > > > WAL
> > > > > > module.. Because current WAL module only recognizes the last unsealed
> > > > > > TsFile..
> > > > > >
> > > > > > By the way, "sealed" is better than "closed", I think..  A sealed file
> > > > > > means the file which has the magic string at the head and the tail.
> > > > > >
> > > > > > Best,
> > > > > > -----------------------------------
> > > > > > Xiangdong Huang
> > > > > > School of Software, Tsinghua University
> > > > > >
> > > > > >  黄向东
> > > > > > 清华大学 软件学院
> > > > > >
> > > > > >
> > > > > > Jialin Qiao <qj...@mails.tsinghua.edu.cn> 于2019年6月22日周六 下午2:54写道:
> > > > > >
> > > > > >>
> > > > > >> Hi, I am solving the long-tail latency problem.
> > > > > >>
> > > > > >> There are some cases (blocking points) that blocking the insertion.
> > > > For a
> > > > > >> better understanding of this problem, I first introduce the writing
> > > > process
> > > > > >> of IoTDB:
> > > > > >>
> > > > > >> IoTDB maintains several independent engines (storage group) that
> > > > supports
> > > > > >> read and write. In the following, we focus on one engine. A engine
> > > > > >> maintains several closed data files and one unclosed data file that
> > > > > >> receives appended data. In memory, there is only one working memtable
> > > > (m1)
> > > > > >> that receives writes. There is also another memtable (m2) that will
> > > > take
> > > > > >> place m1 when m1 is full and being flushed.
> > > > > >>
> > > > > >> When a data item is inserted:
> > > > > >>
> > > > > >> (1)We insert it into the working memtable.
> > > > > >> (2)We check the size of the memtable. If it reaches a threshold, we
> > > > > >> submit a flush task “after the previous flush task is finished” and
> > > > switch
> > > > > >> the two memtables.
> > > > > >> (3)We check the size of the unclosed file. If it reaches a threshold,
> > > > we
> > > > > >> close it “after the previous flush task is finished”.
> > > > > >>
> > > > > >> In the above steps, all the "after the previous flush task is
> > > > finished"
> > > > > >> will block the insertion process. One solution is to make all flush
> > > > and
> > > > > >> close task asynchronous. Some questions need to carefully considered:
> > > > > >>
> > > > > >> (1) Many memtables may be flushed concurrently to an unclosed file.
> > > > How
> > > > > >> to guarantee the order of serialization?
> > > > > >> (2) Once a close task is submitted, a new unclosed file will be
> > > > created
> > > > > >> and receives appended data. So there will exists many unclosed files.
> > > > How
> > > > > >> the query and compaction process will be impacted?
> > > > > >>
> > > > > >> Thanks,
> > > > > >>
> > > > > >> Jialin Qiao
> > > > > >> School of Software, Tsinghua University
> > > > > >>
> > > > > >> 乔嘉林
> > > > > >> 清华大学 软件学院
> > > > > >>
> > > > > >> > -----原始邮件-----
> > > > > >> > 发件人: "Xiangdong Huang" <sa...@gmail.com>
> > > > > >> > 发送时间: 2019-06-04 23:08:34 (星期二)
> > > > > >> > 收件人: dev@iotdb.apache.org, "江天" <jt...@163.com>
> > > > > >> > 抄送:
> > > > > >> > 主题: Re: [jira] [Created] (IOTDB-112) Avoid long tail insertion
> > > > which is
> > > > > >> caused by synchronized close-bufferwrite
> > > > > >> >
> > > > > >> > I attached the histogram of the latency in the JIRA.
> > > > > >> >
> > > > > >> > The x-axis is the latency while the y-axis is the cumulative
> > > > > >> distribution.
> > > > > >> > We can see that about 30% insertion can be finished in 20ms, and 60%
> > > > > >> > insertion can be finished in 40ms even though the IoTDB instance is
> > > > > >> serving
> > > > > >> > for a heavy workload... So, eliminating the long tail insertion can
> > > > make
> > > > > >> > the average latency far better.
> > > > > >> >
> > > > > >> > If someone is working on the refactor_overflow or
> > > > refactor_bufferwrite,
> > > > > >> > please pay attention to the code branch for this issue.
> > > > > >> >
> > > > > >> > Best,
> > > > > >> >
> > > > > >> > -----------------------------------
> > > > > >> > Xiangdong Huang
> > > > > >> > School of Software, Tsinghua University
> > > > > >> >
> > > > > >> >  黄向东
> > > > > >> > 清华大学 软件学院
> > > > > >> >
> > > > > >> >
> > > > > >> > xiangdong Huang (JIRA) <ji...@apache.org> 于2019年6月4日周二 下午11:00写道:
> > > > > >> >
> > > > > >> > > xiangdong Huang created IOTDB-112:
> > > > > >> > > -------------------------------------
> > > > > >> > >
> > > > > >> > >              Summary: Avoid long tail insertion which is caused by
> > > > > >> > > synchronized close-bufferwrite
> > > > > >> > >                  Key: IOTDB-112
> > > > > >> > >                  URL:
> > > > https://issues.apache.org/jira/browse/IOTDB-112
> > > > > >> > >              Project: Apache IoTDB
> > > > > >> > >           Issue Type: Improvement
> > > > > >> > >             Reporter: xiangdong Huang
> > > > > >> > >
> > > > > >> > >
> > > > > >> > > In our test, IoTDB has a good insertion performance, and the
> > > > average
> > > > > >> > > latency can be ~200 ms in a given workload and hardware.
> > > > > >> > >
> > > > > >> > > However, when we draw the histogram of the latency, we find that
> > > > 97.5%
> > > > > >> > > latencies are less than 200 ms, while 2.7% latencies are greater.
> > > > The
> > > > > >> > > result shows that there are some long tail latency.
> > > > > >> > >
> > > > > >> > > Then we find that some insertion latencies are about 30 seconds...
> > > > > >> (but
> > > > > >> > > the ratio is less than 0.5%). Indeed, for each connection, a long
> > > > tail
> > > > > >> > > insertion appears per 1 or 2 minutes....
> > > > > >> > >
> > > > > >> > > By reading source codes, I think it is because that in the
> > > > insertion
> > > > > >> > > function,
> > > > > >> > >
> > > > > >> > > `private void insertBufferWrite(FileNodeProcessor
> > > > fileNodeProcessor,
> > > > > >> long
> > > > > >> > > timestamp,
> > > > > >> > >  boolean isMonitor, TSRecord tsRecord, String deviceId)`,
> > > > > >> > >
> > > > > >> > > if the corresponding TsFile is too large, the function is blocked
> > > > > >> until
> > > > > >> > > the memtable is flushed on disk and the TsFile is sealed (we call
> > > > it
> > > > > >> as
> > > > > >> > > closing a TsFile). The latencies of the long tail insertions are
> > > > very
> > > > > >> close
> > > > > >> > > to the time cost of flushing and sealing a TsFile.
> > > > > >> > >
> > > > > >> > > So, if we set the closing function using the async mode, we can
> > > > avoid
> > > > > >> the
> > > > > >> > > long tail insertion.
> > > > > >> > >
> > > > > >> > > However,  there are some side effects we have to fix:
> > > > > >> > >  # At the same time, if a new insertion comes, then a new memtable
> > > > > >> should
> > > > > >> > > be assigned, and a new unsealed TsFile is created;
> > > > > >> > >  # That means that there are more than 1 unsealed TsFiles if the
> > > > > >> system is
> > > > > >> > > crashed before the closing function is finished. So, we have to
> > > > > >> modify the
> > > > > >> > > startup process to recover these files.
> > > > > >> > >
> > > > > >> > > Is there any other side effect that I have to pay attention to?
> > > > > >> > >
> > > > > >> > >
> > > > > >> > >
> > > > > >> > > --
> > > > > >> > > This message was sent by Atlassian JIRA
> > > > > >> > > (v7.6.3#76005)
> > > > > >> > >
> > > > > >>
> > > > > >
> > > >

Re: Avoid long-tail insertion

Posted by Jialin Qiao <qj...@mails.tsinghua.edu.cn>.
Hi,

The new storage engine is designed to have the following components: 

(1) MemTable: A memory structure, which stores all inserted data in memory. 

(2) MemtablePool: Manages all memtables. All memtables are gotten from this pool. The total number of memtables is fixed 
in the system. Once the pool do not has available memtables, the getMemtable() operation will wait or directly return.

(3) UnsealedTsFileProcessor (UFP): A writer for one data file. It always has one working memtable that receives writes and a 
list (flushing list) of memtables that for flush. Once the working memtable reaches a threshold, it will be moved to the 
flushing list and the working memtable is set null. When a new write arrives, if the working memtable is null, UFP will 
call getMemtable() of the MemtablePool to get one as the working memtable.

(4) StorageGroupProcessor (SGP): Each SGP is responsible for all writes and reads in one storage group. It always has one 
working UFP that receives write and a list (closing list) of UFPs that for close. Once the file size of the working UFP reaches 
a threshold, the UFP is moved to the closing list and the working UFP is set null. When a new write arrives, if the working UFP 
is null, a new UFP is generated as working UFP and receives write. 

(5) StorageGroupManager (SGM): A manager of all SGPs in IoTDB. It is only responsible for routing read and write operations 
to its corresponding SGP according to the deviceId of the operation.

(6) Flush thread: The flush thread poll a memtable from the flushing list in UFP and flush a memtable to disk. After flushing, 
the memtable is returned to the MemtablePool.

These are only the main components of the new storage engine. Some things may be lost. It would be great if someone could 
give some advices or supplementations.

Best,
--
Jialin Qiao
School of Software, Tsinghua University

乔嘉林
清华大学 软件学院

> -----原始邮件-----
> 发件人: "Jialin Qiao" <qj...@mails.tsinghua.edu.cn>
> 发送时间: 2019-06-24 20:24:05 (星期一)
> 收件人: dev@iotdb.apache.org
> 抄送: 
> 主题: Re: Re: Re: Avoid long-tail insertion
> 
> 
> Yes, there are many changes. The branch I am working on is feature_async_close_tsfile. 
> Anyone interested is welcome to join and discuss.
> 
> Best,
> --
> Jialin Qiao
> School of Software, Tsinghua University
> 
> 乔嘉林
> 清华大学 软件学院
> 
> > -----原始邮件-----
> > 发件人: "Xiangdong Huang" <sa...@gmail.com>
> > 发送时间: 2019-06-23 10:59:29 (星期日)
> > 收件人: dev@iotdb.apache.org
> > 抄送: 
> > 主题: Re: Re: Avoid long-tail insertion
> > 
> > Hi,
> > 
> > Once your work branch is almost ready, let me know so I can help to review.
> > I think it is a HUGE PR...
> > 
> > -----------------------------------
> > Xiangdong Huang
> > School of Software, Tsinghua University
> > 
> >  黄向东
> > 清华大学 软件学院
> > 
> > 
> > Jialin Qiao <qj...@mails.tsinghua.edu.cn> 于2019年6月22日周六 下午9:57写道:
> > 
> > > Hi Xiangdong,
> > >
> > > I will merge this patch. Let "Directories" manage the folders of both
> > > sequence and unSequence files is good.
> > >
> > > However, the naming of "Directories" is not clear. It would be better to
> > > rename to "DirectoryManager"
> > >
> > > Best,
> > > --
> > > Jialin Qiao
> > > School of Software, Tsinghua University
> > >
> > > 乔嘉林
> > > 清华大学 软件学院
> > >
> > > > -----原始邮件-----
> > > > 发件人: "Xiangdong Huang" <sa...@gmail.com>
> > > > 发送时间: 2019-06-22 16:35:29 (星期六)
> > > > 收件人: dev@iotdb.apache.org
> > > > 抄送:
> > > > 主题: Re: Avoid long-tail insertion
> > > >
> > > > Hi jialin,
> > > >
> > > > I submit some modifications for:
> > > >
> > > > * add the overflow data folder location setting in the
> > > > iotdb-engine.properties;
> > > > * let Directories.java to manage the above folder.
> > > >
> > > > If you need to refactor the overflow when you solving the long tail
> > > issue,
> > > > you can apply the patch from [1] first to simplify your work.
> > > >
> > > > [1]
> > > >
> > > https://issues.apache.org/jira/secure/attachment/12972547/overflow-folder.patch
> > > >
> > > > Best,
> > > > -----------------------------------
> > > > Xiangdong Huang
> > > > School of Software, Tsinghua University
> > > >
> > > >  黄向东
> > > > 清华大学 软件学院
> > > >
> > > >
> > > > Xiangdong Huang <sa...@gmail.com> 于2019年6月22日周六 下午3:19写道:
> > > >
> > > > > If you change the process like this, i.e., there are more than one
> > > > > unsealed TsFiles for each storage group, then  you have to modify the
> > > WAL
> > > > > module.. Because current WAL module only recognizes the last unsealed
> > > > > TsFile..
> > > > >
> > > > > By the way, "sealed" is better than "closed", I think..  A sealed file
> > > > > means the file which has the magic string at the head and the tail.
> > > > >
> > > > > Best,
> > > > > -----------------------------------
> > > > > Xiangdong Huang
> > > > > School of Software, Tsinghua University
> > > > >
> > > > >  黄向东
> > > > > 清华大学 软件学院
> > > > >
> > > > >
> > > > > Jialin Qiao <qj...@mails.tsinghua.edu.cn> 于2019年6月22日周六 下午2:54写道:
> > > > >
> > > > >>
> > > > >> Hi, I am solving the long-tail latency problem.
> > > > >>
> > > > >> There are some cases (blocking points) that blocking the insertion.
> > > For a
> > > > >> better understanding of this problem, I first introduce the writing
> > > process
> > > > >> of IoTDB:
> > > > >>
> > > > >> IoTDB maintains several independent engines (storage group) that
> > > supports
> > > > >> read and write. In the following, we focus on one engine. A engine
> > > > >> maintains several closed data files and one unclosed data file that
> > > > >> receives appended data. In memory, there is only one working memtable
> > > (m1)
> > > > >> that receives writes. There is also another memtable (m2) that will
> > > take
> > > > >> place m1 when m1 is full and being flushed.
> > > > >>
> > > > >> When a data item is inserted:
> > > > >>
> > > > >> (1)We insert it into the working memtable.
> > > > >> (2)We check the size of the memtable. If it reaches a threshold, we
> > > > >> submit a flush task “after the previous flush task is finished” and
> > > switch
> > > > >> the two memtables.
> > > > >> (3)We check the size of the unclosed file. If it reaches a threshold,
> > > we
> > > > >> close it “after the previous flush task is finished”.
> > > > >>
> > > > >> In the above steps, all the "after the previous flush task is
> > > finished"
> > > > >> will block the insertion process. One solution is to make all flush
> > > and
> > > > >> close task asynchronous. Some questions need to carefully considered:
> > > > >>
> > > > >> (1) Many memtables may be flushed concurrently to an unclosed file.
> > > How
> > > > >> to guarantee the order of serialization?
> > > > >> (2) Once a close task is submitted, a new unclosed file will be
> > > created
> > > > >> and receives appended data. So there will exists many unclosed files.
> > > How
> > > > >> the query and compaction process will be impacted?
> > > > >>
> > > > >> Thanks,
> > > > >>
> > > > >> Jialin Qiao
> > > > >> School of Software, Tsinghua University
> > > > >>
> > > > >> 乔嘉林
> > > > >> 清华大学 软件学院
> > > > >>
> > > > >> > -----原始邮件-----
> > > > >> > 发件人: "Xiangdong Huang" <sa...@gmail.com>
> > > > >> > 发送时间: 2019-06-04 23:08:34 (星期二)
> > > > >> > 收件人: dev@iotdb.apache.org, "江天" <jt...@163.com>
> > > > >> > 抄送:
> > > > >> > 主题: Re: [jira] [Created] (IOTDB-112) Avoid long tail insertion
> > > which is
> > > > >> caused by synchronized close-bufferwrite
> > > > >> >
> > > > >> > I attached the histogram of the latency in the JIRA.
> > > > >> >
> > > > >> > The x-axis is the latency while the y-axis is the cumulative
> > > > >> distribution.
> > > > >> > We can see that about 30% insertion can be finished in 20ms, and 60%
> > > > >> > insertion can be finished in 40ms even though the IoTDB instance is
> > > > >> serving
> > > > >> > for a heavy workload... So, eliminating the long tail insertion can
> > > make
> > > > >> > the average latency far better.
> > > > >> >
> > > > >> > If someone is working on the refactor_overflow or
> > > refactor_bufferwrite,
> > > > >> > please pay attention to the code branch for this issue.
> > > > >> >
> > > > >> > Best,
> > > > >> >
> > > > >> > -----------------------------------
> > > > >> > Xiangdong Huang
> > > > >> > School of Software, Tsinghua University
> > > > >> >
> > > > >> >  黄向东
> > > > >> > 清华大学 软件学院
> > > > >> >
> > > > >> >
> > > > >> > xiangdong Huang (JIRA) <ji...@apache.org> 于2019年6月4日周二 下午11:00写道:
> > > > >> >
> > > > >> > > xiangdong Huang created IOTDB-112:
> > > > >> > > -------------------------------------
> > > > >> > >
> > > > >> > >              Summary: Avoid long tail insertion which is caused by
> > > > >> > > synchronized close-bufferwrite
> > > > >> > >                  Key: IOTDB-112
> > > > >> > >                  URL:
> > > https://issues.apache.org/jira/browse/IOTDB-112
> > > > >> > >              Project: Apache IoTDB
> > > > >> > >           Issue Type: Improvement
> > > > >> > >             Reporter: xiangdong Huang
> > > > >> > >
> > > > >> > >
> > > > >> > > In our test, IoTDB has a good insertion performance, and the
> > > average
> > > > >> > > latency can be ~200 ms in a given workload and hardware.
> > > > >> > >
> > > > >> > > However, when we draw the histogram of the latency, we find that
> > > 97.5%
> > > > >> > > latencies are less than 200 ms, while 2.7% latencies are greater.
> > > The
> > > > >> > > result shows that there are some long tail latency.
> > > > >> > >
> > > > >> > > Then we find that some insertion latencies are about 30 seconds...
> > > > >> (but
> > > > >> > > the ratio is less than 0.5%). Indeed, for each connection, a long
> > > tail
> > > > >> > > insertion appears per 1 or 2 minutes....
> > > > >> > >
> > > > >> > > By reading source codes, I think it is because that in the
> > > insertion
> > > > >> > > function,
> > > > >> > >
> > > > >> > > `private void insertBufferWrite(FileNodeProcessor
> > > fileNodeProcessor,
> > > > >> long
> > > > >> > > timestamp,
> > > > >> > >  boolean isMonitor, TSRecord tsRecord, String deviceId)`,
> > > > >> > >
> > > > >> > > if the corresponding TsFile is too large, the function is blocked
> > > > >> until
> > > > >> > > the memtable is flushed on disk and the TsFile is sealed (we call
> > > it
> > > > >> as
> > > > >> > > closing a TsFile). The latencies of the long tail insertions are
> > > very
> > > > >> close
> > > > >> > > to the time cost of flushing and sealing a TsFile.
> > > > >> > >
> > > > >> > > So, if we set the closing function using the async mode, we can
> > > avoid
> > > > >> the
> > > > >> > > long tail insertion.
> > > > >> > >
> > > > >> > > However,  there are some side effects we have to fix:
> > > > >> > >  # At the same time, if a new insertion comes, then a new memtable
> > > > >> should
> > > > >> > > be assigned, and a new unsealed TsFile is created;
> > > > >> > >  # That means that there are more than 1 unsealed TsFiles if the
> > > > >> system is
> > > > >> > > crashed before the closing function is finished. So, we have to
> > > > >> modify the
> > > > >> > > startup process to recover these files.
> > > > >> > >
> > > > >> > > Is there any other side effect that I have to pay attention to?
> > > > >> > >
> > > > >> > >
> > > > >> > >
> > > > >> > > --
> > > > >> > > This message was sent by Atlassian JIRA
> > > > >> > > (v7.6.3#76005)
> > > > >> > >
> > > > >>
> > > > >
> > >

Re: Re: Re: Avoid long-tail insertion

Posted by Jialin Qiao <qj...@mails.tsinghua.edu.cn>.
Yes, there are many changes. The branch I am working on is feature_async_close_tsfile. 
Anyone interested is welcome to join and discuss.

Best,
--
Jialin Qiao
School of Software, Tsinghua University

乔嘉林
清华大学 软件学院

> -----原始邮件-----
> 发件人: "Xiangdong Huang" <sa...@gmail.com>
> 发送时间: 2019-06-23 10:59:29 (星期日)
> 收件人: dev@iotdb.apache.org
> 抄送: 
> 主题: Re: Re: Avoid long-tail insertion
> 
> Hi,
> 
> Once your work branch is almost ready, let me know so I can help to review.
> I think it is a HUGE PR...
> 
> -----------------------------------
> Xiangdong Huang
> School of Software, Tsinghua University
> 
>  黄向东
> 清华大学 软件学院
> 
> 
> Jialin Qiao <qj...@mails.tsinghua.edu.cn> 于2019年6月22日周六 下午9:57写道:
> 
> > Hi Xiangdong,
> >
> > I will merge this patch. Let "Directories" manage the folders of both
> > sequence and unSequence files is good.
> >
> > However, the naming of "Directories" is not clear. It would be better to
> > rename to "DirectoryManager"
> >
> > Best,
> > --
> > Jialin Qiao
> > School of Software, Tsinghua University
> >
> > 乔嘉林
> > 清华大学 软件学院
> >
> > > -----原始邮件-----
> > > 发件人: "Xiangdong Huang" <sa...@gmail.com>
> > > 发送时间: 2019-06-22 16:35:29 (星期六)
> > > 收件人: dev@iotdb.apache.org
> > > 抄送:
> > > 主题: Re: Avoid long-tail insertion
> > >
> > > Hi jialin,
> > >
> > > I submit some modifications for:
> > >
> > > * add the overflow data folder location setting in the
> > > iotdb-engine.properties;
> > > * let Directories.java to manage the above folder.
> > >
> > > If you need to refactor the overflow when you solving the long tail
> > issue,
> > > you can apply the patch from [1] first to simplify your work.
> > >
> > > [1]
> > >
> > https://issues.apache.org/jira/secure/attachment/12972547/overflow-folder.patch
> > >
> > > Best,
> > > -----------------------------------
> > > Xiangdong Huang
> > > School of Software, Tsinghua University
> > >
> > >  黄向东
> > > 清华大学 软件学院
> > >
> > >
> > > Xiangdong Huang <sa...@gmail.com> 于2019年6月22日周六 下午3:19写道:
> > >
> > > > If you change the process like this, i.e., there are more than one
> > > > unsealed TsFiles for each storage group, then  you have to modify the
> > WAL
> > > > module.. Because current WAL module only recognizes the last unsealed
> > > > TsFile..
> > > >
> > > > By the way, "sealed" is better than "closed", I think..  A sealed file
> > > > means the file which has the magic string at the head and the tail.
> > > >
> > > > Best,
> > > > -----------------------------------
> > > > Xiangdong Huang
> > > > School of Software, Tsinghua University
> > > >
> > > >  黄向东
> > > > 清华大学 软件学院
> > > >
> > > >
> > > > Jialin Qiao <qj...@mails.tsinghua.edu.cn> 于2019年6月22日周六 下午2:54写道:
> > > >
> > > >>
> > > >> Hi, I am solving the long-tail latency problem.
> > > >>
> > > >> There are some cases (blocking points) that blocking the insertion.
> > For a
> > > >> better understanding of this problem, I first introduce the writing
> > process
> > > >> of IoTDB:
> > > >>
> > > >> IoTDB maintains several independent engines (storage group) that
> > supports
> > > >> read and write. In the following, we focus on one engine. A engine
> > > >> maintains several closed data files and one unclosed data file that
> > > >> receives appended data. In memory, there is only one working memtable
> > (m1)
> > > >> that receives writes. There is also another memtable (m2) that will
> > take
> > > >> place m1 when m1 is full and being flushed.
> > > >>
> > > >> When a data item is inserted:
> > > >>
> > > >> (1)We insert it into the working memtable.
> > > >> (2)We check the size of the memtable. If it reaches a threshold, we
> > > >> submit a flush task “after the previous flush task is finished” and
> > switch
> > > >> the two memtables.
> > > >> (3)We check the size of the unclosed file. If it reaches a threshold,
> > we
> > > >> close it “after the previous flush task is finished”.
> > > >>
> > > >> In the above steps, all the "after the previous flush task is
> > finished"
> > > >> will block the insertion process. One solution is to make all flush
> > and
> > > >> close task asynchronous. Some questions need to carefully considered:
> > > >>
> > > >> (1) Many memtables may be flushed concurrently to an unclosed file.
> > How
> > > >> to guarantee the order of serialization?
> > > >> (2) Once a close task is submitted, a new unclosed file will be
> > created
> > > >> and receives appended data. So there will exists many unclosed files.
> > How
> > > >> the query and compaction process will be impacted?
> > > >>
> > > >> Thanks,
> > > >>
> > > >> Jialin Qiao
> > > >> School of Software, Tsinghua University
> > > >>
> > > >> 乔嘉林
> > > >> 清华大学 软件学院
> > > >>
> > > >> > -----原始邮件-----
> > > >> > 发件人: "Xiangdong Huang" <sa...@gmail.com>
> > > >> > 发送时间: 2019-06-04 23:08:34 (星期二)
> > > >> > 收件人: dev@iotdb.apache.org, "江天" <jt...@163.com>
> > > >> > 抄送:
> > > >> > 主题: Re: [jira] [Created] (IOTDB-112) Avoid long tail insertion
> > which is
> > > >> caused by synchronized close-bufferwrite
> > > >> >
> > > >> > I attached the histogram of the latency in the JIRA.
> > > >> >
> > > >> > The x-axis is the latency while the y-axis is the cumulative
> > > >> distribution.
> > > >> > We can see that about 30% insertion can be finished in 20ms, and 60%
> > > >> > insertion can be finished in 40ms even though the IoTDB instance is
> > > >> serving
> > > >> > for a heavy workload... So, eliminating the long tail insertion can
> > make
> > > >> > the average latency far better.
> > > >> >
> > > >> > If someone is working on the refactor_overflow or
> > refactor_bufferwrite,
> > > >> > please pay attention to the code branch for this issue.
> > > >> >
> > > >> > Best,
> > > >> >
> > > >> > -----------------------------------
> > > >> > Xiangdong Huang
> > > >> > School of Software, Tsinghua University
> > > >> >
> > > >> >  黄向东
> > > >> > 清华大学 软件学院
> > > >> >
> > > >> >
> > > >> > xiangdong Huang (JIRA) <ji...@apache.org> 于2019年6月4日周二 下午11:00写道:
> > > >> >
> > > >> > > xiangdong Huang created IOTDB-112:
> > > >> > > -------------------------------------
> > > >> > >
> > > >> > >              Summary: Avoid long tail insertion which is caused by
> > > >> > > synchronized close-bufferwrite
> > > >> > >                  Key: IOTDB-112
> > > >> > >                  URL:
> > https://issues.apache.org/jira/browse/IOTDB-112
> > > >> > >              Project: Apache IoTDB
> > > >> > >           Issue Type: Improvement
> > > >> > >             Reporter: xiangdong Huang
> > > >> > >
> > > >> > >
> > > >> > > In our test, IoTDB has a good insertion performance, and the
> > average
> > > >> > > latency can be ~200 ms in a given workload and hardware.
> > > >> > >
> > > >> > > However, when we draw the histogram of the latency, we find that
> > 97.5%
> > > >> > > latencies are less than 200 ms, while 2.7% latencies are greater.
> > The
> > > >> > > result shows that there are some long tail latency.
> > > >> > >
> > > >> > > Then we find that some insertion latencies are about 30 seconds...
> > > >> (but
> > > >> > > the ratio is less than 0.5%). Indeed, for each connection, a long
> > tail
> > > >> > > insertion appears per 1 or 2 minutes....
> > > >> > >
> > > >> > > By reading source codes, I think it is because that in the
> > insertion
> > > >> > > function,
> > > >> > >
> > > >> > > `private void insertBufferWrite(FileNodeProcessor
> > fileNodeProcessor,
> > > >> long
> > > >> > > timestamp,
> > > >> > >  boolean isMonitor, TSRecord tsRecord, String deviceId)`,
> > > >> > >
> > > >> > > if the corresponding TsFile is too large, the function is blocked
> > > >> until
> > > >> > > the memtable is flushed on disk and the TsFile is sealed (we call
> > it
> > > >> as
> > > >> > > closing a TsFile). The latencies of the long tail insertions are
> > very
> > > >> close
> > > >> > > to the time cost of flushing and sealing a TsFile.
> > > >> > >
> > > >> > > So, if we set the closing function using the async mode, we can
> > avoid
> > > >> the
> > > >> > > long tail insertion.
> > > >> > >
> > > >> > > However,  there are some side effects we have to fix:
> > > >> > >  # At the same time, if a new insertion comes, then a new memtable
> > > >> should
> > > >> > > be assigned, and a new unsealed TsFile is created;
> > > >> > >  # That means that there are more than 1 unsealed TsFiles if the
> > > >> system is
> > > >> > > crashed before the closing function is finished. So, we have to
> > > >> modify the
> > > >> > > startup process to recover these files.
> > > >> > >
> > > >> > > Is there any other side effect that I have to pay attention to?
> > > >> > >
> > > >> > >
> > > >> > >
> > > >> > > --
> > > >> > > This message was sent by Atlassian JIRA
> > > >> > > (v7.6.3#76005)
> > > >> > >
> > > >>
> > > >
> >

Re: Re: Avoid long-tail insertion

Posted by Xiangdong Huang <sa...@gmail.com>.
Hi,

Once your work branch is almost ready, let me know so I can help to review.
I think it is a HUGE PR...

-----------------------------------
Xiangdong Huang
School of Software, Tsinghua University

 黄向东
清华大学 软件学院


Jialin Qiao <qj...@mails.tsinghua.edu.cn> 于2019年6月22日周六 下午9:57写道:

> Hi Xiangdong,
>
> I will merge this patch. Let "Directories" manage the folders of both
> sequence and unSequence files is good.
>
> However, the naming of "Directories" is not clear. It would be better to
> rename to "DirectoryManager"
>
> Best,
> --
> Jialin Qiao
> School of Software, Tsinghua University
>
> 乔嘉林
> 清华大学 软件学院
>
> > -----原始邮件-----
> > 发件人: "Xiangdong Huang" <sa...@gmail.com>
> > 发送时间: 2019-06-22 16:35:29 (星期六)
> > 收件人: dev@iotdb.apache.org
> > 抄送:
> > 主题: Re: Avoid long-tail insertion
> >
> > Hi jialin,
> >
> > I submit some modifications for:
> >
> > * add the overflow data folder location setting in the
> > iotdb-engine.properties;
> > * let Directories.java to manage the above folder.
> >
> > If you need to refactor the overflow when you solving the long tail
> issue,
> > you can apply the patch from [1] first to simplify your work.
> >
> > [1]
> >
> https://issues.apache.org/jira/secure/attachment/12972547/overflow-folder.patch
> >
> > Best,
> > -----------------------------------
> > Xiangdong Huang
> > School of Software, Tsinghua University
> >
> >  黄向东
> > 清华大学 软件学院
> >
> >
> > Xiangdong Huang <sa...@gmail.com> 于2019年6月22日周六 下午3:19写道:
> >
> > > If you change the process like this, i.e., there are more than one
> > > unsealed TsFiles for each storage group, then  you have to modify the
> WAL
> > > module.. Because current WAL module only recognizes the last unsealed
> > > TsFile..
> > >
> > > By the way, "sealed" is better than "closed", I think..  A sealed file
> > > means the file which has the magic string at the head and the tail.
> > >
> > > Best,
> > > -----------------------------------
> > > Xiangdong Huang
> > > School of Software, Tsinghua University
> > >
> > >  黄向东
> > > 清华大学 软件学院
> > >
> > >
> > > Jialin Qiao <qj...@mails.tsinghua.edu.cn> 于2019年6月22日周六 下午2:54写道:
> > >
> > >>
> > >> Hi, I am solving the long-tail latency problem.
> > >>
> > >> There are some cases (blocking points) that blocking the insertion.
> For a
> > >> better understanding of this problem, I first introduce the writing
> process
> > >> of IoTDB:
> > >>
> > >> IoTDB maintains several independent engines (storage group) that
> supports
> > >> read and write. In the following, we focus on one engine. A engine
> > >> maintains several closed data files and one unclosed data file that
> > >> receives appended data. In memory, there is only one working memtable
> (m1)
> > >> that receives writes. There is also another memtable (m2) that will
> take
> > >> place m1 when m1 is full and being flushed.
> > >>
> > >> When a data item is inserted:
> > >>
> > >> (1)We insert it into the working memtable.
> > >> (2)We check the size of the memtable. If it reaches a threshold, we
> > >> submit a flush task “after the previous flush task is finished” and
> switch
> > >> the two memtables.
> > >> (3)We check the size of the unclosed file. If it reaches a threshold,
> we
> > >> close it “after the previous flush task is finished”.
> > >>
> > >> In the above steps, all the "after the previous flush task is
> finished"
> > >> will block the insertion process. One solution is to make all flush
> and
> > >> close task asynchronous. Some questions need to carefully considered:
> > >>
> > >> (1) Many memtables may be flushed concurrently to an unclosed file.
> How
> > >> to guarantee the order of serialization?
> > >> (2) Once a close task is submitted, a new unclosed file will be
> created
> > >> and receives appended data. So there will exists many unclosed files.
> How
> > >> the query and compaction process will be impacted?
> > >>
> > >> Thanks,
> > >>
> > >> Jialin Qiao
> > >> School of Software, Tsinghua University
> > >>
> > >> 乔嘉林
> > >> 清华大学 软件学院
> > >>
> > >> > -----原始邮件-----
> > >> > 发件人: "Xiangdong Huang" <sa...@gmail.com>
> > >> > 发送时间: 2019-06-04 23:08:34 (星期二)
> > >> > 收件人: dev@iotdb.apache.org, "江天" <jt...@163.com>
> > >> > 抄送:
> > >> > 主题: Re: [jira] [Created] (IOTDB-112) Avoid long tail insertion
> which is
> > >> caused by synchronized close-bufferwrite
> > >> >
> > >> > I attached the histogram of the latency in the JIRA.
> > >> >
> > >> > The x-axis is the latency while the y-axis is the cumulative
> > >> distribution.
> > >> > We can see that about 30% insertion can be finished in 20ms, and 60%
> > >> > insertion can be finished in 40ms even though the IoTDB instance is
> > >> serving
> > >> > for a heavy workload... So, eliminating the long tail insertion can
> make
> > >> > the average latency far better.
> > >> >
> > >> > If someone is working on the refactor_overflow or
> refactor_bufferwrite,
> > >> > please pay attention to the code branch for this issue.
> > >> >
> > >> > Best,
> > >> >
> > >> > -----------------------------------
> > >> > Xiangdong Huang
> > >> > School of Software, Tsinghua University
> > >> >
> > >> >  黄向东
> > >> > 清华大学 软件学院
> > >> >
> > >> >
> > >> > xiangdong Huang (JIRA) <ji...@apache.org> 于2019年6月4日周二 下午11:00写道:
> > >> >
> > >> > > xiangdong Huang created IOTDB-112:
> > >> > > -------------------------------------
> > >> > >
> > >> > >              Summary: Avoid long tail insertion which is caused by
> > >> > > synchronized close-bufferwrite
> > >> > >                  Key: IOTDB-112
> > >> > >                  URL:
> https://issues.apache.org/jira/browse/IOTDB-112
> > >> > >              Project: Apache IoTDB
> > >> > >           Issue Type: Improvement
> > >> > >             Reporter: xiangdong Huang
> > >> > >
> > >> > >
> > >> > > In our test, IoTDB has a good insertion performance, and the
> average
> > >> > > latency can be ~200 ms in a given workload and hardware.
> > >> > >
> > >> > > However, when we draw the histogram of the latency, we find that
> 97.5%
> > >> > > latencies are less than 200 ms, while 2.7% latencies are greater.
> The
> > >> > > result shows that there are some long tail latency.
> > >> > >
> > >> > > Then we find that some insertion latencies are about 30 seconds...
> > >> (but
> > >> > > the ratio is less than 0.5%). Indeed, for each connection, a long
> tail
> > >> > > insertion appears per 1 or 2 minutes....
> > >> > >
> > >> > > By reading source codes, I think it is because that in the
> insertion
> > >> > > function,
> > >> > >
> > >> > > `private void insertBufferWrite(FileNodeProcessor
> fileNodeProcessor,
> > >> long
> > >> > > timestamp,
> > >> > >  boolean isMonitor, TSRecord tsRecord, String deviceId)`,
> > >> > >
> > >> > > if the corresponding TsFile is too large, the function is blocked
> > >> until
> > >> > > the memtable is flushed on disk and the TsFile is sealed (we call
> it
> > >> as
> > >> > > closing a TsFile). The latencies of the long tail insertions are
> very
> > >> close
> > >> > > to the time cost of flushing and sealing a TsFile.
> > >> > >
> > >> > > So, if we set the closing function using the async mode, we can
> avoid
> > >> the
> > >> > > long tail insertion.
> > >> > >
> > >> > > However,  there are some side effects we have to fix:
> > >> > >  # At the same time, if a new insertion comes, then a new memtable
> > >> should
> > >> > > be assigned, and a new unsealed TsFile is created;
> > >> > >  # That means that there are more than 1 unsealed TsFiles if the
> > >> system is
> > >> > > crashed before the closing function is finished. So, we have to
> > >> modify the
> > >> > > startup process to recover these files.
> > >> > >
> > >> > > Is there any other side effect that I have to pay attention to?
> > >> > >
> > >> > >
> > >> > >
> > >> > > --
> > >> > > This message was sent by Atlassian JIRA
> > >> > > (v7.6.3#76005)
> > >> > >
> > >>
> > >
>

Re: Re: Avoid long-tail insertion

Posted by Jialin Qiao <qj...@mails.tsinghua.edu.cn>.
Hi Xiangdong,

I will merge this patch. Let "Directories" manage the folders of both sequence and unSequence files is good.

However, the naming of "Directories" is not clear. It would be better to rename to "DirectoryManager"

Best,
--
Jialin Qiao
School of Software, Tsinghua University

乔嘉林
清华大学 软件学院

> -----原始邮件-----
> 发件人: "Xiangdong Huang" <sa...@gmail.com>
> 发送时间: 2019-06-22 16:35:29 (星期六)
> 收件人: dev@iotdb.apache.org
> 抄送: 
> 主题: Re: Avoid long-tail insertion
> 
> Hi jialin,
> 
> I submit some modifications for:
> 
> * add the overflow data folder location setting in the
> iotdb-engine.properties;
> * let Directories.java to manage the above folder.
> 
> If you need to refactor the overflow when you solving the long tail issue,
> you can apply the patch from [1] first to simplify your work.
> 
> [1]
> https://issues.apache.org/jira/secure/attachment/12972547/overflow-folder.patch
> 
> Best,
> -----------------------------------
> Xiangdong Huang
> School of Software, Tsinghua University
> 
>  黄向东
> 清华大学 软件学院
> 
> 
> Xiangdong Huang <sa...@gmail.com> 于2019年6月22日周六 下午3:19写道:
> 
> > If you change the process like this, i.e., there are more than one
> > unsealed TsFiles for each storage group, then  you have to modify the WAL
> > module.. Because current WAL module only recognizes the last unsealed
> > TsFile..
> >
> > By the way, "sealed" is better than "closed", I think..  A sealed file
> > means the file which has the magic string at the head and the tail.
> >
> > Best,
> > -----------------------------------
> > Xiangdong Huang
> > School of Software, Tsinghua University
> >
> >  黄向东
> > 清华大学 软件学院
> >
> >
> > Jialin Qiao <qj...@mails.tsinghua.edu.cn> 于2019年6月22日周六 下午2:54写道:
> >
> >>
> >> Hi, I am solving the long-tail latency problem.
> >>
> >> There are some cases (blocking points) that blocking the insertion. For a
> >> better understanding of this problem, I first introduce the writing process
> >> of IoTDB:
> >>
> >> IoTDB maintains several independent engines (storage group) that supports
> >> read and write. In the following, we focus on one engine. A engine
> >> maintains several closed data files and one unclosed data file that
> >> receives appended data. In memory, there is only one working memtable (m1)
> >> that receives writes. There is also another memtable (m2) that will take
> >> place m1 when m1 is full and being flushed.
> >>
> >> When a data item is inserted:
> >>
> >> (1)We insert it into the working memtable.
> >> (2)We check the size of the memtable. If it reaches a threshold, we
> >> submit a flush task “after the previous flush task is finished” and switch
> >> the two memtables.
> >> (3)We check the size of the unclosed file. If it reaches a threshold, we
> >> close it “after the previous flush task is finished”.
> >>
> >> In the above steps, all the "after the previous flush task is finished"
> >> will block the insertion process. One solution is to make all flush and
> >> close task asynchronous. Some questions need to carefully considered:
> >>
> >> (1) Many memtables may be flushed concurrently to an unclosed file. How
> >> to guarantee the order of serialization?
> >> (2) Once a close task is submitted, a new unclosed file will be created
> >> and receives appended data. So there will exists many unclosed files. How
> >> the query and compaction process will be impacted?
> >>
> >> Thanks,
> >>
> >> Jialin Qiao
> >> School of Software, Tsinghua University
> >>
> >> 乔嘉林
> >> 清华大学 软件学院
> >>
> >> > -----原始邮件-----
> >> > 发件人: "Xiangdong Huang" <sa...@gmail.com>
> >> > 发送时间: 2019-06-04 23:08:34 (星期二)
> >> > 收件人: dev@iotdb.apache.org, "江天" <jt...@163.com>
> >> > 抄送:
> >> > 主题: Re: [jira] [Created] (IOTDB-112) Avoid long tail insertion which is
> >> caused by synchronized close-bufferwrite
> >> >
> >> > I attached the histogram of the latency in the JIRA.
> >> >
> >> > The x-axis is the latency while the y-axis is the cumulative
> >> distribution.
> >> > We can see that about 30% insertion can be finished in 20ms, and 60%
> >> > insertion can be finished in 40ms even though the IoTDB instance is
> >> serving
> >> > for a heavy workload... So, eliminating the long tail insertion can make
> >> > the average latency far better.
> >> >
> >> > If someone is working on the refactor_overflow or refactor_bufferwrite,
> >> > please pay attention to the code branch for this issue.
> >> >
> >> > Best,
> >> >
> >> > -----------------------------------
> >> > Xiangdong Huang
> >> > School of Software, Tsinghua University
> >> >
> >> >  黄向东
> >> > 清华大学 软件学院
> >> >
> >> >
> >> > xiangdong Huang (JIRA) <ji...@apache.org> 于2019年6月4日周二 下午11:00写道:
> >> >
> >> > > xiangdong Huang created IOTDB-112:
> >> > > -------------------------------------
> >> > >
> >> > >              Summary: Avoid long tail insertion which is caused by
> >> > > synchronized close-bufferwrite
> >> > >                  Key: IOTDB-112
> >> > >                  URL: https://issues.apache.org/jira/browse/IOTDB-112
> >> > >              Project: Apache IoTDB
> >> > >           Issue Type: Improvement
> >> > >             Reporter: xiangdong Huang
> >> > >
> >> > >
> >> > > In our test, IoTDB has a good insertion performance, and the average
> >> > > latency can be ~200 ms in a given workload and hardware.
> >> > >
> >> > > However, when we draw the histogram of the latency, we find that 97.5%
> >> > > latencies are less than 200 ms, while 2.7% latencies are greater. The
> >> > > result shows that there are some long tail latency.
> >> > >
> >> > > Then we find that some insertion latencies are about 30 seconds...
> >> (but
> >> > > the ratio is less than 0.5%). Indeed, for each connection, a long tail
> >> > > insertion appears per 1 or 2 minutes....
> >> > >
> >> > > By reading source codes, I think it is because that in the insertion
> >> > > function,
> >> > >
> >> > > `private void insertBufferWrite(FileNodeProcessor fileNodeProcessor,
> >> long
> >> > > timestamp,
> >> > >  boolean isMonitor, TSRecord tsRecord, String deviceId)`,
> >> > >
> >> > > if the corresponding TsFile is too large, the function is blocked
> >> until
> >> > > the memtable is flushed on disk and the TsFile is sealed (we call it
> >> as
> >> > > closing a TsFile). The latencies of the long tail insertions are very
> >> close
> >> > > to the time cost of flushing and sealing a TsFile.
> >> > >
> >> > > So, if we set the closing function using the async mode, we can avoid
> >> the
> >> > > long tail insertion.
> >> > >
> >> > > However,  there are some side effects we have to fix:
> >> > >  # At the same time, if a new insertion comes, then a new memtable
> >> should
> >> > > be assigned, and a new unsealed TsFile is created;
> >> > >  # That means that there are more than 1 unsealed TsFiles if the
> >> system is
> >> > > crashed before the closing function is finished. So, we have to
> >> modify the
> >> > > startup process to recover these files.
> >> > >
> >> > > Is there any other side effect that I have to pay attention to?
> >> > >
> >> > >
> >> > >
> >> > > --
> >> > > This message was sent by Atlassian JIRA
> >> > > (v7.6.3#76005)
> >> > >
> >>
> >

Re: Avoid long-tail insertion

Posted by Xiangdong Huang <sa...@gmail.com>.
Hi jialin,

I submit some modifications for:

* add the overflow data folder location setting in the
iotdb-engine.properties;
* let Directories.java to manage the above folder.

If you need to refactor the overflow when you solving the long tail issue,
you can apply the patch from [1] first to simplify your work.

[1]
https://issues.apache.org/jira/secure/attachment/12972547/overflow-folder.patch

Best,
-----------------------------------
Xiangdong Huang
School of Software, Tsinghua University

 黄向东
清华大学 软件学院


Xiangdong Huang <sa...@gmail.com> 于2019年6月22日周六 下午3:19写道:

> If you change the process like this, i.e., there are more than one
> unsealed TsFiles for each storage group, then  you have to modify the WAL
> module.. Because current WAL module only recognizes the last unsealed
> TsFile..
>
> By the way, "sealed" is better than "closed", I think..  A sealed file
> means the file which has the magic string at the head and the tail.
>
> Best,
> -----------------------------------
> Xiangdong Huang
> School of Software, Tsinghua University
>
>  黄向东
> 清华大学 软件学院
>
>
> Jialin Qiao <qj...@mails.tsinghua.edu.cn> 于2019年6月22日周六 下午2:54写道:
>
>>
>> Hi, I am solving the long-tail latency problem.
>>
>> There are some cases (blocking points) that blocking the insertion. For a
>> better understanding of this problem, I first introduce the writing process
>> of IoTDB:
>>
>> IoTDB maintains several independent engines (storage group) that supports
>> read and write. In the following, we focus on one engine. A engine
>> maintains several closed data files and one unclosed data file that
>> receives appended data. In memory, there is only one working memtable (m1)
>> that receives writes. There is also another memtable (m2) that will take
>> place m1 when m1 is full and being flushed.
>>
>> When a data item is inserted:
>>
>> (1)We insert it into the working memtable.
>> (2)We check the size of the memtable. If it reaches a threshold, we
>> submit a flush task “after the previous flush task is finished” and switch
>> the two memtables.
>> (3)We check the size of the unclosed file. If it reaches a threshold, we
>> close it “after the previous flush task is finished”.
>>
>> In the above steps, all the "after the previous flush task is finished"
>> will block the insertion process. One solution is to make all flush and
>> close task asynchronous. Some questions need to carefully considered:
>>
>> (1) Many memtables may be flushed concurrently to an unclosed file. How
>> to guarantee the order of serialization?
>> (2) Once a close task is submitted, a new unclosed file will be created
>> and receives appended data. So there will exists many unclosed files. How
>> the query and compaction process will be impacted?
>>
>> Thanks,
>>
>> Jialin Qiao
>> School of Software, Tsinghua University
>>
>> 乔嘉林
>> 清华大学 软件学院
>>
>> > -----原始邮件-----
>> > 发件人: "Xiangdong Huang" <sa...@gmail.com>
>> > 发送时间: 2019-06-04 23:08:34 (星期二)
>> > 收件人: dev@iotdb.apache.org, "江天" <jt...@163.com>
>> > 抄送:
>> > 主题: Re: [jira] [Created] (IOTDB-112) Avoid long tail insertion which is
>> caused by synchronized close-bufferwrite
>> >
>> > I attached the histogram of the latency in the JIRA.
>> >
>> > The x-axis is the latency while the y-axis is the cumulative
>> distribution.
>> > We can see that about 30% insertion can be finished in 20ms, and 60%
>> > insertion can be finished in 40ms even though the IoTDB instance is
>> serving
>> > for a heavy workload... So, eliminating the long tail insertion can make
>> > the average latency far better.
>> >
>> > If someone is working on the refactor_overflow or refactor_bufferwrite,
>> > please pay attention to the code branch for this issue.
>> >
>> > Best,
>> >
>> > -----------------------------------
>> > Xiangdong Huang
>> > School of Software, Tsinghua University
>> >
>> >  黄向东
>> > 清华大学 软件学院
>> >
>> >
>> > xiangdong Huang (JIRA) <ji...@apache.org> 于2019年6月4日周二 下午11:00写道:
>> >
>> > > xiangdong Huang created IOTDB-112:
>> > > -------------------------------------
>> > >
>> > >              Summary: Avoid long tail insertion which is caused by
>> > > synchronized close-bufferwrite
>> > >                  Key: IOTDB-112
>> > >                  URL: https://issues.apache.org/jira/browse/IOTDB-112
>> > >              Project: Apache IoTDB
>> > >           Issue Type: Improvement
>> > >             Reporter: xiangdong Huang
>> > >
>> > >
>> > > In our test, IoTDB has a good insertion performance, and the average
>> > > latency can be ~200 ms in a given workload and hardware.
>> > >
>> > > However, when we draw the histogram of the latency, we find that 97.5%
>> > > latencies are less than 200 ms, while 2.7% latencies are greater. The
>> > > result shows that there are some long tail latency.
>> > >
>> > > Then we find that some insertion latencies are about 30 seconds...
>> (but
>> > > the ratio is less than 0.5%). Indeed, for each connection, a long tail
>> > > insertion appears per 1 or 2 minutes....
>> > >
>> > > By reading source codes, I think it is because that in the insertion
>> > > function,
>> > >
>> > > `private void insertBufferWrite(FileNodeProcessor fileNodeProcessor,
>> long
>> > > timestamp,
>> > >  boolean isMonitor, TSRecord tsRecord, String deviceId)`,
>> > >
>> > > if the corresponding TsFile is too large, the function is blocked
>> until
>> > > the memtable is flushed on disk and the TsFile is sealed (we call it
>> as
>> > > closing a TsFile). The latencies of the long tail insertions are very
>> close
>> > > to the time cost of flushing and sealing a TsFile.
>> > >
>> > > So, if we set the closing function using the async mode, we can avoid
>> the
>> > > long tail insertion.
>> > >
>> > > However,  there are some side effects we have to fix:
>> > >  # At the same time, if a new insertion comes, then a new memtable
>> should
>> > > be assigned, and a new unsealed TsFile is created;
>> > >  # That means that there are more than 1 unsealed TsFiles if the
>> system is
>> > > crashed before the closing function is finished. So, we have to
>> modify the
>> > > startup process to recover these files.
>> > >
>> > > Is there any other side effect that I have to pay attention to?
>> > >
>> > >
>> > >
>> > > --
>> > > This message was sent by Atlassian JIRA
>> > > (v7.6.3#76005)
>> > >
>>
>

Re: Avoid long-tail insertion

Posted by Xiangdong Huang <sa...@gmail.com>.
If you change the process like this, i.e., there are more than one unsealed
TsFiles for each storage group, then  you have to modify the WAL module..
Because current WAL module only recognizes the last unsealed TsFile..

By the way, "sealed" is better than "closed", I think..  A sealed file
means the file which has the magic string at the head and the tail.

Best,
-----------------------------------
Xiangdong Huang
School of Software, Tsinghua University

 黄向东
清华大学 软件学院


Jialin Qiao <qj...@mails.tsinghua.edu.cn> 于2019年6月22日周六 下午2:54写道:

>
> Hi, I am solving the long-tail latency problem.
>
> There are some cases (blocking points) that blocking the insertion. For a
> better understanding of this problem, I first introduce the writing process
> of IoTDB:
>
> IoTDB maintains several independent engines (storage group) that supports
> read and write. In the following, we focus on one engine. A engine
> maintains several closed data files and one unclosed data file that
> receives appended data. In memory, there is only one working memtable (m1)
> that receives writes. There is also another memtable (m2) that will take
> place m1 when m1 is full and being flushed.
>
> When a data item is inserted:
>
> (1)We insert it into the working memtable.
> (2)We check the size of the memtable. If it reaches a threshold, we submit
> a flush task “after the previous flush task is finished” and switch the two
> memtables.
> (3)We check the size of the unclosed file. If it reaches a threshold, we
> close it “after the previous flush task is finished”.
>
> In the above steps, all the "after the previous flush task is finished"
> will block the insertion process. One solution is to make all flush and
> close task asynchronous. Some questions need to carefully considered:
>
> (1) Many memtables may be flushed concurrently to an unclosed file. How to
> guarantee the order of serialization?
> (2) Once a close task is submitted, a new unclosed file will be created
> and receives appended data. So there will exists many unclosed files. How
> the query and compaction process will be impacted?
>
> Thanks,
>
> Jialin Qiao
> School of Software, Tsinghua University
>
> 乔嘉林
> 清华大学 软件学院
>
> > -----原始邮件-----
> > 发件人: "Xiangdong Huang" <sa...@gmail.com>
> > 发送时间: 2019-06-04 23:08:34 (星期二)
> > 收件人: dev@iotdb.apache.org, "江天" <jt...@163.com>
> > 抄送:
> > 主题: Re: [jira] [Created] (IOTDB-112) Avoid long tail insertion which is
> caused by synchronized close-bufferwrite
> >
> > I attached the histogram of the latency in the JIRA.
> >
> > The x-axis is the latency while the y-axis is the cumulative
> distribution.
> > We can see that about 30% insertion can be finished in 20ms, and 60%
> > insertion can be finished in 40ms even though the IoTDB instance is
> serving
> > for a heavy workload... So, eliminating the long tail insertion can make
> > the average latency far better.
> >
> > If someone is working on the refactor_overflow or refactor_bufferwrite,
> > please pay attention to the code branch for this issue.
> >
> > Best,
> >
> > -----------------------------------
> > Xiangdong Huang
> > School of Software, Tsinghua University
> >
> >  黄向东
> > 清华大学 软件学院
> >
> >
> > xiangdong Huang (JIRA) <ji...@apache.org> 于2019年6月4日周二 下午11:00写道:
> >
> > > xiangdong Huang created IOTDB-112:
> > > -------------------------------------
> > >
> > >              Summary: Avoid long tail insertion which is caused by
> > > synchronized close-bufferwrite
> > >                  Key: IOTDB-112
> > >                  URL: https://issues.apache.org/jira/browse/IOTDB-112
> > >              Project: Apache IoTDB
> > >           Issue Type: Improvement
> > >             Reporter: xiangdong Huang
> > >
> > >
> > > In our test, IoTDB has a good insertion performance, and the average
> > > latency can be ~200 ms in a given workload and hardware.
> > >
> > > However, when we draw the histogram of the latency, we find that 97.5%
> > > latencies are less than 200 ms, while 2.7% latencies are greater. The
> > > result shows that there are some long tail latency.
> > >
> > > Then we find that some insertion latencies are about 30 seconds... (but
> > > the ratio is less than 0.5%). Indeed, for each connection, a long tail
> > > insertion appears per 1 or 2 minutes....
> > >
> > > By reading source codes, I think it is because that in the insertion
> > > function,
> > >
> > > `private void insertBufferWrite(FileNodeProcessor fileNodeProcessor,
> long
> > > timestamp,
> > >  boolean isMonitor, TSRecord tsRecord, String deviceId)`,
> > >
> > > if the corresponding TsFile is too large, the function is blocked until
> > > the memtable is flushed on disk and the TsFile is sealed (we call it as
> > > closing a TsFile). The latencies of the long tail insertions are very
> close
> > > to the time cost of flushing and sealing a TsFile.
> > >
> > > So, if we set the closing function using the async mode, we can avoid
> the
> > > long tail insertion.
> > >
> > > However,  there are some side effects we have to fix:
> > >  # At the same time, if a new insertion comes, then a new memtable
> should
> > > be assigned, and a new unsealed TsFile is created;
> > >  # That means that there are more than 1 unsealed TsFiles if the
> system is
> > > crashed before the closing function is finished. So, we have to modify
> the
> > > startup process to recover these files.
> > >
> > > Is there any other side effect that I have to pay attention to?
> > >
> > >
> > >
> > > --
> > > This message was sent by Atlassian JIRA
> > > (v7.6.3#76005)
> > >
>

Re: Avoid long-tail insertion

Posted by suyue <su...@mails.tsinghua.edu.cn>.
Hi, I’m modifying the query process of this pr with LeiRui. In previous versions, the system had at most one unsealed tsfile and two overflow files, one for merge another for unordered insert.   (Overflow file and tsfile have the same format , but overflow file isn’t global sorted.) After making all flush and close task asynchronous,  there will exists many unsealed tsfiles and overflow files. The query process is divided into the underlying data reader and the upper processing logic, so I will modify the data reader part to accommodate the new writing process.


> 在 2019年6月22日,下午2:54,Jialin Qiao <qj...@mails.tsinghua.edu.cn> 写道:
> 
> 
> Hi, I am solving the long-tail latency problem. 
> 
> There are some cases (blocking points) that blocking the insertion. For a better understanding of this problem, I first introduce the writing process of IoTDB:
> 
> IoTDB maintains several independent engines (storage group) that supports read and write. In the following, we focus on one engine. A engine maintains several closed data files and one unclosed data file that receives appended data. In memory, there is only one working memtable (m1) that receives writes. There is also another memtable (m2) that will take place m1 when m1 is full and being flushed. 
> 
> When a data item is inserted:
> 
> (1)We insert it into the working memtable.
> (2)We check the size of the memtable. If it reaches a threshold, we submit a flush task “after the previous flush task is finished” and switch the two memtables. 
> (3)We check the size of the unclosed file. If it reaches a threshold, we close it “after the previous flush task is finished”. 
> 
> In the above steps, all the "after the previous flush task is finished" will block the insertion process. One solution is to make all flush and close task asynchronous. Some questions need to carefully considered:
> 
> (1) Many memtables may be flushed concurrently to an unclosed file. How to guarantee the order of serialization?
> (2) Once a close task is submitted, a new unclosed file will be created and receives appended data. So there will exists many unclosed files. How the query and compaction process will be impacted?
> 
> Thanks,
> 
> Jialin Qiao
> School of Software, Tsinghua University
> 
> 乔嘉林
> 清华大学 软件学院
> 
>> -----原始邮件-----
>> 发件人: "Xiangdong Huang" <sa...@gmail.com>
>> 发送时间: 2019-06-04 23:08:34 (星期二)
>> 收件人: dev@iotdb.apache.org, "江天" <jt...@163.com>
>> 抄送: 
>> 主题: Re: [jira] [Created] (IOTDB-112) Avoid long tail insertion which is caused by synchronized close-bufferwrite
>> 
>> I attached the histogram of the latency in the JIRA.
>> 
>> The x-axis is the latency while the y-axis is the cumulative distribution.
>> We can see that about 30% insertion can be finished in 20ms, and 60%
>> insertion can be finished in 40ms even though the IoTDB instance is serving
>> for a heavy workload... So, eliminating the long tail insertion can make
>> the average latency far better.
>> 
>> If someone is working on the refactor_overflow or refactor_bufferwrite,
>> please pay attention to the code branch for this issue.
>> 
>> Best,
>> 
>> -----------------------------------
>> Xiangdong Huang
>> School of Software, Tsinghua University
>> 
>> 黄向东
>> 清华大学 软件学院
>> 
>> 
>> xiangdong Huang (JIRA) <ji...@apache.org> 于2019年6月4日周二 下午11:00写道:
>> 
>>> xiangdong Huang created IOTDB-112:
>>> -------------------------------------
>>> 
>>>             Summary: Avoid long tail insertion which is caused by
>>> synchronized close-bufferwrite
>>>                 Key: IOTDB-112
>>>                 URL: https://issues.apache.org/jira/browse/IOTDB-112
>>>             Project: Apache IoTDB
>>>          Issue Type: Improvement
>>>            Reporter: xiangdong Huang
>>> 
>>> 
>>> In our test, IoTDB has a good insertion performance, and the average
>>> latency can be ~200 ms in a given workload and hardware.
>>> 
>>> However, when we draw the histogram of the latency, we find that 97.5%
>>> latencies are less than 200 ms, while 2.7% latencies are greater. The
>>> result shows that there are some long tail latency.
>>> 
>>> Then we find that some insertion latencies are about 30 seconds... (but
>>> the ratio is less than 0.5%). Indeed, for each connection, a long tail
>>> insertion appears per 1 or 2 minutes....
>>> 
>>> By reading source codes, I think it is because that in the insertion
>>> function,
>>> 
>>> `private void insertBufferWrite(FileNodeProcessor fileNodeProcessor, long
>>> timestamp,
>>> boolean isMonitor, TSRecord tsRecord, String deviceId)`,
>>> 
>>> if the corresponding TsFile is too large, the function is blocked until
>>> the memtable is flushed on disk and the TsFile is sealed (we call it as
>>> closing a TsFile). The latencies of the long tail insertions are very close
>>> to the time cost of flushing and sealing a TsFile.
>>> 
>>> So, if we set the closing function using the async mode, we can avoid the
>>> long tail insertion.
>>> 
>>> However,  there are some side effects we have to fix:
>>> # At the same time, if a new insertion comes, then a new memtable should
>>> be assigned, and a new unsealed TsFile is created;
>>> # That means that there are more than 1 unsealed TsFiles if the system is
>>> crashed before the closing function is finished. So, we have to modify the
>>> startup process to recover these files.
>>> 
>>> Is there any other side effect that I have to pay attention to?
>>> 
>>> 
>>> 
>>> --
>>> This message was sent by Atlassian JIRA
>>> (v7.6.3#76005)
>>> 


Avoid long-tail insertion

Posted by Jialin Qiao <qj...@mails.tsinghua.edu.cn>.
Hi, I am solving the long-tail latency problem. 

There are some cases (blocking points) that blocking the insertion. For a better understanding of this problem, I first introduce the writing process of IoTDB:

IoTDB maintains several independent engines (storage group) that supports read and write. In the following, we focus on one engine. A engine maintains several closed data files and one unclosed data file that receives appended data. In memory, there is only one working memtable (m1) that receives writes. There is also another memtable (m2) that will take place m1 when m1 is full and being flushed. 

When a data item is inserted:

(1)We insert it into the working memtable.
(2)We check the size of the memtable. If it reaches a threshold, we submit a flush task “after the previous flush task is finished” and switch the two memtables. 
(3)We check the size of the unclosed file. If it reaches a threshold, we close it “after the previous flush task is finished”. 

In the above steps, all the "after the previous flush task is finished" will block the insertion process. One solution is to make all flush and close task asynchronous. Some questions need to carefully considered:

(1) Many memtables may be flushed concurrently to an unclosed file. How to guarantee the order of serialization?
(2) Once a close task is submitted, a new unclosed file will be created and receives appended data. So there will exists many unclosed files. How the query and compaction process will be impacted?

Thanks,

Jialin Qiao
School of Software, Tsinghua University

乔嘉林
清华大学 软件学院

> -----原始邮件-----
> 发件人: "Xiangdong Huang" <sa...@gmail.com>
> 发送时间: 2019-06-04 23:08:34 (星期二)
> 收件人: dev@iotdb.apache.org, "江天" <jt...@163.com>
> 抄送: 
> 主题: Re: [jira] [Created] (IOTDB-112) Avoid long tail insertion which is caused by synchronized close-bufferwrite
> 
> I attached the histogram of the latency in the JIRA.
> 
> The x-axis is the latency while the y-axis is the cumulative distribution.
> We can see that about 30% insertion can be finished in 20ms, and 60%
> insertion can be finished in 40ms even though the IoTDB instance is serving
> for a heavy workload... So, eliminating the long tail insertion can make
> the average latency far better.
> 
> If someone is working on the refactor_overflow or refactor_bufferwrite,
> please pay attention to the code branch for this issue.
> 
> Best,
> 
> -----------------------------------
> Xiangdong Huang
> School of Software, Tsinghua University
> 
>  黄向东
> 清华大学 软件学院
> 
> 
> xiangdong Huang (JIRA) <ji...@apache.org> 于2019年6月4日周二 下午11:00写道:
> 
> > xiangdong Huang created IOTDB-112:
> > -------------------------------------
> >
> >              Summary: Avoid long tail insertion which is caused by
> > synchronized close-bufferwrite
> >                  Key: IOTDB-112
> >                  URL: https://issues.apache.org/jira/browse/IOTDB-112
> >              Project: Apache IoTDB
> >           Issue Type: Improvement
> >             Reporter: xiangdong Huang
> >
> >
> > In our test, IoTDB has a good insertion performance, and the average
> > latency can be ~200 ms in a given workload and hardware.
> >
> > However, when we draw the histogram of the latency, we find that 97.5%
> > latencies are less than 200 ms, while 2.7% latencies are greater. The
> > result shows that there are some long tail latency.
> >
> > Then we find that some insertion latencies are about 30 seconds... (but
> > the ratio is less than 0.5%). Indeed, for each connection, a long tail
> > insertion appears per 1 or 2 minutes....
> >
> > By reading source codes, I think it is because that in the insertion
> > function,
> >
> > `private void insertBufferWrite(FileNodeProcessor fileNodeProcessor, long
> > timestamp,
> >  boolean isMonitor, TSRecord tsRecord, String deviceId)`,
> >
> > if the corresponding TsFile is too large, the function is blocked until
> > the memtable is flushed on disk and the TsFile is sealed (we call it as
> > closing a TsFile). The latencies of the long tail insertions are very close
> > to the time cost of flushing and sealing a TsFile.
> >
> > So, if we set the closing function using the async mode, we can avoid the
> > long tail insertion.
> >
> > However,  there are some side effects we have to fix:
> >  # At the same time, if a new insertion comes, then a new memtable should
> > be assigned, and a new unsealed TsFile is created;
> >  # That means that there are more than 1 unsealed TsFiles if the system is
> > crashed before the closing function is finished. So, we have to modify the
> > startup process to recover these files.
> >
> > Is there any other side effect that I have to pay attention to?
> >
> >
> >
> > --
> > This message was sent by Atlassian JIRA
> > (v7.6.3#76005)
> >

Re: [jira] [Created] (IOTDB-112) Avoid long tail insertion which is caused by synchronized close-bufferwrite

Posted by Xiangdong Huang <sa...@gmail.com>.
I attached the histogram of the latency in the JIRA.

The x-axis is the latency while the y-axis is the cumulative distribution.
We can see that about 30% insertion can be finished in 20ms, and 60%
insertion can be finished in 40ms even though the IoTDB instance is serving
for a heavy workload... So, eliminating the long tail insertion can make
the average latency far better.

If someone is working on the refactor_overflow or refactor_bufferwrite,
please pay attention to the code branch for this issue.

Best,

-----------------------------------
Xiangdong Huang
School of Software, Tsinghua University

 黄向东
清华大学 软件学院


xiangdong Huang (JIRA) <ji...@apache.org> 于2019年6月4日周二 下午11:00写道:

> xiangdong Huang created IOTDB-112:
> -------------------------------------
>
>              Summary: Avoid long tail insertion which is caused by
> synchronized close-bufferwrite
>                  Key: IOTDB-112
>                  URL: https://issues.apache.org/jira/browse/IOTDB-112
>              Project: Apache IoTDB
>           Issue Type: Improvement
>             Reporter: xiangdong Huang
>
>
> In our test, IoTDB has a good insertion performance, and the average
> latency can be ~200 ms in a given workload and hardware.
>
> However, when we draw the histogram of the latency, we find that 97.5%
> latencies are less than 200 ms, while 2.7% latencies are greater. The
> result shows that there are some long tail latency.
>
> Then we find that some insertion latencies are about 30 seconds... (but
> the ratio is less than 0.5%). Indeed, for each connection, a long tail
> insertion appears per 1 or 2 minutes....
>
> By reading source codes, I think it is because that in the insertion
> function,
>
> `private void insertBufferWrite(FileNodeProcessor fileNodeProcessor, long
> timestamp,
>  boolean isMonitor, TSRecord tsRecord, String deviceId)`,
>
> if the corresponding TsFile is too large, the function is blocked until
> the memtable is flushed on disk and the TsFile is sealed (we call it as
> closing a TsFile). The latencies of the long tail insertions are very close
> to the time cost of flushing and sealing a TsFile.
>
> So, if we set the closing function using the async mode, we can avoid the
> long tail insertion.
>
> However,  there are some side effects we have to fix:
>  # At the same time, if a new insertion comes, then a new memtable should
> be assigned, and a new unsealed TsFile is created;
>  # That means that there are more than 1 unsealed TsFiles if the system is
> crashed before the closing function is finished. So, we have to modify the
> startup process to recover these files.
>
> Is there any other side effect that I have to pay attention to?
>
>
>
>
>
>
>
>
>
>
>
>
>
>
>
> --
> This message was sent by Atlassian JIRA
> (v7.6.3#76005)
>