>From that link, one workaround is to set the buffer to null and force a GC.
Not sure if that's a good idea though.

Thanks,

Jun


On Tue, Jul 9, 2013 at 10:13 PM, Sriram Subramanian <
srsubraman...@linkedin.com> wrote:

> As far as I am aware it is not possible to resize mapped buffer without
> unmapping in Windows. W.r.t Java the bug here gives more context on why it
> does not support synchronous unmap function.
>
> http://bugs.sun.com/view_bug.do?bug_id=4724038
>
>
>
> On 7/9/13 9:54 PM, "Jay Kreps" <jay.kr...@gmail.com> wrote:
>
> >The problem appears to be that we are resizing a memory mapped file which
> >it looks like windows does not allow (which is kind of sucky).
> >
> >The offending method is OffsetIndex.resize().
> >
> >The most obvious fix would be to first unmap the file, then resize, then
> >remap it. We can't do this though because Java actually doesn't support
> >unmapping files (it does this lazily with garbage collection, which really
> >sucks). In fact as far as I know there is NO way to guarantee an unmap
> >occurs at a particular time, so if this is correct and windows doesn't
> >allow resizing then this combination of suckiness means that there is no
> >way to resize a file that has ever been mapped short of closing the
> >process.
> >
> >I actually don't have access to a windows machine so it is a little hard
> >for me to test this. The question is whether there is any work around. I
> >am
> >happy to change that method but we do need to be able to resize memory
> >mapped files.
> >
> >
> >
> >
> >
> >
> >On Tue, Jul 9, 2013 at 9:04 PM, Jun Rao <jun...@gmail.com> wrote:
> >
> >> Hmm, not sure what the issue is. Any windows user wants to chime in?
> >>
> >> Thanks,
> >>
> >> Jun
> >>
> >>
> >> On Tue, Jul 9, 2013 at 9:00 AM, Denny Lee <denny.g....@gmail.com>
> wrote:
> >>
> >> > Hey Jun,
> >> >
> >> > We've been running into this issue when running perf.Performance as
> >>per
> >> > http://blog.liveramp.com/2013/04/08/kafka-0-8-producer-performance-2/
> .
> >> > When running it using 100K messages, it works fine on Windows with
> >>about
> >> > 20-30K msg/s.  But when running it with 1M messages, then the broker
> >> fails
> >> > as per the message below.  It does not appear that modifying the JVM
> >> > memory configurations nor running on SSDs has any effect.   As for
> >>JVMs -
> >> > no plug ins and we've tried both 1.6 and OpenJDK 1.7.
> >> >
> >> > This looks like a JVM memory map issue on Windows issue - perhaps
> >>running
> >> > some System.gc() to prevent the roll?
> >> >
> >> > Any thoughts?
> >> >
> >> > Thanks!
> >> > Denny
> >> >
> >> >
> >> >
> >> >
> >> > On 7/9/13 7:55 AM, "Jun Rao" <jun...@gmail.com> wrote:
> >> >
> >> > >A couple of users seem to be able to get 0.8 working on Windows. Any
> >> thing
> >> > >special about your Windows environment? Are you using any jvm
> >>plugins?
> >> > >
> >> > >Thanks,
> >> > >
> >> > >Jun
> >> > >
> >> > >
> >> > >On Tue, Jul 9, 2013 at 12:59 AM, Timothy Chen <tnac...@gmail.com>
> >> wrote:
> >> > >
> >> > >> Hi all,
> >> > >>
> >> > >> I've tried pushing a large amount of messages into Kafka on
> >>Windows,
> >> and
> >> > >> got the following error:
> >> > >>
> >> > >> Caused by: java.io.IOException: The requested operation cannot be
> >> > >>performed
> >> > >> on a
> >> > >>  file with a user-mapped section open
> >> > >>         at java.io.RandomAccessFile.setLength(Native Method)
> >> > >>         at
> >>kafka.log.OffsetIndex.liftedTree2$1(OffsetIndex.scala:263)
> >> > >>         at kafka.log.OffsetIndex.resize(OffsetIndex.scala:262)
> >> > >>         at
> >> kafka.log.OffsetIndex.trimToValidSize(OffsetIndex.scala:247)
> >> > >>         at kafka.log.Log.rollToOffset(Log.scala:518)
> >> > >>         at kafka.log.Log.roll(Log.scala:502)
> >> > >>         at kafka.log.Log.maybeRoll(Log.scala:484)
> >> > >>         at kafka.log.Log.append(Log.scala:297)
> >> > >>         ... 19 more
> >> > >>
> >> > >> I suspect that Windows is not releasing memory mapped file
> >>references
> >> > >>soon
> >> > >> enough.
> >> > >>
> >> > >> I wonder if there is any good workaround or solutions for this?
> >> > >>
> >> > >> Thanks!
> >> > >>
> >> > >> Tim
> >> > >>
> >> >
> >> >
> >> >
> >>
>
>

Reply via email to