kafka-users mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "M. Manna" <manme...@gmail.com>
Subject Re: Kafka 2.11-1.1.0 crashes brokers and brings cluster down on Windows
Date Mon, 14 May 2018 16:23:33 GMT
Hey Ted,

Thanks again for reaching out. Yes the screengrab seems to have got lost.
Once the cluster is down based because of file handling related issues, I
did a full start, and here is the log:


[2018-05-14 16:35:28,580] INFO Starting log flusher with a default period
of 9223372036854775807 ms. (kafka.log.LogManager)
[2018-05-14 16:35:28,814] ERROR Failed to clean up log for
__consumer_offsets-21 in dir C:\kafka1 due to IOException
(kafka.server.LogDirFailureChannel)
java.nio.file.FileSystemException:
C:\kafka1\__consumer_offsets-21\00000000000000000000.log.cleaned ->
C:\kafka1\__consumer_offsets-21\00000000000000000000.log.swap: The process
cannot access the file because it is being used by another process.

        at
sun.nio.fs.WindowsException.translateToIOException(WindowsException.java:86)
        at
sun.nio.fs.WindowsException.rethrowAsIOException(WindowsException.java:97)
        at sun.nio.fs.WindowsFileCopy.move(WindowsFileCopy.java:387)
        at
sun.nio.fs.WindowsFileSystemProvider.move(WindowsFileSystemProvider.java:287)
        at java.nio.file.Files.move(Files.java:1395)
        at
org.apache.kafka.common.utils.Utils.atomicMoveWithFallback(Utils.java:697)
        at
org.apache.kafka.common.record.FileRecords.renameTo(FileRecords.java:212)
        at kafka.log.LogSegment.changeFileSuffixes(LogSegment.scala:415)
        at kafka.log.Log.replaceSegments(Log.scala:1644)
        at kafka.log.Cleaner.cleanSegments(LogCleaner.scala:535)
        at kafka.log.Cleaner$$anonfun$doClean$4.apply(LogCleaner.scala:462)
        at kafka.log.Cleaner$$anonfun$doClean$4.apply(LogCleaner.scala:461)
        at scala.collection.immutable.List.foreach(List.scala:392)
        at kafka.log.Cleaner.doClean(LogCleaner.scala:461)
        at kafka.log.Cleaner.clean(LogCleaner.scala:438)
        at
kafka.log.LogCleaner$CleanerThread.cleanOrSleep(LogCleaner.scala:305)
        at kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:291)
        at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:82)
        Suppressed: java.nio.file.FileSystemException:
C:\kafka1\__consumer_offsets-21\00000000000000000000.log.cleaned ->
C:\kafka1\__consumer_offsets-21\00000000000000000000.log.swap: The process
cannot access the file because it is being used by another process.

                at
sun.nio.fs.WindowsException.translateToIOException(WindowsException.java:86)
                at
sun.nio.fs.WindowsException.rethrowAsIOException(WindowsException.java:97)
                at sun.nio.fs.WindowsFileCopy.move(WindowsFileCopy.java:301)
                at
sun.nio.fs.WindowsFileSystemProvider.move(WindowsFileSystemProvider.java:287)
                at java.nio.file.Files.move(Files.java:1395)
                at
org.apache.kafka.common.utils.Utils.atomicMoveWithFallback(Utils.java:694)
                ... 12 more
[2018-05-14 16:35:28,939] INFO Awaiting socket connections on 0.0.0.0:9092.
(kafka.network.Acceptor)
[2018-05-14 16:35:28,986] ERROR Error while accepting connection
(kafka.network.Acceptor)
java.lang.ArithmeticException: / by zero
        at kafka.network.Acceptor.run(SocketServer.scala:354)
        at java.lang.Thread.run(Thread.java:748)
[2018-05-14 16:35:29,002] ERROR Error while accepting connection
(kafka.network.Acceptor)
java.lang.ArithmeticException: / by zero
        at kafka.network.Acceptor.run(SocketServer.scala:354)
        at java.lang.Thread.run(Thread.java:748)
[2018-05-14 16:35:29,017] ERROR Error while accepting connection
(kafka.network.Acceptor)
java.lang.ArithmeticException: / by zero
        at kafka.network.Acceptor.run(SocketServer.scala:354)
        at java.lang.Thread.run(Thread.java:748)
[2018-05-14 16:35:28,986] ERROR Failed to clean up log for
__consumer_offsets-21 in dir C:\kafka1 due to IOException
(kafka.server.LogDirFailureChannel)
java.nio.file.FileSystemException:
C:\kafka1\__consumer_offsets-21\00000000000000000000.log.cleaned: The
process cannot access the file because it is being used by another process.

        at
sun.nio.fs.WindowsException.translateToIOException(WindowsException.java:86)
        at
sun.nio.fs.WindowsException.rethrowAsIOException(WindowsException.java:97)
        at
