What logs of /172.16.20.16:7000 <http://172.16.20.16:7000/> say when
repair failed. It indicates "validation failed". Can you check
system.log for /172.16.20.16:7000 <http://172.16.20.16:7000/> and see
what they say. Looks like you have some issue with *doc/origdoc,
probably some corrupt sstable. *Try to run repair for individual table
and see for which table repair fails.
Regards
Manish
On Mon, Aug 7, 2023 at 11:39 PM Joe Obernberger
<joseph.obernber...@gmail.com> wrote:
Thank you. I've tried:
nodetool repair --full
nodetool repair -pr
They all get to 57% on any of the nodes, and then fail.
Interestingly the debug log only has INFO - there are no errors.
[2023-08-07 14:02:09,828] Repair command #6 failed with error
Incremental repair session 83dc17d0-354c-11ee-809c-177460b0ed52
has failed
[2023-08-07 14:02:09,830] Repair command #6 finished with error
error: Repair job has failed with the error message: Repair
command #6 failed with error Incremental repair session
83dc17d0-354c-11ee-809c-177460b0ed52 has failed. Check the logs on
the repair participants for further details
-- StackTrace --
java.lang.RuntimeException: Repair job has failed with the error
message: Repair command #6 failed with error Incremental repair
session 83dc17d0-354c-11ee-809c-177460b0ed52 has failed. Check the
logs on the repair participants for further details
at
org.apache.cassandra.tools.RepairRunner.progress(RepairRunner.java:137)
at
org.apache.cassandra.utils.progress.jmx.JMXNotificationProgressListener.handleNotification(JMXNotificationProgressListener.java:77)
at
java.management/com.sun.jmx.remote.internal.ClientNotifForwarder$NotifFetcher.dispatchNotification(ClientNotifForwarder.java:633)
at
java.management/com.sun.jmx.remote.internal.ClientNotifForwarder$NotifFetcher.doRun(ClientNotifForwarder.java:555)
at
java.management/com.sun.jmx.remote.internal.ClientNotifForwarder$NotifFetcher.run(ClientNotifForwarder.java:474)
at
java.management/com.sun.jmx.remote.internal.ClientNotifForwarder$LinearExecutor.lambda$execute$0(ClientNotifForwarder.java:108)
at java.base/java.lang.Thread.run(Thread.java:829)
Full repair results on another node:
[2023-08-04 20:21:42,575] Repair session
14830280-3304-11ee-939d-635768ac938c for range
[(-5756366402057257951,-5754159509763216479],
(-2469484655657848961,-2461953651636879320],
(-5175468354897450191,-5171107677178073434],
(-628587988891618162,-624346074440106568],
(-6615381309032691143,-6603240846496048854],
(6616005974054228159,6628798414170514490],
(8013321283688199900,8017115978405113835],
(-7829682363035100161,-7824999966028871477],
(2848484090138352114,2852114415040125826],
(-2477015659678818602,-2469484655657848961],
(-2483470805982506865,-2477015659678818602]] finished (progress: 57%)
[2023-08-04 20:36:23,786] Repair session
14cbcb50-3304-11ee-939d-635768ac938c for range
[(5193761311910499374,5197212898580538329],
(-1679246469353274066,-1672836360726470435],
(-6927245454058012407,-6922951496140109663],
(1851771008808005661,1854683726231521039],
(5197212898580538329,5200664485250577285],
(1848858291384490283,1851771008808005661],
(-4736378492502250338,-4732073287189625685],
(-2705389975640427939,-2699099608948332293],
(-7806270378003956741,-7796905583991499373],
(466064862768270626,473304202405656261],
(250549667892224144,253421473349298265],
(-6922951496140109663,-6920804517181158291],
(249113765163687083,250549667892224144],
(1854683726231521039,1857596443655036418],
(4687110928509362134,4694325991399541085],
(-6920804517181158291,-6918657538222206919],
(4399045818626652943,4402968741621424236],
(473304202405656261,480543542043041896]] finished (progress: 57%)
[2023-08-04 20:36:23,795] Repair command #12 finished with error
error: Repair job has failed with the error message: Repair
command #12 failed with error Repair session
154f5330-3304-11ee-939d-635768ac938c for range
[(5333449259855342357,5338449508113440752],
(4959134492108085445,4965331080956982133],
(5938148666505886222,5945280202710590417],
(8428867157147807368,8431880058869458408],
(5338449508113440752,5343449756371539147]] failed with error
[repair #154f5330-3304-11ee-939d-635768ac938c on doc/origdoc,
[(5333449259855342357,5338449508113440752],
(4959134492108085445,4965331080956982133],
(5938148666505886222,5945280202710590417],
(8428867157147807368,8431880058869458408],
(5338449508113440752,5343449756371539147]]] Validation failed in
/172.16.20.16:7000 <http://172.16.20.16:7000>. Check the logs on
the repair participants for further details
-- StackTrace --
java.lang.RuntimeException: Repair job has failed with the error
message: Repair command #12 failed with error Repair session
154f5330-3304-11ee-939d-635768ac938c for range
[(5333449259855342357,5338449508113440752],
(4959134492108085445,4965331080956982133],
(5938148666505886222,5945280202710590417],
(8428867157147807368,8431880058869458408],
(5338449508113440752,5343449756371539147]] failed with error
[repair #154f5330-3304-11ee-939d-635768ac938c on doc/origdoc,
[(5333449259855342357,5338449508113440752],
(4959134492108085445,4965331080956982133],
(5938148666505886222,5945280202710590417],
(8428867157147807368,8431880058869458408],
(5338449508113440752,5343449756371539147]]] Validation failed in
/172.16.20.16:7000 <http://172.16.20.16:7000>. Check the logs on
the repair participants for further details
at
org.apache.cassandra.tools.RepairRunner.progress(RepairRunner.java:137)
at
org.apache.cassandra.utils.progress.jmx.JMXNotificationProgressListener.handleNotification(JMXNotificationProgressListener.java:77)
at
java.management/com.sun.jmx.remote.internal.ClientNotifForwarder$NotifFetcher.dispatchNotification(ClientNotifForwarder.java:633)
at
java.management/com.sun.jmx.remote.internal.ClientNotifForwarder$NotifFetcher.doRun(ClientNotifForwarder.java:555)
at
java.management/com.sun.jmx.remote.internal.ClientNotifForwarder$NotifFetcher.run(ClientNotifForwarder.java:474)
at
java.management/com.sun.jmx.remote.internal.ClientNotifForwarder$LinearExecutor.lambda$execute$0(ClientNotifForwarder.java:108)
at java.base/java.lang.Thread.run(Thread.java:829)
I'm not sure what to do next?
-Joe
On 8/6/2023 8:58 AM, Josh McKenzie wrote:
Quick drive-by observation:
Did not get replies from all endpoints.. Check the
logs on the repair participants for further details
dropping message of type HINT_REQ due to error
org.apache.cassandra.net
<http://org.apache.cassandra.net>.AsyncChannelOutputPlus$FlushException:
The
channel this output stream was writing to has been closed
Caused by: io.netty.channel.unix.Errors$NativeIoException:
writeAddress(..) failed: Connection timed out
java.lang.RuntimeException: Did not get replies from all endpoints.
These all point to the same shaped problem: for whatever reason,
the coordinator of this repair didn't receive replies from the
replicas executing it. Could be that they're dead, could be they
took too long, could be they never got the start message, etc.
Distributed operations are tricky like that.
Logs on the replicas doing the actual repairs should give you
more insight; this is a pretty low level generic set of errors
that basically amounts to "we didn't hear back from the other
participants in time so we timed out."
On Fri, Aug 4, 2023, at 12:02 PM, Surbhi Gupta wrote:
Can you please try to do nodetool describecluster from every
node of the cluster?
One time I noticed issue when nodetool status shows all nodes UN
but describecluster was not.
Thanks
Surbhi
On Fri, Aug 4, 2023 at 8:59 AM Joe Obernberger
<joseph.obernber...@gmail.com> wrote:
Hi All - been using reaper to do repairs, but it has hung.
I tried to run:
nodetool repair -pr
on each of the nodes, but they all fail with some form of
this error:
error: Repair job has failed with the error message: Repair
command #521
failed with error Did not get replies from all endpoints..
Check the
logs on the repair participants for further details
-- StackTrace --
java.lang.RuntimeException: Repair job has failed with the
error
message: Repair command #521 failed with error Did not get
replies from
all endpoints.. Check the logs on the repair participants
for further
details
at
org.apache.cassandra.tools.RepairRunner.progress(RepairRunner.java:137)
at
org.apache.cassandra.utils.progress.jmx.JMXNotificationProgressListener.handleNotification(JMXNotificationProgressListener.java:77)
at
java.management/com.sun.jmx.remote.internal.ClientNotifForwarder$NotifFetcher.dispatchNotification(ClientNotifForwarder.java:633)
at
java.management/com.sun.jmx.remote.internal.ClientNotifForwarder$NotifFetcher.doRun(ClientNotifForwarder.java:555)
at
java.management/com.sun.jmx.remote.internal.ClientNotifForwarder$NotifFetcher.run(ClientNotifForwarder.java:474)
at
java.management/com.sun.jmx.remote.internal.ClientNotifForwarder$LinearExecutor.lambda$execute$0(ClientNotifForwarder.java:108)
at java.base/java.lang.Thread.run(Thread.java:829)
Using version 4.1.2-1
nodetool status
Datacenter: datacenter1
=======================
Status=Up/Down
|/ State=Normal/Leaving/Joining/Moving
-- Address Load Tokens Owns Host
ID Rack
UN 172.16.100.45 505.66 GiB 250 ?
07bccfce-45f1-41a3-a5c4-ee748a7a9b98 rack1
UN 172.16.100.251 380.75 GiB 200 ?
274a6e8d-de37-4e0b-b000-02d221d858a5 rack1
UN 172.16.100.35 479.2 GiB 200 ?
59150c47-274a-46fb-9d5e-bed468d36797 rack1
UN 172.16.100.252 248.69 GiB 200 ?
8f0d392f-0750-44e2-91a5-b30708ade8e4 rack1
UN 172.16.100.249 411.53 GiB 200 ?
49e4f571-7d1c-4e1e-aca7-5bbe076596f7 rack1
UN 172.16.100.38 333.26 GiB 200 ?
0d9509cc-2f23-4117-a883-469a1be54baf rack1
UN 172.16.100.36 405.33 GiB 200 ?
d9702f96-256e-45ae-8e12-69a42712be50 rack1
UN 172.16.100.39 437.74 GiB 200 ?
93f9cb0f-ea71-4e3d-b62a-f0ea0e888c47 rack1
UN 172.16.100.248 344.4 GiB 200 ?
4bbbe57c-6219-41e5-bbac-de92a9594d53 rack1
UN 172.16.100.44 409.36 GiB 200 ?
b2e5366e-8386-40ec-a641-27944a5a7cfa rack1
UN 172.16.100.37 236.08 GiB 120 ?
08a19658-40be-4e55-8709-812b3d4ac750 rack1
UN 172.16.20.16 975 GiB 500 ?
1ccd2cc5-3ee5-43c5-a8c3-7065bdc24297 rack1
UN 172.16.100.34 340.77 GiB 200 ?
352fd049-32f8-4be8-9275-68b145ac2832 rack1
UN 172.16.100.42 974.86 GiB 500 ?
b088a8e6-42f3-4331-a583-47ef5149598f rack1
Note: Non-system keyspaces don't have the same replication
settings,
effective ownership information is meaningless
Debug log has:
DEBUG [ScheduledTasks:1] 2023-08-04 11:56:04,955
MigrationCoordinator.java:264 - Pulling unreceived schema
versions...
INFO [HintsDispatcher:11344] 2023-08-04 11:56:21,369
HintsDispatchExecutor.java:318 - Finished hinted handoff of
file
1ccd2cc5-3ee5-43c5-a8c3-7065bdc24297-1690426370160-2.hints
to endpoint
/172.16.20.16:7000 <http://172.16.20.16:7000>:
1ccd2cc5-3ee5-43c5-a8c3-7065bdc24297, partially
WARN
[Messaging-OUT-/172.16.100.34
<http://172.16.100.34>:7000->/172.16.20.16:7000-LARGE_MESSAGES]
2023-08-04 11:56:21,916 OutboundConnection.java:491 -
/172.16.100.34:7000->/172.16.20.16:7000-LARGE_MESSAGES-[no-channel]
dropping message of type HINT_REQ due to error
org.apache.cassandra.net
<http://org.apache.cassandra.net>.AsyncChannelOutputPlus$FlushException:
The
channel this output stream was writing to has been closed
at
org.apache.cassandra.net
<http://org.apache.cassandra.net>.AsyncChannelOutputPlus.propagateFailedFlush(AsyncChannelOutputPlus.java:200)
at
org.apache.cassandra.net
<http://org.apache.cassandra.net>.AsyncChannelOutputPlus.waitUntilFlushed(AsyncChannelOutputPlus.java:158)
at
org.apache.cassandra.net
<http://org.apache.cassandra.net>.AsyncChannelOutputPlus.waitForSpace(AsyncChannelOutputPlus.java:140)
at
org.apache.cassandra.net
<http://org.apache.cassandra.net>.AsyncChannelOutputPlus.beginFlush(AsyncChannelOutputPlus.java:97)
at
org.apache.cassandra.net
<http://org.apache.cassandra.net>.AsyncMessageOutputPlus.doFlush(AsyncMessageOutputPlus.java:100)
at
org.apache.cassandra.io.util.BufferedDataOutputStreamPlus.write(BufferedDataOutputStreamPlus.java:122)
at
org.apache.cassandra.hints.HintMessage$Serializer.serialize(HintMessage.java:139)
at
org.apache.cassandra.hints.HintMessage$Serializer.serialize(HintMessage.java:77)
at
org.apache.cassandra.net
<http://org.apache.cassandra.net>.Message$Serializer.serializePost40(Message.java:844)
at
org.apache.cassandra.net
<http://org.apache.cassandra.net>.Message$Serializer.serialize(Message.java:702)
at
org.apache.cassandra.net
<http://org.apache.cassandra.net>.OutboundConnection$LargeMessageDelivery.doRun(OutboundConnection.java:984)
at
org.apache.cassandra.net
<http://org.apache.cassandra.net>.OutboundConnection$Delivery.run(OutboundConnection.java:690)
at
org.apache.cassandra.net
<http://org.apache.cassandra.net>.OutboundConnection$LargeMessageDelivery.run(OutboundConnection.java:958)
at
org.apache.cassandra.concurrent.ExecutionFailure$1.run(ExecutionFailure.java:124)
at
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
at
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
at
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
at java.base/java.lang.Thread.run(Thread.java:829)
Caused by: io.netty.channel.unix.Errors$NativeIoException:
writeAddress(..) failed: Connection timed out
INFO [Messaging-EventLoop-3-16] 2023-08-04 11:56:21,918
OutboundConnection.java:1153 -
/172.16.100.34:7000(/172.16.100.34:59198)->/172.16.20.16
<http://172.16.20.16>:7000-LARGE_MESSAGES-2fc2c5b9
successfully connected, version = 12, framing = CRC,
encryption =
unencrypted
ERROR [Repair-Task:437] 2023-08-04 11:56:28,592
RepairRunnable.java:160
- Repair 30675c00-32df-11ee-a7d8-05183c68b0d0 failed:
java.lang.RuntimeException: Did not get replies from all
endpoints.
at
org.apache.cassandra.service.ActiveRepairService.failRepair(ActiveRepairService.java:721)
at
org.apache.cassandra.service.ActiveRepairService.prepareForRepair(ActiveRepairService.java:654)
at
org.apache.cassandra.repair.RepairRunnable.prepare(RepairRunnable.java:400)
at
org.apache.cassandra.repair.RepairRunnable.runMayThrow(RepairRunnable.java:279)
at
org.apache.cassandra.repair.RepairRunnable.run(RepairRunnable.java:248)
at
org.apache.cassandra.concurrent.FutureTask$1.call(FutureTask.java:81)
at
org.apache.cassandra.concurrent.FutureTask.call(FutureTask.java:47)
at
org.apache.cassandra.concurrent.FutureTask.run(FutureTask.java:57)
at
org.apache.cassandra.concurrent.FutureTask$1.call(FutureTask.java:81)
at
org.apache.cassandra.concurrent.FutureTask.call(FutureTask.java:47)
at
org.apache.cassandra.concurrent.FutureTask.run(FutureTask.java:57)
at
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
at
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
at
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
at java.base/java.lang.Thread.run(Thread.java:829)
INFO [Repair-Task:437] 2023-08-04 11:56:28,594
RepairRunnable.java:223
- [repair #30675c00-32df-11ee-a7d8-05183c68b0d0]Repair
command #522
finished with error
What to do?
Thanks!
-Joe
--
This email has been checked for viruses by AVG antivirus
software.
www.avg.com <http://www.avg.com>
<http://www.avg.com/email-signature?utm_medium=email&utm_source=link&utm_campaign=sig-email&utm_content=emailclient>
Virus-free.www.avg.com
<http://www.avg.com/email-signature?utm_medium=email&utm_source=link&utm_campaign=sig-email&utm_content=emailclient>
<#m_266557648173100484_DAB4FAD8-2DD7-40BB-A1B8-4E2AA1F9FDF2>