You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@ignite.apache.org by Michael Fong <mc...@gmail.com> on 2018/10/24 14:32:24 UTC

Problem with reading incomplete payload - IGNITE-7153

Hi, all,


I was trying to fix  IGNITE-7153 which relates to parsing incomplete REDIS
packet larger than 8192 bytes. However, I found a random problem  which is
reproducible on TC as well.
That said, GridNioServerRead.processRead() :
  - int cnt = sockCh.read(readBuf);

sometimes does not read the payload fully as the length field in the header
is larger than the ByteBuffer.limit(). As the result, the
BufferUnderFlowException will be thrown.

For example, in a erroneous round run with my IDE, a REDIS payload (sent by
jedis client) looks like the following:

2a 33 d a 24 33 d a 53 45 54 d a 24 32 d a 62 31 d a 24 {38 31 39 32} d a | 65
d a ...etc

GridRedisProtocolParser.readBulkStr(buf) invokes elCnt(buf) which gets
{8192}. However, the limit of buf is 28 which ends at | position. Obviously,
8192 < limit(), therefore, the logic throws BufferUnderFlow soon after.

I traced back and found  sockCh.read(readBuf) only read 28 bytes for
unknown reason

The problem seems totally random. I wonder if this has anything to do
with other NioWorker
or Selector setting.


Any help would be appreciated!


Regards,


Michael

Re: Problem with reading incomplete payload - IGNITE-7153

Posted by Ilya Kasnacheev <il...@gmail.com>.
Hello!

I don't see why we can't. Just take it over, run tests, move to PA, etc.

Regards,
-- 
Ilya Kasnacheev


чт, 3 янв. 2019 г. в 17:00, Dmitriy Pavlov <dp...@apache.org>:

> Hi Igniters,
>
> I'm trying to reach the author of the fix because the ticket is still in In
> Progress.
>
> Could you please advice me how to handle it (because fix seems to be
> useful)? Can we set Patch Available status by lazy consensus and review
> possibly incomplete fix?
> https://issues.apache.org/jira/browse/IGNITE-7153
>
> Sincerely,
> Dmitriy Pavlov
>
> пт, 2 нояб. 2018 г. в 13:20, Michael Fong <mc...@gmail.com>:
>
> > Hi Yakov,
> >
> > Thanks so much for your analysis.
> >
> > Parser expects chunks to be complete and has all the data to read entire
> > > message, but this is not guaranteed and single message can arrive in
> > > several chunks.
> >
> > This is indeed the the assumption to my implementation. I have not come
> up
> > a another parsing algorithm to handle this rainy day case. Perhaps, it
> > would require more refactoring on existing code. In addition, I might
> need
> > to check how Redis dev implements the parser state machine.
> >
> > I would be interested to see how current implementation (based on
> > 2.6/master) behaves if we intentionally split the message into chunks as
> > you suggested for the reproducer.
> >
> > Regards,
> >
> > Michael
> >
> > On Wed, Oct 31, 2018 at 7:08 PM Yakov Zhdanov <yz...@apache.org>
> wrote:
> >
> > > Hi Mike!
> > >
> > > Thanks for reproducer. Now I understand the problem. NIO worker reads
> > > chunks from the network and notifies the parser on data read. Parser
> > > expects chunks to be complete and has all the data to read entire
> > message,
> > > but this is not guaranteed and single message can arrive in several
> > chunks.
> > > Which is demostrated by your test.
> > >
> > > The problem is inside GridRedisProtocolParser. We should add ability to
> > > store the parsing context if we do not have all the data to complete
> > > message parsing, as it is done, for example in GridBufferedParser. So,
> it
> > > is definitely an issue and should be fixed by adding parsing state. I
> see
> > > you attempted to do so in PR
> > > https://github.com/apache/ignite/pull/5044/files. I did not do a
> formal
> > > review, so let's ask community to review your patch.
> > >
> > > Couple of comments about your reproducer.
> > >
> > > 1. Let's dump a proper Redis message bytes sent by Jedis.
> > > 2. Let's split this dump into 5 chunks and send them with 100 ms
> delays.
> > >
> > > This should fail before fix is applied, and should pass with proper
> > message
> > > parsed after we have the issue fixed.
> > >
> > > Thanks!
> > >
> > > --Yakov
> > >
> >
>