sun.nio.fs.WindowsException.rethrowAsIOException(WindowsException.java:102)
        at
sun.nio.fs.WindowsFileSystemProvider.implDelete(WindowsFileSystemProvider.java:269)
        at
sun.nio.fs.AbstractFileSystemProvider.deleteIfExists(AbstractFileSystemProvider.java:108)
        at java.nio.file.Files.deleteIfExists(Files.java:1165)
        at
kafka.log.Cleaner.deleteCleanedFileIfExists$1(LogCleaner.scala:488)
        at kafka.log.Cleaner.cleanSegments(LogCleaner.scala:493)
        at kafka.log.Cleaner$$anonfun$doClean$4.apply(LogCleaner.scala:462)
        at kafka.log.Cleaner$$anonfun$doClean$4.apply(LogCleaner.scala:461)
        at scala.collection.immutable.List.foreach(List.scala:392)
        at kafka.log.Cleaner.doClean(LogCleaner.scala:461)
        at kafka.log.Cleaner.clean(LogCleaner.scala:438)
        at
kafka.log.LogCleaner$CleanerThread.cleanOrSleep(LogCleaner.scala:305)
        at kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:291)
        at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:82)
[2018-05-14 16:35:29,064] INFO [SocketServer brokerId=1] Started 1 acceptor
threads (kafka.network.SocketServer)
[2018-05-14 16:35:29,017] ERROR Error while accepting connection
(kafka.network.Acceptor)
java.lang.ArithmeticException: / by zero
        at kafka.network.Acceptor.run(SocketServer.scala:354)
        at java.lang.Thread.run(Thread.java:748)
[2018-05-14 16:35:29,189] INFO [ExpirationReaper-1-DeleteRecords]: Starting
(kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
[2018-05-14 16:35:29,205] INFO [ExpirationReaper-1-Produce]: Starting
(kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
[2018-05-14 16:35:29,220] INFO [ExpirationReaper-1-Fetch]: Starting
(kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
[2018-05-14 16:35:29,205] ERROR Failed to clean up log for
__consumer_offsets-21 in dir C:\kafka1 due to IOException
(kafka.server.LogDirFailureChannel)
java.nio.file.FileSystemException:
C:\kafka1\__consumer_offsets-21\00000000000000000000.log.cleaned: The
process cannot access the file because it is being used by another process.

        at
sun.nio.fs.WindowsException.translateToIOException(WindowsException.java:86)
        at
sun.nio.fs.WindowsException.rethrowAsIOException(WindowsException.java:97)
        at
sun.nio.fs.WindowsException.rethrowAsIOException(WindowsException.java:102)
        at
sun.nio.fs.WindowsFileSystemProvider.implDelete(WindowsFileSystemProvider.java:269)
        at
sun.nio.fs.AbstractFileSystemProvider.deleteIfExists(AbstractFileSystemProvider.java:108)
        at java.nio.file.Files.deleteIfExists(Files.java:1165)
        at
kafka.log.Cleaner.deleteCleanedFileIfExists$1(LogCleaner.scala:488)
        at kafka.log.Cleaner.cleanSegments(LogCleaner.scala:493)
        at kafka.log.Cleaner$$anonfun$doClean$4.apply(LogCleaner.scala:462)
        at kafka.log.Cleaner$$anonfun$doClean$4.apply(LogCleaner.scala:461)
        at scala.collection.immutable.List.foreach(List.scala:392)
        at kafka.log.Cleaner.doClean(LogCleaner.scala:461)
        at kafka.log.Cleaner.clean(LogCleaner.scala:438)
        at
kafka.log.LogCleaner$CleanerThread.cleanOrSleep(LogCleaner.scala:305)
        at kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:291)
        at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:82)
[2018-05-14 16:35:29,220] INFO [LogDirFailureHandler]: Starting
(kafka.server.ReplicaManager$LogDirFailureHandler)
[2018-05-14 16:35:29,330] INFO Creating /brokers/ids/1 (is it secure?
false) (kafka.zk.KafkaZkClient)
[2018-05-14 16:35:29,330] INFO [ReplicaManager broker=1] Stopping serving
replicas in dir C:\kafka1 (kafka.server.ReplicaManager)
[2018-05-14 16:35:29,345] INFO Result of znode creation at /brokers/ids/1
is: OK (kafka.zk.KafkaZkClient)
[2018-05-14 16:35:29,345] ERROR Failed to clean up log for
__consumer_offsets-21 in dir C:\kafka1 due to IOException
(kafka.server.LogDirFailureChannel)
java.nio.file.FileSystemException:
C:\kafka1\__consumer_offsets-21\00000000000000000000.log.cleaned: The
process cannot access the file because it is being used by another process.

        at
sun.nio.fs.WindowsException.translateToIOException(WindowsException.java:86)
        at
sun.nio.fs.WindowsException.rethrowAsIOException(WindowsException.java:97)
        at
sun.nio.fs.WindowsException.rethrowAsIOException(WindowsException.java:102)
        at
sun.nio.fs.WindowsFileSystemProvider.implDelete(WindowsFileSystemProvider.java:269)
        at
sun.nio.fs.AbstractFileSystemProvider.deleteIfExists(AbstractFileSystemProvider.java:108)
        at java.nio.file.Files.deleteIfExists(Files.java:1165)
        at
kafka.log.Cleaner.deleteCleanedFileIfExists$1(LogCleaner.scala:488)
        at kafka.log.Cleaner.cleanSegments(LogCleaner.scala:493)
        at kafka.log.Cleaner$$anonfun$doClean$4.apply(LogCleaner.scala:462)
        at kafka.log.Cleaner$$anonfun$doClean$4.apply(LogCleaner.scala:461)
        at scala.collection.immutable.List.foreach(List.scala:392)
        at kafka.log.Cleaner.doClean(LogCleaner.scala:461)
        at kafka.log.Cleaner.clean(LogCleaner.scala:438)
        at
kafka.log.LogCleaner$CleanerThread.cleanOrSleep(LogCleaner.scala:305)
        at kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:291)
        at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:82)
