[
https://issues.apache.org/jira/browse/KAFKA-4391?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15676418#comment-15676418
]
Yiquan Zhou commented on KAFKA-4391:
------------------------------------
I did some tests with 0.10.1.0, I still got the same issue with a different
exception, probably due to the use of Files.move instead of Files.renameTo:
{code}
[2016-11-18 11:24:31,357] FATAL [Replica Manager on Broker 0]: Error writing to
highwatermark file: (kafka.server.ReplicaManager)
java.nio.file.FileAlreadyExistsException:
D:\tmp\kafka-logs\replication-offset-checkpoint.tmp ->
D:\tmp\kafka-logs\replication-offset-checkpoint
at
sun.nio.fs.WindowsException.translateToIOException(WindowsException.java:81)
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:1345)
at
org.apache.kafka.common.utils.Utils.atomicMoveWithFallback(Utils.java:670)
at kafka.server.OffsetCheckpoint.write(OffsetCheckpoint.scala:74)
at
kafka.server.ReplicaManager$$anonfun$checkpointHighWatermarks$2.apply(ReplicaManager.scala:927)
at
kafka.server.ReplicaManager$$anonfun$checkpointHighWatermarks$2.apply(ReplicaManager.scala:924)
at
scala.collection.TraversableLike$WithFilter$$anonfun$foreach$1.apply(TraversableLike.scala:733)
at scala.collection.immutable.Map$Map1.foreach(Map.scala:116)
at
scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:732)
at
kafka.server.ReplicaManager.checkpointHighWatermarks(ReplicaManager.scala:924)
at
kafka.server.ReplicaManager$$anonfun$1.apply$mcV$sp(ReplicaManager.scala:162)
at
kafka.utils.KafkaScheduler$$anonfun$1.apply$mcV$sp(KafkaScheduler.scala:110)
at kafka.utils.CoreUtils$$anon$1.run(CoreUtils.scala:58)
at
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:304)
at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:178)
at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:744)
Suppressed: java.nio.file.AccessDeniedException:
D:\tmp\kafka-logs\replication-offset-checkpoint.tmp ->
D:\tmp\kafka-logs\replication-offset-checkpoint
at
sun.nio.fs.WindowsException.translateToIOException(WindowsException.java:83)
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:1345)
at
org.apache.kafka.common.utils.Utils.atomicMoveWithFallback(Utils.java:667)
... 17 more
{code}
So maybe this is not the root cause of the issue.
> On Windows, Kafka server stops with uncaught exception after coming back from
> sleep
> -----------------------------------------------------------------------------------
>
> Key: KAFKA-4391
> URL: https://issues.apache.org/jira/browse/KAFKA-4391
> Project: Kafka
> Issue Type: Bug
> Affects Versions: 0.9.0.1
> Environment: Windows 10, jdk1.8.0_111
> Reporter: Yiquan Zhou
>
> Steps to reproduce:
> 1. start the zookeeper
> $ bin\windows\zookeeper-server-start.bat config/zookeeper.properties
> 2. start the Kafka server with the default properties
> $ bin\windows\kafka-server-start.bat config/server.properties
> 3. put Windows into sleep mode for 1-2 hours
> 4. activate Windows again, an exception occurs in Kafka server console and
> the server is stopped:
> {code:title=kafka console log}
> [2016-11-08 21:45:35,185] INFO Client session timed out, have not heard from
> server in 10081379ms for sessionid 0x1584514da470000, closing socket
> connection and attempting reconnect (org.apache.zookeeper.ClientCnxn)
> [2016-11-08 21:45:40,698] INFO zookeeper state changed (Disconnected)
> (org.I0Itec.zkclient.ZkClient)
> [2016-11-08 21:45:43,029] INFO Opening socket connection to server
> 127.0.0.1/127.0.0.1:2181. Will not attempt to authenticate using SASL
> (unknown error) (org.apache.zookeeper.ClientCnxn)
> [2016-11-08 21:45:43,044] INFO Socket connection established to
> 127.0.0.1/127.0.0.1:2181, initiating session (org.apache.zookeeper.ClientCnxn)
> [2016-11-08 21:45:43,158] INFO Unable to reconnect to ZooKeeper service,
> session 0x1584514da470000 has expired, closing socket connection
> (org.apache.zookeeper.ClientCnxn)
> [2016-11-08 21:45:43,158] INFO zookeeper state changed (Expired)
> (org.I0Itec.zkclient.ZkClient)
> [2016-11-08 21:45:43,236] INFO Initiating client connection,
> connectString=localhost:2181 sessionTimeout=6000
> watcher=org.I0Itec.zkclient.ZkClient@11ca437b (org.apache.zookeeper.ZooKeeper)
> [2016-11-08 21:45:43,280] INFO EventThread shut down
> (org.apache.zookeeper.ClientCnxn)
> log4j:ERROR Failed to rename [/controller.log] to
> [/controller.log.2016-11-08-18].
> [2016-11-08 21:45:43,421] INFO Opening socket connection to server
> 127.0.0.1/127.0.0.1:2181. Will not attempt to authenticate using SASL
> (unknown error) (org.apache.zookeeper.ClientCnxn)
> [2016-11-08 21:45:43,483] INFO Socket connection established to
> 127.0.0.1/127.0.0.1:2181, initiating session (org.apache.zookeeper.ClientCnxn)
> [2016-11-08 21:45:43,811] INFO Session establishment complete on server
> 127.0.0.1/127.0.0.1:2181, sessionid = 0x1584514da470001, negotiated timeout =
> 6000 (org.apache.zookeeper.ClientCnxn)
> [2016-11-08 21:45:43,827] INFO zookeeper state changed (SyncConnected)
> (org.I0Itec.zkclient.ZkClient)
> log4j:ERROR Failed to rename [/server.log] to [/server.log.2016-11-08-18].
> [2016-11-08 21:45:43,827] INFO Creating /controller (is it secure? false)
> (kafka.utils.ZKCheckedEphemeral)
> [2016-11-08 21:45:44,014] INFO Result of znode creation is: OK
> (kafka.utils.ZKCheckedEphemeral)
> [2016-11-08 21:45:44,014] INFO 0 successfully elected as leader
> (kafka.server.ZookeeperLeaderElector)
> log4j:ERROR Failed to rename [/state-change.log] to
> [/state-change.log.2016-11-08-18].
> [2016-11-08 21:45:44,421] INFO re-registering broker info in ZK for broker 0
> (kafka.server.KafkaHealthcheck)
> [2016-11-08 21:45:44,436] INFO Creating /brokers/ids/0 (is it secure? false)
> (kafka.utils.ZKCheckedEphemeral)
> [2016-11-08 21:45:44,686] INFO Result of znode creation is: OK
> (kafka.utils.ZKCheckedEphemeral)
> [2016-11-08 21:45:44,686] INFO Registered broker 0 at path /brokers/ids/0
> with addresses: PLAINTEXT -> EndPoint(192.168.0.15,9092,PLAINTEXT)
> (kafka.utils.ZkUtils)
> [2016-11-08 21:45:44,686] INFO done re-registering broker
> (kafka.server.KafkaHealthcheck)
> [2016-11-08 21:45:44,686] INFO Subscribing to /brokers/topics path to watch
> for new topics (kafka.server.KafkaHealthcheck)
> [2016-11-08 21:45:45,046] INFO [ReplicaFetcherManager on broker 0] Removed
> fetcher for partitions [test,0] (kafka.server.ReplicaFetcherManager)
> [2016-11-08 21:45:45,061] INFO New leader is 0
> (kafka.server.ZookeeperLeaderElector$LeaderChangeListener)
> [2016-11-08 21:45:47,325] ERROR Uncaught exception in scheduled task
> 'kafka-recovery-point-checkpoint' (kafka.utils.KafkaScheduler)
> java.io.IOException: File rename from
> D:\tmp\kafka-logs\recovery-point-offset-checkpoint.tmp to
> D:\tmp\kafka-logs\recovery-point-offset-checkpoint failed.
> at kafka.server.OffsetCheckpoint.write(OffsetCheckpoint.scala:66)
> at
> kafka.log.LogManager.kafka$log$LogManager$$checkpointLogsInDir(LogManager.scala:326)
> at
> kafka.log.LogManager$$anonfun$checkpointRecoveryPointOffsets$1.apply(LogManager.scala:317)
> at
> kafka.log.LogManager$$anonfun$checkpointRecoveryPointOffsets$1.apply(LogManager.scala:317)
> at
> scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
> at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:108)
> at
> kafka.log.LogManager.checkpointRecoveryPointOffsets(LogManager.scala:317)
> at
> kafka.log.LogManager$$anonfun$startup$3.apply$mcV$sp(LogManager.scala:201)
> at
> kafka.utils.KafkaScheduler$$anonfun$1.apply$mcV$sp(KafkaScheduler.scala:110)
> at kafka.utils.CoreUtils$$anon$1.run(CoreUtils.scala:60)
> at
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
> at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:304)
> at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:178)
> at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> at java.lang.Thread.run(Thread.java:744)
> {code}
> {code:title=zookeeper console log}
> [2016-11-08 21:45:35,185] WARN caught end of stream exception
> (org.apache.zookeeper.server.NIOServerCnxn)
> EndOfStreamException: Unable to read additional data from client sessionid
> 0x1584514da470000, likely client has closed socket
> at
> org.apache.zookeeper.server.NIOServerCnxn.doIO(NIOServerCnxn.java:228)
> at
> org.apache.zookeeper.server.NIOServerCnxnFactory.run(NIOServerCnxnFactory.java:208)
> at java.lang.Thread.run(Thread.java:744)
> [2016-11-08 21:45:35,185] INFO Closed socket connection for client
> /0:0:0:0:0:0:0:1:50192 which had sessionid 0x1584514da470000
> (org.apache.zookeeper.server.NIOServerCnxn)
> [2016-11-08 21:45:35,372] INFO Expiring session 0x1584514da470000, timeout of
> 6000ms exceeded (org.apache.zookeeper.server.ZooKeeperServer)
> [2016-11-08 21:45:35,372] INFO Processed session termination for sessionid:
> 0x1584514da470000 (org.apache.zookeeper.server.PrepRequestProcessor)
> [2016-11-08 21:45:40,588] WARN fsync-ing the write ahead log in SyncThread:0
> took 5221ms which will adversely effect operation latency. See the ZooKeeper
> troubleshooting guide (org.apache.zookeeper.server.persistence.FileTxnLog)
> [2016-11-08 21:45:43,044] INFO Accepted socket connection from
> /127.0.0.1:50783 (org.apache.zookeeper.server.NIOServerCnxnFactory)
> [2016-11-08 21:45:43,127] INFO Client attempting to renew session
> 0x1584514da470000 at /127.0.0.1:50783
> (org.apache.zookeeper.server.ZooKeeperServer)
> [2016-11-08 21:45:43,158] INFO Invalid session 0x1584514da470000 for client
> /127.0.0.1:50783, probably expired
> (org.apache.zookeeper.server.ZooKeeperServer)
> [2016-11-08 21:45:43,158] INFO Closed socket connection for client
> /127.0.0.1:50783 which had sessionid 0x1584514da470000
> (org.apache.zookeeper.server.NIOServerCnxn)
> [2016-11-08 21:45:43,483] INFO Accepted socket connection from
> /127.0.0.1:50801 (org.apache.zookeeper.server.NIOServerCnxnFactory)
> [2016-11-08 21:45:43,514] INFO Client attempting to establish new session at
> /127.0.0.1:50801 (org.apache.zookeeper.server.ZooKeeperServer)
> [2016-11-08 21:45:43,811] INFO Established session 0x1584514da470001 with
> negotiated timeout 6000 for client /127.0.0.1:50801
> (org.apache.zookeeper.server.ZooKeeperServer)
> [2016-11-08 21:45:44,327] INFO Got user-level KeeperException when processing
> sessionid:0x1584514da470001 type:delete cxid:0x18 zxid:0x15d txntype:-1
> reqpath:n/a Error Path:/admin/preferred_replica_election
> Error:KeeperErrorCode = NoNode for /admin/preferred_replica_election
> (org.apache.zookeeper.server.PrepRequestProcessor)
> [2016-11-08 21:45:44,436] INFO Got user-level KeeperException when processing
> sessionid:0x1584514da470001 type:create cxid:0x19 zxid:0x15e txntype:-1
> reqpath:n/a Error Path:/brokers Error:KeeperErrorCode = NodeExists for
> /brokers (org.apache.zookeeper.server.PrepRequestProcessor)
> [2016-11-08 21:45:44,452] INFO Got user-level KeeperException when processing
> sessionid:0x1584514da470001 type:create cxid:0x1a zxid:0x15f txntype:-1
> reqpath:n/a Error Path:/brokers/ids Error:KeeperErrorCode = NodeExists for
> /brokers/ids (org.apache.zookeeper.server.PrepRequestProcessor)
> {code}
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)