Re: Problem with reading incomplete payload - IGNITE-7153

Posted by Dmitriy Pavlov <dp...@apache.org>.
Hi Igniters,

I'm trying to reach the author of the fix because the ticket is still in In
Progress.

Could you please advice me how to handle it (because fix seems to be
useful)? Can we set Patch Available status by lazy consensus and review
possibly incomplete fix?
https://issues.apache.org/jira/browse/IGNITE-7153

Sincerely,
Dmitriy Pavlov

пт, 2 нояб. 2018 г. в 13:20, Michael Fong <mc...@gmail.com>:

> Hi Yakov,
>
> Thanks so much for your analysis.
>
> Parser expects chunks to be complete and has all the data to read entire
> > message, but this is not guaranteed and single message can arrive in
> > several chunks.
>
> This is indeed the the assumption to my implementation. I have not come up
> a another parsing algorithm to handle this rainy day case. Perhaps, it
> would require more refactoring on existing code. In addition, I might need
> to check how Redis dev implements the parser state machine.
>
> I would be interested to see how current implementation (based on
> 2.6/master) behaves if we intentionally split the message into chunks as
> you suggested for the reproducer.
>
> Regards,
>
> Michael
>
> On Wed, Oct 31, 2018 at 7:08 PM Yakov Zhdanov <yz...@apache.org> wrote:
>
> > Hi Mike!
> >
> > Thanks for reproducer. Now I understand the problem. NIO worker reads
> > chunks from the network and notifies the parser on data read. Parser
> > expects chunks to be complete and has all the data to read entire
> message,
> > but this is not guaranteed and single message can arrive in several
> chunks.
> > Which is demostrated by your test.
> >
> > The problem is inside GridRedisProtocolParser. We should add ability to
> > store the parsing context if we do not have all the data to complete
> > message parsing, as it is done, for example in GridBufferedParser. So, it
> > is definitely an issue and should be fixed by adding parsing state. I see
> > you attempted to do so in PR
> > https://github.com/apache/ignite/pull/5044/files. I did not do a formal
> > review, so let's ask community to review your patch.
> >
> > Couple of comments about your reproducer.
> >
> > 1. Let's dump a proper Redis message bytes sent by Jedis.
> > 2. Let's split this dump into 5 chunks and send them with 100 ms delays.
> >
> > This should fail before fix is applied, and should pass with proper
> message
> > parsed after we have the issue fixed.
> >
> > Thanks!
> >
> > --Yakov
> >
>

Re: Problem with reading incomplete payload - IGNITE-7153

Posted by Michael Fong <mc...@gmail.com>.
Hi Yakov,

Thanks so much for your analysis.

Parser expects chunks to be complete and has all the data to read entire
> message, but this is not guaranteed and single message can arrive in
> several chunks.

This is indeed the the assumption to my implementation. I have not come up
a another parsing algorithm to handle this rainy day case. Perhaps, it
would require more refactoring on existing code. In addition, I might need
to check how Redis dev implements the parser state machine.

I would be interested to see how current implementation (based on
2.6/master) behaves if we intentionally split the message into chunks as
you suggested for the reproducer.

Regards,

Michael

On Wed, Oct 31, 2018 at 7:08 PM Yakov Zhdanov <yz...@apache.org> wrote:

> Hi Mike!
>
> Thanks for reproducer. Now I understand the problem. NIO worker reads
> chunks from the network and notifies the parser on data read. Parser
> expects chunks to be complete and has all the data to read entire message,
> but this is not guaranteed and single message can arrive in several chunks.
> Which is demostrated by your test.
>
> The problem is inside GridRedisProtocolParser. We should add ability to
> store the parsing context if we do not have all the data to complete
> message parsing, as it is done, for example in GridBufferedParser. So, it
> is definitely an issue and should be fixed by adding parsing state. I see
> you attempted to do so in PR
> https://github.com/apache/ignite/pull/5044/files. I did not do a formal
> review, so let's ask community to review your patch.
>
> Couple of comments about your reproducer.
>
> 1. Let's dump a proper Redis message bytes sent by Jedis.
> 2. Let's split this dump into 5 chunks and send them with 100 ms delays.
>
> This should fail before fix is applied, and should pass with proper message
> parsed after we have the issue fixed.
>
> Thanks!
>
> --Yakov
>

Re: Problem with reading incomplete payload - IGNITE-7153

