[
https://issues.apache.org/jira/browse/HDFS-15340?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17106014#comment-17106014
]
Jinglun commented on HDFS-15340:
--------------------------------
Hi [~linyiqun], thanks your great comments ! Upload v05.
{quote}How about adding sleep interval time when curProcedure.execute returning
false that means curProcedure executed failed? It can avoid frequently
executing failed procedure.
The balancer job is same, why we always write its journal info to HDFS? Only
once time is enough I think.
{quote}
One procedure may have many phases and all the phases share the same member
variables. Each time execute() returns, the journal is saved. User should
serialize the current phase in write(DataOutput) so the job can continue with
the last unfinished phase after it is recovered. The return value indicates
whether the job should go to the next procedure. Return true after all the
phases finish.
Take DistCpProcedure for example, it actually has 5 phases. Each phase needs
to be written out to journal so it can be recovered correctly. It returns true
after all the phases finish.
If the procedure needs a retry then it should throw a
BalanceProcedure.RetryException. The job would be added to the delay queue. We
shouldn't sleep in the job or the procedure because it will block the worker
thread and affect other pending jobs.
{quote}lastProcedure here is only used for testing, I suggest to remove this as
an input parameter. It seems too confused that we pass lastProcedure but do
nothing in actual BalanceProcedure class. The major function methods need be
clear for others to understand, .
{quote}
The reason passing the lastProcedure is having a context between upstream and
downstream. For example supposing we have a topology below, A might go to
either B1 or B2. With the last procedure C can have different behaviors. Let
me know your thoughts.
I'm also ok to remove it, in v05 I removed the lastProcedure.
{quote}Job:
Procedure A(start) ---> Procedure B1 ---> Procedre C --> end
---> Procedure B2 --->
{quote}
> RBF: Implement BalanceProcedureScheduler basic framework
> --------------------------------------------------------
>
> Key: HDFS-15340
> URL: https://issues.apache.org/jira/browse/HDFS-15340
> Project: Hadoop HDFS
> Issue Type: Sub-task
> Reporter: Jinglun
> Assignee: Jinglun
> Priority: Major
> Attachments: HDFS-15340.001.patch, HDFS-15340.002.patch,
> HDFS-15340.003.patch, HDFS-15340.004.patch
>
>
> Patch in HDFS-15294 is too big to review so we split it into 2 patches. This
> is the first one. Detail can be found at HDFS-15294.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]