[2018-05-14 16:35:29,345] INFO [ReplicaFetcherManager on broker 1] Removed
fetcher for partitions  (kafka.server.ReplicaFetcherManager)
[2018-05-14 16:35:29,361] INFO Registered broker 1 at path /brokers/ids/1
with addresses:
ArrayBuffer(EndPoint(localhost,9092,ListenerName(PLAINTEXT),PLAINTEXT))
(kafka.zk.KafkaZkClient)
[2018-05-14 16:35:29,392] INFO [ReplicaAlterLogDirsManager on broker 1]
Removed fetcher for partitions  (kafka.server.ReplicaAlterLogDirsManager)
[2018-05-14 16:35:29,424] INFO [ReplicaManager broker=1] Broker 1 stopped
fetcher for partitions  and stopped moving logs for partitions  because
they are in the failed log directory C:\kafka1.
(kafka.server.ReplicaManager)
[2018-05-14 16:35:29,424] ERROR Failed to clean up log for
__consumer_offsets-21 in dir C:\kafka1 due to IOException
(kafka.server.LogDirFailureChannel)
java.nio.file.FileSystemException:
C:\kafka1\__consumer_offsets-21\00000000000000000000.log.cleaned: The
process cannot access the file because it is being used by another process.

        at
sun.nio.fs.WindowsException.translateToIOException(WindowsException.java:86)
        at
sun.nio.fs.WindowsException.rethrowAsIOException(WindowsException.java:97)
        at
sun.nio.fs.WindowsException.rethrowAsIOException(WindowsException.java:102)
        at
sun.nio.fs.WindowsFileSystemProvider.implDelete(WindowsFileSystemProvider.java:269)
        at
sun.nio.fs.AbstractFileSystemProvider.deleteIfExists(AbstractFileSystemProvider.java:108)
        at java.nio.file.Files.deleteIfExists(Files.java:1165)
        at
kafka.log.Cleaner.deleteCleanedFileIfExists$1(LogCleaner.scala:488)
        at kafka.log.Cleaner.cleanSegments(LogCleaner.scala:493)
        at kafka.log.Cleaner$$anonfun$doClean$4.apply(LogCleaner.scala:462)
        at kafka.log.Cleaner$$anonfun$doClean$4.apply(LogCleaner.scala:461)
        at scala.collection.immutable.List.foreach(List.scala:392)
        at kafka.log.Cleaner.doClean(LogCleaner.scala:461)
        at kafka.log.Cleaner.clean(LogCleaner.scala:438)
        at
kafka.log.LogCleaner$CleanerThread.cleanOrSleep(LogCleaner.scala:305)
        at kafka.log.LogCleaner$CleanerThread.doWork(LogCleaner.scala:291)
        at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:82)
[2018-05-14 16:35:29,439] INFO Stopping serving logs in dir C:\kafka1
(kafka.log.LogManager)
[2018-05-14 16:35:29,517] INFO [ExpirationReaper-1-topic]: Starting
(kafka.server.DelayedOperationPurgatory$ExpiredOperationReaper)
[2018-05-14 16:35:29,517] ERROR Shutdown broker because all log dirs in
C:\kafka1 have failed (kafka.log.LogManager)

Attached is the full log upon kafka-server-start.bat run. Meanwhile, i am
experimenting whether closing a log and txnIndex (if not closed already)
and unmapping the other segments before renaming helps. The problem is that
if .append() is having a clash then this approach might not help
(Potentially ClosedChannelException if the fileChannel is being closed
twice).
Once again, thanks a lot for helping out on this.