Posted by Yakov Zhdanov <yz...@apache.org>.
Hi Mike!

Thanks for reproducer. Now I understand the problem. NIO worker reads
chunks from the network and notifies the parser on data read. Parser
expects chunks to be complete and has all the data to read entire message,
but this is not guaranteed and single message can arrive in several chunks.
Which is demostrated by your test.

The problem is inside GridRedisProtocolParser. We should add ability to
store the parsing context if we do not have all the data to complete
message parsing, as it is done, for example in GridBufferedParser. So, it
is definitely an issue and should be fixed by adding parsing state. I see
you attempted to do so in PR
https://github.com/apache/ignite/pull/5044/files. I did not do a formal
review, so let's ask community to review your patch.

Couple of comments about your reproducer.

1. Let's dump a proper Redis message bytes sent by Jedis.
2. Let's split this dump into 5 chunks and send them with 100 ms delays.

This should fail before fix is applied, and should pass with proper message
parsed after we have the issue fixed.

Thanks!

--Yakov

Re: Problem with reading incomplete payload - IGNITE-7153

Posted by Michael Fong <mc...@gmail.com>.
Hi,


I have successfully emulated a the packets to Ignite via pure Socket
connection per your request. Please let me know if this could help locate
the root cause of the problem. Thank you. Regards.


public void testSendingRawRedisSocket() throws Exception  {

    Socket socket = new Socket();
    socket = new Socket();

    socket.setReuseAddress(true);
    socket.setKeepAlive(true); // Will monitor the TCP connection is
    socket.setTcpNoDelay(true); // Socket buffer Whetherclosed, to
    // ensure timely delivery of data
    socket.setSoLinger(true, 0); // Control calls close () method,

    socket.connect(new InetSocketAddress("127.0.0.1", 6379), 2000);
    OutputStream outputStream = socket.getOutputStream();

    //In the content of ByteBuffer, GridNioServer.processRead() has
only read() partial data                                 ^- where
ByteBuffer index ends
    //2a 31 d a 24 34 d a 50 49 4e 47 d a 2a 33  d  a 24 33  d  a 53
45 54  d  a 24 32  d  a 62 31 d  a  24 38 31 39 32  d  a|0 0 0 0 0 0 0
0 0 0 0 0 0 ...........

    byte[] errorRawPayload = new byte[]{0x2a, 0x31, 0xd, 0xa, 0x24,
0x34, 0xd, 0xa, 0x50, 0x49, 0x4e, 0x47, 0xd, 0xa, 0x2a, 0x33, 0xd,
0xa, 0x24, 0x33, 0xd, 0xa, 0x53, 0x45, 0x54, 0xd, 0xa, 0x24, 0x32,
0xd, 0xa, 0x62, 0x31, 0xd, 0xa, 0x24, 0x38, 0x31, 0x39, 0x32, 0xd,
0xa};
    outputStream.write(errorRawPayload);

    /*
     * Payload from successful run:
     * //2a 33  d  a 24 33  d  a 53 45 54  d  a 24 32  d  a 62 31 d
a 24 38 31 39 32  d  a 6e 4e 32 58 54 6c 50 79 30 6f 61 78 64 49 52 68
77 71 4a 73 30 34 76 4f 46 < omitted for another 8000 bytes....>
     *  //78 63 30 73 51 4f 6c 6e 56 51 71 34 6d 6d 74 66 75 50 6a 50
78 4c 63 77 4e 46 43 39  d  a|
     */


    Thread.sleep(100);
    socket.close();
}


On Wed, Oct 31, 2018 at 1:06 PM Michael Fong <mc...@gmail.com> wrote:

