On Mon, Dec 30, 2013 at 11:16 PM, Rakesh R <[email protected]> wrote:

> I've tried one simple test case:
>
> Just before connecting 'bootstrap.connect(addr)', I have killed the
> Bookieserver. What I have observed is immediately returning the call with
> failure.
>

Isn't that expected? as there is not server listened on given port. this is
how TCP works, no?


>
> Any thoughts?
>
> -----Original Message-----
> From: Sijie Guo [mailto:[email protected]]
> Sent: 31 December 2013 12:33
> To: [email protected]
> Subject: Re: Build failed in Jenkins: bookkeeper-trunk #489
>
> On Mon, Dec 30, 2013 at 10:45 PM, Rakesh R <[email protected]> wrote:
>
> > Hi Sijie,
> >
> > But I didn't understand why the connection failure is immediately
> > sending without waiting for the timeout.
>
> In general, client should wait for the connection timeout(10secs) and
> > internally retries before throwing failure message. Am I correct?
> >
>
> No idea. from the log, there is less information to tell what was going on
> at that time. I think the better solution is to add  logs about the failure
> so we could catch the details of what's wrong inside.
>
>
> >
> > Do we need to have an explicit retry mechanism in netty?
> >
>
> I don't think we need retry connect in netty. As 1) we already have retry
> mechanism in bookie client ; 2) if connect failed on any bookie, we should
> let netty notify bookkeeper immediately. as connect failure means bookie
> down in most of case, we should change bookie immediately to avoid high
> latency.
>
> - Sijie
>
>
> >
> > -Rakesh
> > -----Original Message-----
> > From: Sijie Guo [mailto:[email protected]]
> > Sent: 31 December 2013 11:59
> > To: [email protected]
> > Subject: Re: Build failed in Jenkins: bookkeeper-trunk #489
> >
> > I don't think its connect timeout setting issue. as by default, netty
> > channel connect timeout is 10 sec (
> > https://github.com/netty/netty/blob/3.2/src/main/java/org/jboss/netty/
> > channel/DefaultChannelConfig.java#L38
> > ).
> > If you checked the log, the log statements show that the connect
> > operation is in same second.
> >
> > 2013-12-30 12:29:36,731 - INFO  -
> > [BookKeeperClientWorker-0-0:PerChannelBookieClient@167] - Connecting
> > to
> > bookie: /67.195.138.30:15039
> > 2013-12-30 12:29:36,732 - ERROR - [New I/O client boss
> > #5-1:PerChannelBookieClient$1@203] - Could not connect to bookie: [id:
> > 0x019a639b, /229.27.250.246:46509 :> /67.195.138.30:15039], current
> > state CONNECTING
> >
> >
> >
> >
> > On Mon, Dec 30, 2013 at 9:31 PM, Rakesh R <[email protected]> wrote:
> >
> > > Hi Flavio,
> > >
> > > As test case name says, it is testing multiple bookie failures.
> > >
> > > On bookiefailure, when doing the ensemble reformation, unfortunately
> > > it is failing to connect to the Bookie-15039. But it should suppose
> > > to get connected and continue write operation. This is the reason
> > > for the test case failure. Please see the following log pattern:
> > >
> > > 2013-12-30 12:29:36,731 - INFO  -
> > > [BookKeeperClientWorker-0-0:PerChannelBookieClient@167] - Connecting
> > > to
> > > bookie: /67.195.138.30:15039
> > > 2013-12-30 12:29:36,732 - ERROR - [New I/O client boss
> > > #5-1:PerChannelBookieClient$1@203] - Could not connect to bookie: [id:
> > > 0x019a639b, /229.27.250.246:46509 :> /67.195.138.30:15039], current
> > > state CONNECTING
> > > 2013-12-30 12:29:36,732 - WARN  -
> > > [BookKeeperClientWorker-0-0:PendingAddOp@158] - Write did not succeed:
> > > L0
> > > E100 on /67.195.138.30:15039
> > > 2013-12-30 12:29:36,733 - INFO  -
> > > [BookKeeperClientWorker-0-0:LedgerHandle@659] - Handling failure of
> > > bookie: /67.195.138.30:15039 index: 2
> > > 2013-12-30 12:29:36,733 - WARN  -
> > > [BookKeeperClientWorker-0-0:RackawareEnsemblePlacementPolicy@491] -
> > > Failed to choose a bookie from /default-rack : excluded [<Bookie:
> > > 67.195.138.30:15036>, <Bookie:67.195.138.30:15038>, <Bookie:
> > > 67.195.138.30:15039>, <Bookie:67.195.138.30:15040>, <Bookie:
> > > 67.195.138.30:15035>], fallback to choose bookie randomly from the
> > > cluster.
> > >
> > >
> > > I'm thinking, there could be chance of small network fluctuations or
> > > slow machine and resulting in connection failure.
> > > To handle this IMHO, we should have netty client connection timeout
> > > in place and should retry for few intervals. Let me do a try with
> > > bootstrap.setOption("connectTimeoutMillis", timeoutvalue); Shall I
> > > raise a JIRA to discuss about these concerns and will reach to a
> > > conclusion. Whats your opinion?
> > >
> > > -Rakesh
> > >
> > > -----Original Message-----
> > > From: Flavio Junqueira [mailto:[email protected]]
> > > Sent: 31 December 2013 01:51
> > > To: [email protected]
> > > Subject: Fwd: Build failed in Jenkins: bookkeeper-trunk #489
> > >
> > > I was wondering if there is a jira open for the test that failed
> > > below, does anyone know?
> > >
> > > -Flavio
> > >
> > > Begin forwarded message:
> > >
> > > > Tests in error:
> > > >
> > >
> > > testWithMultipleBookieFailuresInLastEnsemble[2](org.apache.bookkeeper.
> > > client.BookieWriteLedgerTest)
> > >
> > >
> >
>

Reply via email to