Regards,


On 14 May 2018 at 16:55, Ted Yu <yuzhihong@gmail.com> wrote:

> Looks like you were trying to show some screen which didn't go thru.
>
> Examining LogSegment.scala, I wonder if the time index was being appended
> to (which caused the deletion to fail):
>
>       // append an entry to the index (if needed)
>       if(bytesSinceLastIndexEntry > indexIntervalBytes) {
>         offsetIndex.append(largestOffset, physicalPosition)
>         timeIndex.maybeAppend(maxTimestampSoFar, offsetOfMaxTimestamp)
>
> Can you capture the stack trace and pastebin it ?
>
> If LogSegment shows up in the stack trace, we may have some idea for the
> root cause.
>
> Thanks
>
> On Mon, May 14, 2018 at 8:31 AM, M. Manna <manmedia@gmail.com> wrote:
>
> > I am having some difficulties debugging the cluster. IN CWIKI for kafka,
> > I can see a setup guide for eclipse, but nothing as such for debugger
> > setup. The issue is that every time the log cleaner thread is run, it's
> > having a FATAL shutdown.
> > I tried to close all producers and consumers, but it still locks the log
> > and Transaction index segments. Forcefully unmapping the offset and time
> > offset files work normally, but log and transaction index files don't
> work
> > properly.
> >
> > Could anyone please advise how to setup the kafka debugging on eclipse ?
> I
> > have added the sdb jar noted in scala-debugger.org site. But I couldn't
> > connect to port 5005 at all.
> >
> >
> >
> >
> > On 13 May 2018 at 10:00, M. Manna <manmedia@gmail.com> wrote:
> >
> >> Hi Ted,
> >>
> >> I highly appreciate the response over the weekend, and thanks for
> >> pointing out the JIRAs.
> >>
> >> I don't believe the processes are responsible, but individual threads
> >> which are still holding the log/index files using IO streams. I am
> trying
> >> walk a single node setup through debugger to find out which thread is
> >> locking the file. Apologise but it's a huge application so it might get
> me
> >> some time to get around it :)
> >>
> >> Please do update if you find something new.
> >>
> >> Regards,
> >>
> >> On 12 May 2018 at 22:15, Ted Yu <yuzhihong@gmail.com> wrote:
> >>
> >>> There are two outstanding issues: KAFKA-6059 and KAFKA-6200 which bear
> >>> some
> >>> resemblance.
> >>>
> >>> Can you try to find how the other process uses the file being deleted ?
> >>>
> >>> https://superuser.com/questions/117902/find-out-which-proces
> >>> s-is-locking-a-file-or-folder-in-windows
> >>> https://www.computerhope.com/issues/ch000714.htm
> >>>
> >>> Cheers
> >>>
> >>> On Sat, May 12, 2018 at 1:42 PM, M. Manna <manmedia@gmail.com> wrote:
> >>>
> >>> > Hello,
> >>> >
> >>> > We are still stuck with this issue where 2.11-1.1.0 distro is failing
> >>> to
> >>> > cleanup logs on Windows and brings the entire cluster down one by
> one.
> >>> > Extending the retention hours and sizes don't help because they
> burden
> >>> the
> >>> > hard drive.
> >>> >
> >>> > Here is the log
> >>> >
> >>> > [2018-05-12 21:36:57,673] INFO [Log partition=test-0, dir=C:\kafka1]
> >>> Rolled
> >>> > > new log segment at offset 45 in 105 ms. (kafka.log.Log)
> >>> > > [2018-05-12 21:36:57,673] INFO [Log partition=test-0,
> dir=C:\kafka1]
> >>> > > Scheduling log segment [baseOffset 0, size 2290] for deletion.
> >>> > > (kafka.log.Log)
> >>> > > [2018-05-12 21:36:57,673] ERROR Error while deleting segments
for
> >>> test-0
> >>> > > in dir C:\kafka1 (kafka.server.LogDirFailureChannel)
> >>> > > java.nio.file.FileSystemException:
> >>> > > C:\kafka1\test-0\00000000000000000000.log ->
> >>> > > C:\kafka1\test-0\00000000000000000000.log.deleted: The process
> >>> cannot
> >>> > > access the file because it is being used by another process.
> >>> > >
> >>> > >         at
> >>> > > sun.nio.fs.WindowsException.translateToIOException(
> >>> > WindowsException.java:86)
> >>> > >         at
> >>> > > sun.nio.fs.WindowsException.rethrowAsIOException(
> >>> > WindowsException.java:97)
> >>> > >         at sun.nio.fs.WindowsFileCopy.
> move(WindowsFileCopy.java:387)
> >>> > >         at
> >>> > > sun.nio.fs.WindowsFileSystemProvider.move(
> WindowsFileSystemProvider.
> >>> > java:287)
> >>> > >         at java.nio.file.Files.move(Files.java:1395)
> >>> > >         at
> >>> > > org.apache.kafka.common.utils.Utils.atomicMoveWithFallback(
> >>> > Utils.java:697)
> >>> > >         at
> >>> > > org.apache.kafka.common.record.FileRecords.renameTo(
> >>> > FileRecords.java:212)
> >>> > >         at kafka.log.LogSegment.changeFil
> >>> eSuffixes(LogSegment.scala:415)
> >>> > >         at kafka.log.Log.kafka$log$Log$$a
> >>> syncDeleteSegment(Log.scala:
> >>> > 1601)
> >>> > >         at kafka.log.Log.kafka$log$Log$$d
> >>> eleteSegment(Log.scala:1588)
> >>> > >         at
> >>> > > kafka.log.Log$$anonfun$deleteSegments$1$$anonfun$
> >>> > apply$mcI$sp$1.apply(Log.scala:1170)
> >>> > >         at
> >>> > > kafka.log.Log$$anonfun$deleteSegments$1$$anonfun$
> >>> > apply$mcI$sp$1.apply(Log.scala:1170)
> >>> > >         at
> >>> > > scala.collection.mutable.ResizableArray$class.foreach(
> >>> > ResizableArray.scala:59)
> >>> > >         at
> >>> > > scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
> >>> > >         at
> >>> > > kafka.log.Log$$anonfun$deleteSegments$1.apply$mcI$sp(
> Log.scala:1170)
> >>> > >         at kafka.log.Log$$anonfun$deleteS
> >>> egments$1.apply(Log.scala:1161)
> >>> > >         at kafka.log.Log$$anonfun$deleteS
> >>> egments$1.apply(Log.scala:1161)
> >>> > >         at kafka.log.Log.maybeHandleIOException(Log.scala:1678)
> >>> > >         at kafka.log.Log.deleteSegments(Log.scala:1161)
> >>> > >         at kafka.log.Log.deleteOldSegments(Log.scala:1156)
> >>> > >         at kafka.log.Log.deleteRetentionMsBreachedSegme
> >>> > nts(Log.scala:1228)
> >>> > >         at kafka.log.Log.deleteOldSegments(Log.scala:1222)
> >>> > >         at
> >>> > > kafka.log.LogManager$$anonfun$cleanupLogs$3.apply(LogManager
> >>> .scala:854)
> >>> > >         at
> >>> > > kafka.log.LogManager$$anonfun$cleanupLogs$3.apply(LogManager
> >>> .scala:852)
> >>> > >         at
> >>> > > scala.collection.TraversableLike$WithFilter$$anonfun$foreach
> >>> $1.apply(
> >>> > TraversableLike.scala:733)
> >>> > >         at scala.collection.immutable.List.foreach(List.scala:392)
> >>> > >         at
> >>> > > scala.collection.TraversableLike$WithFilter.
> >>> > foreach(TraversableLike.scala:732)
> >>> > >         at kafka.log.LogManager.cleanupLogs(LogManager.scala:852)
> >>> > >         at
> >>> > > kafka.log.LogManager$$anonfun$startup$1.apply$mcV$sp(
> >>> > LogManager.scala:385)
> >>> > >         at
> >>> > > kafka.utils.KafkaScheduler$$anonfun$1.apply$mcV$sp(
> >>> > KafkaScheduler.scala:110)
> >>> > >         at kafka.utils.CoreUtils$$anon$1.run(CoreUtils.scala:62)
> >>> > >         at
> >>> > > java.util.concurrent.Executors$RunnableAdapter.call(Executor
> >>> s.java:511)
> >>>
> >>> > >         at java.util.concurrent.FutureTask.runAndReset(
> >>> > FutureTask.java:308)
> >>> > >         at
> >>> > > java.util.concurrent.ScheduledThreadPoolExecutor$
> >>> > ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
> >>> > >         at
> >>> > > java.util.concurrent.ScheduledThreadPoolExecutor$
> >>> > ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
> >>> > >         at
> >>> > > java.util.concurrent.ThreadPoolExecutor.runWorker(
> >>> > ThreadPoolExecutor.java:1149)
> >>> > >         at
> >>> > > java.util.concurrent.ThreadPoolExecutor$Worker.run(
> >>> > ThreadPoolExecutor.java:624)
> >>> > >         at java.lang.Thread.run(Thread.java:748)
> >>> > >         Suppressed: java.nio.file.FileSystemException:
> >>> > > C:\kafka1\test-0\00000000000000000000.log ->
> >>> > > C:\kafka1\test-0\00000000000000000000.log.deleted: The process
> >>> cannot
> >>> > > access the file because it is being used by another process.
> >>> > >
> >>> > >                 at
> >>> > > sun.nio.fs.WindowsException.translateToIOException(
> >>> > WindowsException.java:86)
> >>> > >                 at
> >>> > > sun.nio.fs.WindowsException.rethrowAsIOException(
> >>> > WindowsException.java:97)
> >>> > >                 at
> >>> > > sun.nio.fs.WindowsFileCopy.move(WindowsFileCopy.java:301)
> >>> > >                 at
> >>> > > sun.nio.fs.WindowsFileSystemProvider.move(
> WindowsFileSystemProvider.
> >>> > java:287)
> >>> > >                 at java.nio.file.Files.move(Files.java:1395)
> >>> > >                 at
> >>> > > org.apache.kafka.common.utils.Utils.atomicMoveWithFallback(
> >>> > Utils.java:694)
> >>> > >                 ... 32 more
> >>> > > [2018-05-12 21:36:57,689] INFO [ReplicaManager broker=1] Stopping
> >>> serving
> >>> > > replicas in dir C:\kafka1 (kafka.server.ReplicaManager)
> >>> > > [2018-05-12 21:36:57,689] ERROR Uncaught exception in scheduled
> task
> >>> > > 'kafka-log-retention' (kafka.utils.KafkaScheduler)
> >>> > > org.apache.kafka.common.errors.KafkaStorageException: Error while
> >>> > deleting
> >>> > > segments for test-0 in dir C:\kafka1
> >>> > > Caused by: java.nio.file.FileSystemException:
> >>> > > C:\kafka1\test-0\00000000000000000000.log ->
> >>> > > C:\kafka1\test-0\00000000000000000000.log.deleted: The process
> >>> cannot
> >>> > > access the file because it is being used by another process.
> >>> > >
> >>> > >         at
> >>> > > sun.nio.fs.WindowsException.translateToIOException(
> >>> > WindowsException.java:86)
> >>> > >         at
> >>> > > sun.nio.fs.WindowsException.rethrowAsIOException(
> >>> > WindowsException.java:97)
> >>> > >         at sun.nio.fs.WindowsFileCopy.
> move(WindowsFileCopy.java:387)
> >>> > >         at
> >>> > > sun.nio.fs.WindowsFileSystemProvider.move(
> WindowsFileSystemProvider.
> >>> > java:287)
> >>> > >         at java.nio.file.Files.move(Files.java:1395)
> >>> > >         at
> >>> > > org.apache.kafka.common.utils.Utils.atomicMoveWithFallback(
> >>> > Utils.java:697)
> >>> > >         at
> >>> > > org.apache.kafka.common.record.FileRecords.renameTo(
> >>> > FileRecords.java:212)
> >>> > >         at kafka.log.LogSegment.changeFil
> >>> eSuffixes(LogSegment.scala:415)
> >>> > >         at kafka.log.Log.kafka$log$Log$$a
> >>> syncDeleteSegment(Log.scala:
> >>> > 1601)
> >>> > >         at kafka.log.Log.kafka$log$Log$$d
> >>> eleteSegment(Log.scala:1588)
> >>> > >         at
> >>> > > kafka.log.Log$$anonfun$deleteSegments$1$$anonfun$
> >>> > apply$mcI$sp$1.apply(Log.scala:1170)
> >>> > >         at
> >>> > > kafka.log.Log$$anonfun$deleteSegments$1$$anonfun$
> >>> > apply$mcI$sp$1.apply(Log.scala:1170)
> >>> > >         at
> >>> > > scala.collection.mutable.ResizableArray$class.foreach(
> >>> > ResizableArray.scala:59)
> >>> > >         at
> >>> > > scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
> >>> > >         at
> >>> > > kafka.log.Log$$anonfun$deleteSegments$1.apply$mcI$sp(
> Log.scala:1170)
> >>> > >         at kafka.log.Log$$anonfun$deleteS
> >>> egments$1.apply(Log.scala:1161)
> >>> > >         at kafka.log.Log$$anonfun$deleteS
> >>> egments$1.apply(Log.scala:1161)
> >>> > >         at kafka.log.Log.maybeHandleIOException(Log.scala:1678)
> >>> > >         at kafka.log.Log.deleteSegments(Log.scala:1161)
> >>> > >         at kafka.log.Log.deleteOldSegments(Log.scala:1156)
> >>> > >         at kafka.log.Log.deleteRetentionMsBreachedSegme
> >>> > nts(Log.scala:1228)
> >>> > >         at kafka.log.Log.deleteOldSegments(Log.scala:1222)
> >>> > >         at
> >>> > > kafka.log.LogManager$$anonfun$cleanupLogs$3.apply(LogManager
> >>> .scala:854)
> >>> > >         at
> >>> > > kafka.log.LogManager$$anonfun$cleanupLogs$3.apply(LogManager
> >>> .scala:852)
> >>> > >         at
> >>> > > scala.collection.TraversableLike$WithFilter$$anonfun$foreach
> >>> $1.apply(
> >>> > TraversableLike.scala:733)
> >>> > >         at scala.collection.immutable.List.foreach(List.scala:392)
> >>> > >         at
> >>> > > scala.collection.TraversableLike$WithFilter.
> >>> > foreach(TraversableLike.scala:732)
> >>> > >         at kafka.log.LogManager.cleanupLogs(LogManager.scala:852)
> >>> > >         at
> >>> > > kafka.log.LogManager$$anonfun$startup$1.apply$mcV$sp(
> >>> > LogManager.scala:385)
> >>> > >         at
> >>> > > kafka.utils.KafkaScheduler$$anonfun$1.apply$mcV$sp(
> >>> > KafkaScheduler.scala:110)
> >>> > >         at kafka.utils.CoreUtils$$anon$1.run(CoreUtils.scala:62)
> >>> > >         at
> >>> > > java.util.concurrent.Executors$RunnableAdapter.call(Executor
> >>> s.java:511)
> >>>
> >>> > >         at java.util.concurrent.FutureTask.runAndReset(
> >>> > FutureTask.java:308)
> >>> > >         at
> >>> > > java.util.concurrent.ScheduledThreadPoolExecutor$
> >>> > ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
> >>> > >         at
> >>> > > java.util.concurrent.ScheduledThreadPoolExecutor$
> >>> > ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
> >>> > >         at
> >>> > > java.util.concurrent.ThreadPoolExecutor.runWorker(
> >>> > ThreadPoolExecutor.java:1149)
> >>> > >         at
> >>> > > java.util.concurrent.ThreadPoolExecutor$Worker.run(
> >>> > ThreadPoolExecutor.java:624)
> >>> > >         at java.lang.Thread.run(Thread.java:748)
> >>> > >         Suppressed: java.nio.file.FileSystemException:
> >>> > > C:\kafka1\test-0\00000000000000000000.log ->
> >>> > > C:\kafka1\test-0\00000000000000000000.log.deleted: The process
> >>> cannot
> >>> > > access the file because it is being used by another process.
> >>> > >
> >>> > >                 at
> >>> > > sun.nio.fs.WindowsException.translateToIOException(
> >>> > WindowsException.java:86)
> >>> > >                 at
> >>> > > sun.nio.fs.WindowsException.rethrowAsIOException(
> >>> > WindowsException.java:97)
> >>> > >                 at
> >>> > > sun.nio.fs.WindowsFileCopy.move(WindowsFileCopy.java:301)
> >>> > >                 at
> >>> > > sun.nio.fs.WindowsFileSystemProvider.move(
> WindowsFileSystemProvider.
> >>> > java:287)
> >>> > >                 at java.nio.file.Files.move(Files.java:1395)
> >>> > >                 at
> >>> > > org.apache.kafka.common.utils.Utils.atomicMoveWithFallback(
> >>> > Utils.java:694)
> >>> > >                 ... 32 more
> >>> > > [2018-05-12 21:36:57,689] INFO [ReplicaFetcherManager on broker
1]
> >>> > Removed
> >>> > > fetcher for partitions
> >>> > > __consumer_offsets-22,__consumer_offsets-30,__
> >>> > consumer_offsets-8,__consumer_offsets-21,__consumer_offsets-
> >>> > 4,__consumer_offsets-27,__consumer_offsets-7,__consumer_
> >>> > offsets-9,__consumer_offsets-46,__consumer_offsets-25,__
> >>> > consumer_offsets-35,__consumer_offsets-41,__consumer_offsets-33,__
> >>> > consumer_offsets-23,__consumer_offsets-49,__consumer_offsets-47,__
> >>> > consumer_offsets-16,test-0,__consumer_offsets-28,__consumer_
> >>> offsets-31,__
> >>> > consumer_offsets-36,__consumer_offsets-42,__consumer_offsets
> >>> -3,__consumer_
> >>> > offsets-18,test1-0,__consumer_offsets-37,__consumer_offsets-
> >>> > 15,__consumer_offsets-24,__consumer_offsets-38,__consumer_of
> >>> fsets-17,__
> >>> > consumer_offsets-48,__consumer_offsets-19,__consumer_offsets-11,__
> >>> > consumer_offsets-13,__consumer_offsets-2,__consumer_
> >>> > offsets-43,__consumer_offsets-6,__consumer_offsets-14,__
> >>> > consumer_offsets-20,__consumer_offsets-0,__consumer_
> >>> > offsets-44,__consumer_offsets-39,__consumer_offsets-12,__
> >>> > consumer_offsets-45,__consumer_offsets-1,__consumer_
> >>> > offsets-5,__consumer_offsets-26,__consumer_offsets-29,__
> >>> > consumer_offsets-34,__consumer_offsets-10,__consumer_offsets-32,__
> >>> > consumer_offsets-40
> >>> > > (kafka.server.ReplicaFetcherManager)
> >>> > > [2018-05-12 21:36:57,689] INFO [ReplicaAlterLogDirsManager on
> broker
> >>> 1]
> >>> > > Removed fetcher for partitions
> >>> > > __consumer_offsets-22,__consumer_offsets-30,__
> >>> > consumer_offsets-8,__consumer_offsets-21,__consumer_offsets-
> >>> > 4,__consumer_offsets-27,__consumer_offsets-7,__consumer_
> >>> > offsets-9,__consumer_offsets-46,__consumer_offsets-25,__
> >>> > consumer_offsets-35,__consumer_offsets-41,__consumer_offsets-33,__
> >>> > consumer_offsets-23,__consumer_offsets-49,__consumer_offsets-47,__
> >>> > consumer_offsets-16,test-0,__consumer_offsets-28,__consumer_
> >>> offsets-31,__
> >>> > consumer_offsets-36,__consumer_offsets-42,__consumer_offsets
> >>> -3,__consumer_
> >>> > offsets-18,test1-0,__consumer_offsets-37,__consumer_offsets-
> >>> > 15,__consumer_offsets-24,__consumer_offsets-38,__consumer_of
> >>> fsets-17,__
> >>> > consumer_offsets-48,__consumer_offsets-19,__consumer_offsets-11,__
> >>> > consumer_offsets-13,__consumer_offsets-2,__consumer_
> >>> > offsets-43,__consumer_offsets-6,__consumer_offsets-14,__
> >>> > consumer_offsets-20,__consumer_offsets-0,__consumer_
> >>> > offsets-44,__consumer_offsets-39,__consumer_offsets-12,__
> >>> > consumer_offsets-45,__consumer_offsets-1,__consumer_
> >>> > offsets-5,__consumer_offsets-26,__consumer_offsets-29,__
> >>> > consumer_offsets-34,__consumer_offsets-10,__consumer_offsets-32,__
> >>> > consumer_offsets-40
> >>> > > (kafka.server.ReplicaAlterLogDirsManager)
> >>> > > [2018-05-12 21:36:57,751] INFO [ReplicaManager broker=1] Broker
1
> >>> stopped
> >>> > > fetcher for partitions
> >>> > > __consumer_offsets-22,__consumer_offsets-30,__
> >>> > consumer_offsets-8,__consumer_offsets-21,__consumer_offsets-
> >>> > 4,__consumer_offsets-27,__consumer_offsets-7,__consumer_
> >>> > offsets-9,__consumer_offsets-46,__consumer_offsets-25,__
> >>> > consumer_offsets-35,__consumer_offsets-41,__consumer_offsets-33,__
> >>> > consumer_offsets-23,__consumer_offsets-49,__consumer_offsets-47,__
> >>> > consumer_offsets-16,test-0,__consumer_offsets-28,__consumer_
> >>> offsets-31,__
> >>> > consumer_offsets-36,__consumer_offsets-42,__consumer_offsets
> >>> -3,__consumer_
> >>> > offsets-18,test1-0,__consumer_offsets-37,__consumer_offsets-
> >>> > 15,__consumer_offsets-24,__consumer_offsets-38,__consumer_of
> >>> fsets-17,__
> >>> > consumer_offsets-48,__consumer_offsets-19,__consumer_offsets-11,__
> >>> > consumer_offsets-13,__consumer_offsets-2,__consumer_
> >>> > offsets-43,__consumer_offsets-6,__consumer_offsets-14,__
> >>> > consumer_offsets-20,__consumer_offsets-0,__consumer_
> >>> > offsets-44,__consumer_offsets-39,__consumer_offsets-12,__
> >>> > consumer_offsets-45,__consumer_offsets-1,__consumer_
> >>> > offsets-5,__consumer_offsets-26,__consumer_offsets-29,__
> >>> > consumer_offsets-34,__consumer_offsets-10,__consumer_offsets-32,__
> >>> > consumer_offsets-40
> >>> > > and stopped moving logs for partitions  because they are in the
> >>> failed
> >>> > log
> >>> > > directory C:\kafka1. (kafka.server.ReplicaManager)
> >>> > > [2018-05-12 21:36:57,751] INFO Stopping serving logs in dir
> C:\kafka1
> >>> > > (kafka.log.LogManager)
> >>> > > [2018-05-12 21:36:57,767] ERROR Shutdown broker because all log
> dirs
> >>> in
> >>> > > C:\kafka1 have failed (kafka.log.LogManager)
> >>> > >
> >>> >
> >>> > Could someone please share some ideas how to rectify this on Windows?
> >>> If
> >>> > this will never be supported on Windows, could we get some official
> >>> > communication perhaps?
> >>> >
> >>> > Regards,
> >>> >
> >>>
> >>
> >>
> >
>

Mime
View raw message