> Hi Yakov,
>
> Thanks for your reply. :)
>
> Please allow me to explain the whole situation and how to reproduce
> the problem:
>
> In the PR of IGNITE-7153, I added a test case under
> RedisProtocolConnectSelfTest, sending some binary data via Jedis client,
> large enough (> 8192 bytes) to reproduce the original problem of
> IGNITE-7153 from my understanding.
> Later on, I found another problem which I explain in the previous email
> that payload is sometimes not fully read via
> GridNioServer.processRead(). This, however, does not seem to be a Redis
> protocol parsing problem. Unfortunately, the only way that I found to
> reproduce this NioServer packet-reading problem is to add a for-loop in the
> new test case. I would say 95% of time, it will occur, i.e.
>
> for(int i = 0; i < 100; i ++) {
>
>      String randomString1 = RandomStringUtils.random(8 << 10, true, true); //8192
>
>      try (Jedis jedis = pool.getResource()) {
>           jedis.set("b1".getBytes(), randomString1.getBytes());
>           Assert.assertEquals(randomString1, jedis.get("b1"));
>
>           Thread.sleep(100);
>      }
>
> }
>
>
> For troubleshooting, I simply print out the content of ByteBuffer along
> the data flow, and that is how I got packet in the previous email. I could
> try to emulate the erroneous packet then add the added in another test case
> via pure socket connection per your request. Thanks!
>
> Regards,
>
> Michael
>
>
> On Tue, Oct 30, 2018 at 6:00 PM Yakov Zhdanov <yz...@apache.org> wrote:
>
>> Michael, can you please share a reproducer? Is it possible to snapshot a
>> packet that causes the error and just emulate packet send with manually
>> opened socket bypassing Redis client lib?
>>
>> --Yakov
>>
>

Re: Problem with reading incomplete payload - IGNITE-7153

Posted by Michael Fong <mc...@gmail.com>.
Hi Yakov,

Thanks for your reply. :)

Please allow me to explain the whole situation and how to reproduce
the problem:

In the PR of IGNITE-7153, I added a test case under
RedisProtocolConnectSelfTest, sending some binary data via Jedis client,
large enough (> 8192 bytes) to reproduce the original problem of
IGNITE-7153 from my understanding.
Later on, I found another problem which I explain in the previous email
that payload is sometimes not fully read via
GridNioServer.processRead(). This,
however, does not seem to be a Redis protocol parsing problem.
Unfortunately, the only way that I found to reproduce this NioServer
packet-reading problem is to add a for-loop in the new test case. I would
say 95% of time, it will occur, i.e.

for(int i = 0; i < 100; i ++) {

     String randomString1 = RandomStringUtils.random(8 << 10, true,
true); //8192

     try (Jedis jedis = pool.getResource()) {
          jedis.set("b1".getBytes(), randomString1.getBytes());
          Assert.assertEquals(randomString1, jedis.get("b1"));

          Thread.sleep(100);
     }

}


For troubleshooting, I simply print out the content of ByteBuffer along the
data flow, and that is how I got packet in the previous email. I could try
to emulate the erroneous packet then add the added in another test case via
pure socket connection per your request. Thanks!

Regards,

Michael


On Tue, Oct 30, 2018 at 6:00 PM Yakov Zhdanov <yz...@apache.org> wrote:

> Michael, can you please share a reproducer? Is it possible to snapshot a
> packet that causes the error and just emulate packet send with manually
> opened socket bypassing Redis client lib?
>
> --Yakov
>

Re: Problem with reading incomplete payload - IGNITE-7153

Posted by Yakov Zhdanov <yz...@apache.org>.
Michael, can you please share a reproducer? Is it possible to snapshot a
packet that causes the error and just emulate packet send with manually
opened socket bypassing Redis client lib?

--Yakov

Re: Problem with reading incomplete payload - IGNITE-7153

Posted by Michael Fong <mc...@gmail.com>.
bump :)

On Fri, Oct 26, 2018 at 4:00 PM Michael Fong <mc...@gmail.com> wrote:

