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

 黄向东
清华大学 软件学院


李天安 <[email protected]> 于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" <[email protected]>
> > 发送时间: 2019-06-27 11:27:24 (星期四)
> > 收件人: [email protected]
> > 抄送:
> > 主题: 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" <[email protected]>
> > > 发送时间: 2019-06-24 20:24:05 (星期一)
> > > 收件人: [email protected]
> > > 抄送:
> > > 主题: 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" <[email protected]>
> > > > 发送时间: 2019-06-23 10:59:29 (星期日)
> > > > 收件人: [email protected]
> > > > 抄送:
> > > > 主题: 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 <[email protected]> 于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" <[email protected]>
> > > > > > 发送时间: 2019-06-22 16:35:29 (星期六)
> > > > > > 收件人: [email protected]
> > > > > > 抄送:
> > > > > > 主题: 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 <[email protected]> 于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 <[email protected]> 于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" <[email protected]>
> > > > > > >> > 发送时间: 2019-06-04 23:08:34 (星期二)
> > > > > > >> > 收件人: [email protected], "江天" <[email protected]>
> > > > > > >> > 抄送:
> > > > > > >> > 主题: 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) <[email protected]> 于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)
> > > > > > >> > >
> > > > > > >>
> > > > > > >
> > > > >
>

Reply via email to