Is reconfiguring your firewall an option? A stateful firewall really
shouldn't remove a TCP connection in such short time, unless the number
of connections is very large and generally short lived (which often see
in web servers).
On 24/01/2022 13:03, manish khandelwal wrote:
Hi All
Thanks for the suggestions. The issue was *tcp_keepalive_time* has the
default value (7200 seconds). So once the idle connection is broken by
the firewall, the application (Cassandra node) was getting notified
very late. Thus we were seeing one node sending merkle tree and other
not receiving it. Reducing it to 60 solved the problem.
Thanks again for the help.
Regards
Manish
On Sat, Jan 22, 2022 at 12:25 PM C. Scott Andreas
<sc...@paradoxica.net> wrote:
Hi Manish,
I understand this answer is non-specific and might not be the most
helpful, but figured I’d mention — Cassandra 3.11.2 is nearly four
years old and a large number of bugs in repair and other
subsystems have been resolved in the time since.
I’d recommend upgrading to the latest release in the 3.11 series
at minimum (3.11.11). You may find that the issue is resolved; or
if not, be able to draw upon the community’s knowledge of a
current release of the database.
— Scott
On Jan 21, 2022, at 8:51 PM, manish khandelwal
<manishkhandelwa...@gmail.com> wrote:
Hi All
After going through the system.logs, I still see sometimes the
merkle tree is not received from remote DC nodes. Local DC nodes
respond back as soon as they send. But in case of remote DC, it
happens that one or two dcs does not respond.
There is considerable time lag (15-16 minutes) between log
snippet "*Sending completed merkle tree to /10.11.12.123
<http://10.11.12.123> for <tablename>"* seen on remote DC and log
snippet "*Received merkle tree for <tablename> from /10.12.11.231
<http://10.12.11.231>" *seen on node where repair was triggered.
Regards
Manish
On Wed, Jan 19, 2022 at 4:29 PM manish khandelwal
<manishkhandelwa...@gmail.com> wrote:
We use nodetool repair -pr -full. We have scheduled these to
run automatically. For us also it has been seamless on most
of the clusters. This particular node is misbehaving for
reasons unknown to me. As per your suggestion, going through
system.logs to find that unknown. Will keep you posted if am
able to find something.
Regards
Manish
On Wed, Jan 19, 2022 at 4:10 PM Bowen Song <bo...@bso.ng> wrote:
May I ask how do you run the repair? Is it manually via
the nodetool command line tool, or a tool or script, such
as Cassandra Reaper? If you are running the repairs
manually, would you mind give Cassandra Reaper a try?
I have a fairly large cluster under my management, and
last time I tried "nodetool repair -full -pr" on a large
table was maybe 3 years ago, and it randomly stuck (i.e.
it sometimes works fine, sometimes stuck). To finish the
repair, I had to either keep retrying or break down the
token ranges into smaller subsets and use the "-st" and
"-et" parameters. Since then I've switched to use
Cassandra Reaper and have never had similar issues.
On 19/01/2022 02:22, manish khandelwal wrote:
Agree with you on that. Just wanted to highlight that I
am experiencing the same behavior.
Regards
Manish
On Tue, Jan 18, 2022, 22:50 Bowen Song <bo...@bso.ng> wrote:
The link was related to Cassandra 1.2, and it was 9
years ago. Cassandra was full of bugs at that time,
and it has improved a lot since then. For that
reason, I would rather not compare the issue you
have with some 9 years old issues someone else had.
On 18/01/2022 16:11, manish khandelwal wrote:
I am not sure what is happening but it has happened
thrice. It is happening that merkle trees are not
received from nodes of other data center. Getting
issue on similar lines as mentioned here
https://user.cassandra.apache.narkive.com/GTbqO6za/repair-hangs-when-merkle-tree-request-is-not-acknowledged
Regards
Manish
On Tue, Jan 18, 2022, 18:18 Bowen Song
<bo...@bso.ng> wrote:
Keep reading the log on the initiator and the
node sending the merkle tree, anything follows
that? FYI, not all log has the repair ID in it,
therefore please read the relevant logs in the
chronological order without filtering (e.g.
"grep") on the repair ID.
I'm sceptical network issue is causing all
this. The merkle tree is send over TCP
connections, therefore some dropped packets
over a few second of network connectivity issue
occasionally should not cause any issue to the
repair. You should only start to see network
related issues if the network problem persists
over a period of time close to or longer than
the timeout values set in the cassandra.yaml
file, in the case of repair it's the
request_timeout_in_ms which is default to 10
seconds.
Carry on examine the logs, you may find
something useful.
BTW, talking about stuck repair, in my
experience this can happen if two or more
repairs were ran concurrently on the same node
(regardless which node was the initiator)
involving the same table. This could happen if
you accidentally ran "nodetool repair" on two
nodes and both involve the same table, or if
you cancelled and then restarted a "nodetool
repair" on a node without waiting or killing
the remannings of the first repair session on
other nodes.
On 18/01/2022 11:55, manish khandelwal wrote:
In the system logs, on the node where repair
was initiated, I see that the node has
requested merkle tree from all nodes including
itself
INFO [Repair#3:1] 2022-01-14 03:32:18,805
RepairJob.java:172 - *[repair
#6e3385e0-74d1-11ec-8e66-9f084ace9968*]
Requesting merkle trees for *tablename* (to
[*/xyz.abc.def.14, /xyz.abc.def.13,
/xyz.abc.def.12, /xyz.mkn.pq.18,
/xyz.mkn.pq.16, /xyz.mkn.pq.17*])
INFO [AntiEntropyStage:1] 2022-01-14
03:32:18,841 RepairSession.java:180 - [repair
#6e3385e0-74d1-11ec-8e66-9f084ace9968]
Received merkle tree for *tablename* from
*/xyz.mkn.pq.17*
INFO [AntiEntropyStage:1] 2022-01-14
03:32:18,847 RepairSession.java:180 - [repair
#6e3385e0-74d1-11ec-8e66-9f084ace9968]
Received merkle tree for *tablename* from
*/xyz.mkn.pq.16*
INFO [AntiEntropyStage:1] 2022-01-14
03:32:18,851 RepairSession.java:180 - [repair
#6e3385e0-74d1-11ec-8e66-9f084ace9968]
Received merkle tree for *tablename* from
*/xyz.mkn.pq.18*
INFO [AntiEntropyStage:1] 2022-01-14
03:32:18,856 RepairSession.java:180 - [repair
#6e3385e0-74d1-11ec-8e66-9f084ace9968]
Received merkle tree for *tablename* from
*/xyz.abc.def.14*
Line 2480: INFO [AntiEntropyStage:1]
*2022-01-14 03:32:18*,876
RepairSession.java:180 - [*repair
#6e3385e0-74d1-11ec-8e66-9f084ace9968*]
Received merkle tree for *tablename* from
*/xyz.abc.def.12*
*
*
As per the logs merkle tree is not received
from node with ip *xyz.abc.def.13*
*
*
In the system logs of node with ip
*xyz.abc.def.13, *I can see following logs
NFO [AntiEntropyStage:1] *2022-01-14
03:32:18*,850 Validator.java:281 - [*repair
#6e3385e0-74d1-11ec-8e66-9f084ace9968*]
Sending completed merkle tree to */*
*xyz.mkn.pq.17* for *keyspace.tablename*
From the above I inferred that the repair task
has become orphaned since it is waiting for
merkle tree from a node and it is not going to
receive it since it has been lost in the
network somewhere between.
Regards
Manish
On Tue, Jan 18, 2022 at 4:39 PM Bowen Song
<bo...@bso.ng> wrote:
The entry in the debug.log is not specific
to a repair session, and it could also be
caused by reasons other than network
connectivity issue, such as long STW GC
pauses. I usually don't start
troubleshooting an issue from the debug
log, as it can be rather noisy. The
system.log is a better starting point.
If I was to troubleshoot the issue, I
would start from the system logs on the
node that initiated the repair, i.e. the
node you ran the "nodetool repair" command
on. Follow the repair ID (an UUID) in the
logs on all nodes involved in the repair
and read all related logs in chronological
order to find out what exactly had happened.
BTW, If the issue is easily reproducible,
I would re-run the repair with a reduce
scope (such as table and token range) to
get less logs related to the repair
session. Less logs means less time spend
on reading and analysing them.
Hope this helps.
On 18/01/2022 10:03, manish khandelwal wrote:
I have a Cassandra 3.11.2 cluster with
two DCs. While running repair , I am
observing the following behavior.
I am seeing that node is not able to
receive merkle tree from one or two
nodes. Also I am able to see that the
missing nodes did send the merkle tree
but it was not received. This make repair
hangs on consistent basis. In netstats I
can see output as follows
*Mode: NORMAL*
*Not sending any streams. Attempted: 7858888*
*Mismatch (Blocking): 2560*
*Mismatch (Background): 17173*
*Pool Name Active Pending Completed Dropped*
*Large messages n/a 0 6313 3*
*Small messages n/a 0 55978004 3*
*Gossip messages n/a 0 93756 125**Does it
represent network issues? In Debug logs I
saw something*DEBUG
[MessagingService-Outgoing-hostname/xxx.yy.zz.kk-Large]
2022-01-14 05:00:19,031
OutboundTcpConnection.java:349 - Error
writing to hostname/xxx.yy.zz.kk
java.io.IOException: Connection timed out
at sun.nio.ch
<http://sun.nio.ch/>.FileDispatcherImpl.write0(Native
Method) ~[na:1.8.0_221]
at sun.nio.ch
<http://sun.nio.ch/>.SocketDispatcher.write(SocketDispatcher.java:47)
~[na:1.8.0_221]
at sun.nio.ch
<http://sun.nio.ch/>.IOUtil.writeFromNativeBuffer(IOUtil.java:93)
~[na:1.8.0_221]
at sun.nio.ch
<http://sun.nio.ch/>.IOUtil.write(IOUtil.java:65)
~[na:1.8.0_221]
at sun.nio.ch
<http://sun.nio.ch/>.SocketChannelImpl.write(SocketChannelImpl.java:471)
~[na:1.8.0_221]
at
java.nio.channels.Channels.writeFullyImpl(Channels.java:78)
~[na:1.8.0_221]
at
java.nio.channels.Channels.writeFully(Channels.java:98)
~[na:1.8.0_221]
at
java.nio.channels.Channels.access$000(Channels.java:61)
~[na:1.8.0_221]
at
java.nio.channels.Channels$1.write(Channels.java:174)
~[na:1.8.0_221]
at
net.jpountz.lz4.LZ4BlockOutputStream.flushBufferedData(LZ4BlockOutputStream.java:205)
~[lz4-1.3.0.jar:na]
at
net.jpountz.lz4.LZ4BlockOutputStream.write(LZ4BlockOutputStream.java:158)
~[lz4-1.3.0.jar:na] (edited)
Does this show any network fluctuations?
Regards
Manish