> Hi,
>
> Thanks for your reply. I think current implementation (based on R2.6) does
> exactly what you mentioned as of in GridNioServer
> <https://github.com/apache/ignite/blob/ignite-2.6/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java#L1087>.
> The problem is that in redis protocol defines message length right before
> the message context. If a huge message comes in and unfortunately Selector
> / NioServer has not read the whole payload fully. Furthermore, the
> incomplete message is passed to Redis packet parser, then logic error would
> occur - in this case BufferUnderFlowError. I wonder if this is expected
> behavior of how Ignite receive data from server perspective?
>
> Regards,
>
> Michael
>
> On Fri, Oct 26, 2018 at 1:13 PM Jörn Franke <jo...@gmail.com> wrote:
>
>> I have not checked the exact routine, but you need to reexecute read
>> until it returns -1 or if known that all bytes that are expected have been
>> read (if sockCh is based on some kind of input stream)
>>
>> > Am 26.10.2018 um 04:24 schrieb Michael Fong <mc...@gmail.com>:
>> >
>> > Bump!
>> >
>> > Anyone who could help me solve this random issue? Thanks!
>> >
>> > Regards,
>> >
>> >
>> > Michael
>> >
>> >> On Wed, Oct 24, 2018 at 10:32 PM Michael Fong <mc...@gmail.com>
>> wrote:
>> >>
>> >> Hi, all,
>> >>
>> >>
>> >> I was trying to fix  IGNITE-7153 which relates to parsing incomplete
>> REDIS
>> >> packet larger than 8192 bytes. However, I found a random problem
>> which is
>> >> reproducible on TC as well.
>> >> That said, GridNioServerRead.processRead() :
>> >>  - int cnt = sockCh.read(readBuf);
>> >>
>> >> sometimes does not read the payload fully as the length field in the
>> >> header is larger than the ByteBuffer.limit(). As the result, the
>> >> BufferUnderFlowException will be thrown.
>> >>
>> >> For example, in a erroneous round run with my IDE, a REDIS payload
>> (sent
>> >> by jedis client) looks like the following:
>> >>
>> >> 2a 33 d a 24 33 d a 53 45 54 d a 24 32 d a 62 31 d a 24 {38 31 39 32}
>> d a | 65
>> >> d a ...etc
>> >>
>> >> GridRedisProtocolParser.readBulkStr(buf) invokes elCnt(buf) which gets
>> >> {8192}. However, the limit of buf is 28 which ends at | position.
>> Obviously,
>> >> 8192 < limit(), therefore, the logic throws BufferUnderFlow soon after.
>> >>
>> >> I traced back and found  sockCh.read(readBuf) only read 28 bytes for
>> unknown reason
>> >>
>> >> The problem seems totally random. I wonder if this has anything to do
>> with other NioWorker
>> >> or Selector setting.
>> >>
>> >>
>> >> Any help would be appreciated!
>> >>
>> >>
>> >> Regards,
>> >>
>> >>
>> >> Michael
>> >>
>> >>
>>
>

Re: Problem with reading incomplete payload - IGNITE-7153

Posted by Michael Fong <mc...@gmail.com>.
Hi,

Thanks for your reply. I think current implementation (based on R2.6) does
exactly what you mentioned as of in GridNioServer
<https://github.com/apache/ignite/blob/ignite-2.6/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java#L1087>.
The problem is that in redis protocol defines message length right before
the message context. If a huge message comes in and unfortunately Selector
/ NioServer has not read the whole payload fully. Furthermore, the
incomplete message is passed to Redis packet parser, then logic error would
occur - in this case BufferUnderFlowError. I wonder if this is expected
behavior of how Ignite receive data from server perspective?

Regards,

Michael

On Fri, Oct 26, 2018 at 1:13 PM Jörn Franke <jo...@gmail.com> wrote:

> I have not checked the exact routine, but you need to reexecute read until
> it returns -1 or if known that all bytes that are expected have been read
> (if sockCh is based on some kind of input stream)
>
> > Am 26.10.2018 um 04:24 schrieb Michael Fong <mc...@gmail.com>:
> >
> > Bump!
> >
> > Anyone who could help me solve this random issue? Thanks!
> >
> > Regards,
> >
> >
> > Michael
> >
> >> On Wed, Oct 24, 2018 at 10:32 PM Michael Fong <mc...@gmail.com>
> wrote:
> >>
> >> Hi, all,
> >>
> >>
> >> I was trying to fix  IGNITE-7153 which relates to parsing incomplete
> REDIS
> >> packet larger than 8192 bytes. However, I found a random problem  which
> is
> >> reproducible on TC as well.
> >> That said, GridNioServerRead.processRead() :
> >>  - int cnt = sockCh.read(readBuf);
> >>
> >> sometimes does not read the payload fully as the length field in the
> >> header is larger than the ByteBuffer.limit(). As the result, the
> >> BufferUnderFlowException will be thrown.
> >>
> >> For example, in a erroneous round run with my IDE, a REDIS payload (sent
> >> by jedis client) looks like the following:
> >>
> >> 2a 33 d a 24 33 d a 53 45 54 d a 24 32 d a 62 31 d a 24 {38 31 39 32} d
> a | 65
> >> d a ...etc
> >>
> >> GridRedisProtocolParser.readBulkStr(buf) invokes elCnt(buf) which gets
> >> {8192}. However, the limit of buf is 28 which ends at | position.
> Obviously,
> >> 8192 < limit(), therefore, the logic throws BufferUnderFlow soon after.
> >>
> >> I traced back and found  sockCh.read(readBuf) only read 28 bytes for
> unknown reason
> >>
> >> The problem seems totally random. I wonder if this has anything to do
> with other NioWorker
> >> or Selector setting.
> >>
> >>
> >> Any help would be appreciated!
> >>
> >>
> >> Regards,
> >>
> >>
> >> Michael
> >>
> >>
>

