Skip to content

Conversation

@sijie
Copy link
Member

@sijie sijie commented Sep 6, 2019

Descriptions of the changes in this PR:

Motivation

We introduced a smart buffer allocator since 4.9.0. It falls back to allocate buffer
on heap when running out direct memory so that the bookie can run in a degraded mode
rather than crashes.

However if we use directBuffer directly, it bypasses the fallback mechanism. So
the bookie crashes when running out of direct memory.

Modifications

Use buffer instead of directBuffer and let the allocator decide what is the best
to use.


In order to uphold a high standard for quality for code contributions, Apache BookKeeper runs various precommit
checks for pull requests. A pull request can only be merged when it passes precommit checks. However running all
the precommit checks can take a long time, some trivial changes don't need to run all the precommit checks. You
can check following list to skip the tests that don't need to run for your pull request. Leave them unchecked if
you are not sure, committers will help you:

  • [skip bookkeeper-server bookie tests]: skip testing org.apache.bookkeeper.bookie in bookkeeper-server module.
  • [skip bookkeeper-server client tests]: skip testing org.apache.bookkeeper.client in bookkeeper-server module.
  • [skip bookkeeper-server replication tests]: skip testing org.apache.bookkeeper.replication in bookkeeper-server module.
  • [skip bookkeeper-server tls tests]: skip testing org.apache.bookkeeper.tls in bookkeeper-server module.
  • [skip bookkeeper-server remaining tests]: skip testing all other tests in bookkeeper-server module.
  • [skip integration tests]: skip docker based integration tests. if you make java code changes, you shouldn't skip integration tests.
  • [skip build java8]: skip build on java8. ONLY skip this when ONLY changing files under documentation under site.
  • [skip build java11]: skip build on java11. ONLY skip this when ONLY changing files under documentation under site.


Be sure to do all of the following to help us incorporate your contribution
quickly and easily:

If this PR is a BookKeeper Proposal (BP):

  • Make sure the PR title is formatted like:
    <BP-#>: Description of bookkeeper proposal
    e.g. BP-1: 64 bits ledger is support
  • Attach the master issue link in the description of this PR.
  • Attach the google doc link if the BP is written in Google Doc.

Otherwise:

  • Make sure the PR title is formatted like:
    <Issue #>: Description of pull request
    e.g. Issue 123: Description ...
  • Make sure tests pass via mvn clean apache-rat:check install spotbugs:check.
  • Replace <Issue #> in the title with the actual Issue number.

*Motivation*

We introduced a smart buffer allocator since 4.9.0. It falls back to allocate buffer
on heap when running out direct memory so that the bookie can run in a degraded mode
rather than crashes.

However if we use `directBuffer` directly, it bypasses the fallback mechanism. So
the bookie crashes when running out of direct memory.

*Modifications*

Use `buffer` instead of `directBuffer` and let the allocator decide what is the best
to use.
Copy link
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM
Will merge as soon as CI passes

@eolivelli
Copy link
Contributor

2019-09-06T09:38:45.945 [INFO] Starting audit...
[ERROR] /home/jenkins/jenkins-slave/workspace/bookkeeper_precommit_pullrequest_validation/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/WriteCache.java:27:8: Unused import: io.netty.buffer.Unpooled. [UnusedImports]
Audit done.
2019-09-06T09:38:45.945 [INFO] There is 1 error reported by Checkstyle 6.19 with buildtools/src/main/resources/bookkeeper/checkstyle.xml ruleset.

@eolivelli
Copy link
Contributor

@diegosalvi this may be your problem of today :-)
Please take a look

@diegosalvi
Copy link
Contributor

@eolivelli Unfortunately I think no. I've got many OOM due to newDirectBuffer invocation by netty itself as:

org.apache.bookkeeper.common.allocator.impl.ByteBufAllocatorImpl.newDirectBuffer(ByteBufAllocatorImpl.java:158)
at io.netty.buffer.AbstractByteBufAllocator.directBuffer(AbstractByteBufAllocator.java:187)

My guess in such cases newDirectBuffer "interface" method should route to buffer public buffer method and add a new private internalNewDirectBuffer to really instance buffers.. but I think is out of scope.

Stacktrace