Re: Problem with reading incomplete payload - IGNITE-7153

Posted by Jörn Franke <jo...@gmail.com>.
I have not checked the exact routine, but you need to reexecute read until it returns -1 or if known that all bytes that are expected have been read (if sockCh is based on some kind of input stream)

> Am 26.10.2018 um 04:24 schrieb Michael Fong <mc...@gmail.com>:
> 
> Bump!
> 
> Anyone who could help me solve this random issue? Thanks!
> 
> Regards,
> 
> 
> Michael
> 
>> On Wed, Oct 24, 2018 at 10:32 PM Michael Fong <mc...@gmail.com> wrote:
>> 
>> Hi, all,
>> 
>> 
>> I was trying to fix  IGNITE-7153 which relates to parsing incomplete REDIS
>> packet larger than 8192 bytes. However, I found a random problem  which is
>> reproducible on TC as well.
>> That said, GridNioServerRead.processRead() :
>>  - int cnt = sockCh.read(readBuf);
>> 
>> sometimes does not read the payload fully as the length field in the
>> header is larger than the ByteBuffer.limit(). As the result, the
>> BufferUnderFlowException will be thrown.
>> 
>> For example, in a erroneous round run with my IDE, a REDIS payload (sent
>> by jedis client) looks like the following:
>> 
>> 2a 33 d a 24 33 d a 53 45 54 d a 24 32 d a 62 31 d a 24 {38 31 39 32} d a | 65
>> d a ...etc
>> 
>> GridRedisProtocolParser.readBulkStr(buf) invokes elCnt(buf) which gets
>> {8192}. However, the limit of buf is 28 which ends at | position. Obviously,
>> 8192 < limit(), therefore, the logic throws BufferUnderFlow soon after.
>> 
>> I traced back and found  sockCh.read(readBuf) only read 28 bytes for unknown reason
>> 
>> The problem seems totally random. I wonder if this has anything to do with other NioWorker
>> or Selector setting.
>> 
>> 
>> Any help would be appreciated!
>> 
>> 
>> Regards,
>> 
>> 
>> Michael
>> 
>> 

Re: Problem with reading incomplete payload - IGNITE-7153

Posted by Michael Fong <mc...@gmail.com>.
Bump!

Anyone who could help me solve this random issue? Thanks!

Regards,


Michael

On Wed, Oct 24, 2018 at 10:32 PM Michael Fong <mc...@gmail.com> wrote:

> Hi, all,
>
>
> I was trying to fix  IGNITE-7153 which relates to parsing incomplete REDIS
> packet larger than 8192 bytes. However, I found a random problem  which is
> reproducible on TC as well.
> That said, GridNioServerRead.processRead() :
>   - int cnt = sockCh.read(readBuf);
>
> sometimes does not read the payload fully as the length field in the
> header is larger than the ByteBuffer.limit(). As the result, the
> BufferUnderFlowException will be thrown.
>
> For example, in a erroneous round run with my IDE, a REDIS payload (sent
> by jedis client) looks like the following:
>
> 2a 33 d a 24 33 d a 53 45 54 d a 24 32 d a 62 31 d a 24 {38 31 39 32} d a | 65
> d a ...etc
>
> GridRedisProtocolParser.readBulkStr(buf) invokes elCnt(buf) which gets
> {8192}. However, the limit of buf is 28 which ends at | position. Obviously,
> 8192 < limit(), therefore, the logic throws BufferUnderFlow soon after.
>
> I traced back and found  sockCh.read(readBuf) only read 28 bytes for unknown reason
>
> The problem seems totally random. I wonder if this has anything to do with other NioWorker
> or Selector setting.
>
>
> Any help would be appreciated!
>
>
> Regards,
>
>
> Michael
>
>