19-09-06-10-58-40       io.netty.util.internal.OutOfDirectMemoryError: failed to allocate 16777216 byte(s) of direct memory (used: 1056964887, max: 1073741824)
io.netty.util.internal.OutOfDirectMemoryError: failed to allocate 16777216 byte(s) of direct memory (used: 1056964887, max: 1073741824)
        at io.netty.util.internal.PlatformDependent.incrementMemoryCounter(PlatformDependent.java:656)
        at io.netty.util.internal.PlatformDependent.allocateDirectNoCleaner(PlatformDependent.java:611)
        at io.netty.buffer.PoolArena$DirectArena.allocateDirect(PoolArena.java:768)
        at io.netty.buffer.PoolArena$DirectArena.newChunk(PoolArena.java:744)
        at io.netty.buffer.PoolArena.allocateNormal(PoolArena.java:245)
        at io.netty.buffer.PoolArena.allocate(PoolArena.java:215)
        at io.netty.buffer.PoolArena.allocate(PoolArena.java:147)
        at io.netty.buffer.PooledByteBufAllocator.newDirectBuffer(PooledByteBufAllocator.java:327)
        at io.netty.buffer.AbstractByteBufAllocator.directBuffer(AbstractByteBufAllocator.java:187)
        at org.apache.bookkeeper.common.allocator.impl.ByteBufAllocatorImpl.newDirectBuffer(ByteBufAllocatorImpl.java:164)
        at org.apache.bookkeeper.common.allocator.impl.ByteBufAllocatorImpl.newDirectBuffer(ByteBufAllocatorImpl.java:158)
        at io.netty.buffer.AbstractByteBufAllocator.directBuffer(AbstractByteBufAllocator.java:187)

@eolivelli
Copy link
Contributor

@sijie please fix checkstyle issues

@sijie
Copy link
Member Author

sijie commented Sep 9, 2019

@eolivelli done

@eolivelli
Copy link
Contributor

Let's wait for CI

@eolivelli
Copy link
Contributor

Some tests failed with an unrelated error:
Could not use Netty Epoll event loop for bookie server:
java.lang.NoClassDefFoundError: Could not initialize class io.netty.channel.epoll.EpollEventLoop
at io.netty.channel.epoll.EpollEventLoopGroup.newChild(EpollEventLoopGroup.java:134)
at io.netty.channel.epoll.EpollEventLoopGroup.newChild(EpollEventLoopGroup.java:35)
at io.netty.util.concurrent.MultithreadEventExecutorGroup.(MultithreadEventExecutorGroup

@eolivelli
Copy link
Contributor

rebuild java8
run integration tests
run pr validation

@sijie
Copy link
Member Author

sijie commented Sep 10, 2019

run pr validation


for (int i = 0; i < segmentsCount; i++) {
cacheSegments.add(Unpooled.directBuffer(segmentSize, segmentSize));
cacheSegments.add(Unpooled.buffer(segmentSize, segmentSize));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is not really coming from the allocator. Rather, the intention is to really get unpooled direct memory for the read cache.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

what is the side effect of being pooled?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

In this case that the memory is not returned to the pool (but rather the direct mem chunks will be just GCed when the handle object is GCed)

For this case, we allocate big chunks of mem (eg: 1GB each) and we keep using them indefinitely. Effectively this is a kind of "manual pooling" of the read and write cache memory.

There would be no advantage in pooling that memory again.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

but it doesn't hurt to use pooled, correct? from code maintenance perspective, isn't it clear to use allocator across the bookkeeper code base? If we need to use unpooled, we can use unpooled and document why do we need to use unpooled to make people know exact the reason behind the scene.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

but it doesn't hurt to use pooled, correct?

We'd have to at least release the buffers when the ReadCache/WriteCache are closed

If we need to use unpooled, we can use unpooled and document why do we need to use unpooled to make people know exact the reason behind the scene.

Sure, that would be better to be explicitely noted why it was that way

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure, I'm just saying that in this PR we need to add that release when switching from unpooled to pooled.

I didn't still get the point why there are benefits using unpooled buffers.

I would avoid the tracking of these buffers by the allocator. The overhead might be minimal, OTOH there's no advantage from the allocator either.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I would avoid the tracking of these buffers by the allocator. The overhead might be minimal, OTOH there's no advantage from the allocator either.

the intention is to make sure all the memory allocation is consistent across the bookie server and leverage whatever changes we made in our allocator, unless there are a very strong reason to not do so.

since there is no advantages and also no disadvantages, why not just to make things consistent and easier to maintain?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm not saying not to do it. just saying to add the release on close().

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If you are okay with using allocator, I will fix the release part.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

actually those buffers are already released on close

Copy link
Contributor

@merlimat merlimat left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

👍

@merlimat
Copy link
Contributor

[ERROR] /home/jenkins/jenkins-slave/workspace/bookkeeper_precommit_pullrequest_validation/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/ReadCache.java:27:8: Unused import: io.netty.buffer.Unpooled. [UnusedImports]

@eolivelli
Copy link
Contributor

run pr validation
run integration tests
run bookkeeper-server client tests
run bookkeeper-server replication tests

@eolivelli
Copy link
Contributor

@sijie tests are failing consistently:

2019-09-22 12:59:48,207 - ERROR - [DL-io-0:ByteBufAllocatorImpl@62] - Unable to allocate memory
io.netty.util.internal.OutOfDirectMemoryError: failed to allocate 16777216 byte(s) of direct memory (used: 2147483648, max: 2147483648)
at io.netty.util.internal.PlatformDependent.incrementMemoryCounter(PlatformDependent.java:655)
at io.netty.util.internal.PlatformDependent.allocateDirectNoCleaner(PlatformDependent.java:610)
at io.netty.buffer.PoolArena$DirectArena.allocateDirect(PoolArena.java:769)
at io.netty.buffer.PoolArena$DirectArena.newChunk(PoolArena.java:745)
at io.netty.buffer.PoolArena.allocateNormal(PoolArena.java:244)
at io.netty.buffer.PoolArena.allocate(PoolArena.java:214)
at io.netty.buffer.PoolArena.allocate(PoolArena.java:146)
at io.netty.buffer.PooledByteBufAllocator.newDirectBuffer(PooledByteBufAllocator.java:324)
at io.netty.buffer.AbstractByteBufAllocator.directBuffer(AbstractByteBufAllocator.java:185)
at org.apache.bookkeeper.common.allocator.impl.ByteBufAllocatorImpl.newDirectBuffer(ByteBufAllocatorImpl.java:164)
at org.apache.bookkeeper.common.allocator.impl.ByteBufAllocatorImpl.newDirectBuffer(ByteBufAllocatorImpl.java:158)
at io.netty.buffer.AbstractByteBufAllocator.directBuffer(AbstractByteBufAllocator.java:185)
at io.netty.buffer.AbstractByteBufAllocator.directBuffer(AbstractByteBufAllocator.java:176)
at io.netty.channel.nio.AbstractNioChannel.newDirectBuffer(AbstractNioChannel.java:448)
at io.netty.channel.nio.AbstractNioByteChannel.filterOutboundMessage(AbstractNioByteChannel.java:275)
at io.netty.channel.AbstractChannel$AbstractUnsafe.write(AbstractChannel.java:881)
at io.netty.channel.DefaultChannelPipeline$HeadContext.write(DefaultChannelPipeline.java:1391)
at io.netty.channel.AbstractChannelHandlerContext.invokeWrite0(AbstractChannelHandlerContext.java:738)
at io.netty.channel.AbstractChannelHandlerContext.invokeWrite(AbstractChannelHandlerContext.java:730)
at io.netty.channel.AbstractChannelHandlerContext.write(AbstractChannelHandlerContext.java:816)
at io.netty.channel.AbstractChannelHandlerContext.write(AbstractChannelHandlerContext.java:723)
at org.apache.bookkeeper.util.ByteBufList$Encoder.write(ByteBufList.java:328)
at io.netty.channel.AbstractChannelHandlerContext.invokeWrite0(AbstractChannelHandlerContext.java:738)
at io.netty.channel.AbstractChannelHandlerContext.invokeWrite(AbstractChannelHandlerContext.java:730)
at io.netty.channel.AbstractChannelHandlerContext.write(AbstractChannelHandlerContext.java:816)
at io.netty.channel.AbstractChannelHandlerContext.write(AbstractChannelHandlerContext.java:723)
at io.netty.channel.AbstractChannelHandlerContext.write(AbstractChannelHandlerContext.java:704)
at io.netty.handler.codec.MessageToMessageEncoder.writePromiseCombiner(MessageToMessageEncoder.java:137)
at io.netty.handler.codec.MessageToMessageEncoder.write(MessageToMessageEncoder.java:119)
at io.netty.channel.AbstractChannelHandlerContext.invokeWrite0(AbstractChannelHandlerContext.java:738)
at io.netty.channel.AbstractChannelHandlerContext.invokeWrite(AbstractChannelHandlerContext.java:730)
at io.netty.channel.AbstractChannelHandlerContext.write(AbstractChannelHandlerContext.java:816)
at io.netty.channel.AbstractChannelHandlerContext.write(AbstractChannelHandlerContext.java:723)
at org.apache.bookkeeper.proto.BookieProtoEncoding$RequestEncoder.write(BookieProtoEncoding.java:391)

@eolivelli
Copy link
Contributor

@sijie do you have time to finish this work in a way that we get it into 4.10?
Tests are failing

@sijie
Copy link
Member Author

sijie commented Oct 26, 2019

@eolivelli feel free to move 4.11. will cherry-pick it to 4.10.1

@wolfstudy
Copy link
Member

run pr validation

@jiazhai jiazhai modified the milestones: 4.10.0, 4.11.0 Oct 29, 2019
Copy link
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This patch needs rebase and also we have tests failing

@eolivelli
Copy link
Contributor

@sijie what's the status of this patch?

@Ghatage
Copy link
Contributor

Ghatage commented Jun 10, 2020

run pr validation

@eolivelli
Copy link
Contributor

@Ghatage this patch needs a rebase/resolve conflicts, it is not ready for merge.

I feel this patch is quite dangerous, it is not a good candidate for 4.11.

btw the magic word to trigger a build is
"rerun failure tests"

@eolivelli
Copy link
Contributor

@sijie I am closing this PR. It needs a rebase and it does not run GitHub actions.
feel free to reopen once it is ready again

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

7 participants