You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Jason Brown (JIRA)" <ji...@apache.org> on 2018/09/11 05:02:00 UTC

[jira] [Commented] (CASSANDRA-13938) Default repair is broken, crashes other nodes participating in repair (in trunk)

    [ https://issues.apache.org/jira/browse/CASSANDRA-13938?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16610105#comment-16610105 ] 

Jason Brown commented on CASSANDRA-13938:
-----------------------------------------

[~dimitarndimitrov], Thanks for your comments, and apologies for the late response.

While your proposed simplification indeed clarifies the logic, unfortunately it doesn't resolve the bug (my dtest still fails - this is due to the need to reset a 'some' value, like the currentBufferOffset, after rebufferring). However, your observation about simplifying this patch (in particular eliminate {{currentBufferOffset}} made me reconsider the needs of this class. Basically, we just need to correctly track the streamOffset for the current buffer, and that's all. When I ported this clas from 3.11, I over-complicated the offsets and
 counters into the first version of this class (committed with CASSANDRA-12229), and then confused it again (while resolving the error) with the first patch.

In short: as long as I correctly calculate streamOffset, that should satisfy the needs for the class. Thus, I eliminated both {{current}} and {{currentBufferOffset}}, and the result is clearer and correct.

I've pushed a cleaned up branch (which has been rebased to trunk). Please note that, as with the first patch, the majority of this patch is refactoring to clean up the class in general. I've also updated my dtest patch as my version required a stress profile (based on [~zznate]'s original) to be committed, as well. (Note: my dtest branch also includes [~pauloricardomg]'s patch, but, as before, I'm unable to get that to fail on trunk.)

> Default repair is broken, crashes other nodes participating in repair (in trunk)
> --------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-13938
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-13938
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Repair
>            Reporter: Nate McCall
>            Assignee: Jason Brown
>            Priority: Critical
>             Fix For: 4.x
>
>         Attachments: 13938.yaml, test.sh
>
>
> Running through a simple scenario to test some of the new repair features, I was not able to make a repair command work. Further, the exception seemed to trigger a nasty failure state that basically shuts down the netty connections for messaging *and* CQL on the nodes transferring back data to the node being repaired. The following steps reproduce this issue consistently.
> Cassandra stress profile (probably not necessary, but this one provides a really simple schema and consistent data shape):
> {noformat}
> keyspace: standard_long
> keyspace_definition: |
>   CREATE KEYSPACE standard_long WITH replication = {'class':'SimpleStrategy', 'replication_factor':3};
> table: test_data
> table_definition: |
>   CREATE TABLE test_data (
>   key text,
>   ts bigint,
>   val text,
>   PRIMARY KEY (key, ts)
>   ) WITH COMPACT STORAGE AND
>   CLUSTERING ORDER BY (ts DESC) AND
>   bloom_filter_fp_chance=0.010000 AND
>   caching={'keys':'ALL', 'rows_per_partition':'NONE'} AND
>   comment='' AND
>   dclocal_read_repair_chance=0.000000 AND
>   gc_grace_seconds=864000 AND
>   read_repair_chance=0.000000 AND
>   compaction={'class': 'SizeTieredCompactionStrategy'} AND
>   compression={'sstable_compression': 'LZ4Compressor'};
> columnspec:
>   - name: key
>     population: uniform(1..50000000) # 50 million records available
>   - name: ts
>     cluster: gaussian(1..50) # Up to 50 inserts per record
>   - name: val
>     population: gaussian(128..1024) # varrying size of value data
> insert:
>   partitions: fixed(1) # only one insert per batch for individual partitions
>   select: fixed(1)/1 # each insert comes in one at a time
>   batchtype: UNLOGGED
> queries:
>   single:
>     cql: select * from test_data where key = ? and ts = ? limit 1;
>   series:
>     cql: select key,ts,val from test_data where key = ? limit 10;
> {noformat}
> The commands to build and run:
> {noformat}
> ccm create 4_0_test -v git:trunk -n 3 -s
> ccm stress user profile=./histo-test-schema.yml ops\(insert=20,single=1,series=1\) duration=15s -rate threads=4
> # flush the memtable just to get everything on disk
> ccm node1 nodetool flush
> ccm node2 nodetool flush
> ccm node3 nodetool flush
> # disable hints for nodes 2 and 3
> ccm node2 nodetool disablehandoff
> ccm node3 nodetool disablehandoff
> # stop node1
> ccm node1 stop
> ccm stress user profile=./histo-test-schema.yml ops\(insert=20,single=1,series=1\) duration=45s -rate threads=4
> # wait 10 seconds
> ccm node1 start
> # Note that we are local to ccm's nodetool install 'cause repair preview is not reported yet
> node1/bin/nodetool repair --preview
> node1/bin/nodetool repair standard_long test_data
> {noformat} 
> The error outputs from the last repair command follow. First, this is stdout from node1:
> {noformat}
> $ node1/bin/nodetool repair standard_long test_data
> objc[47876]: Class JavaLaunchHelper is implemented in both /Library/Java/JavaVirtualMachines/jdk1.8.0_101.jdk/Contents/Home/bin/java (0x10274d4c0) and /Library/Java/JavaVirtualMachines/jdk1.8.0_101.jdk/Contents/Home/jre/lib/libinstrument.dylib (0x1047b64e0). One of the two will be used. Which one is undefined.
> [2017-10-05 14:31:52,425] Starting repair command #4 (7e1a9150-a98e-11e7-ad86-cbd2801b8de2), repairing keyspace standard_long with repair options (parallelism: parallel, primary range: false, incremental: true, job threads: 1, ColumnFamilies: [test_data], dataCenters: [], hosts: [], previewKind: NONE, # of ranges: 3, pull repair: false, force repair: false)
> [2017-10-05 14:32:07,045] Repair session 7e2e8e80-a98e-11e7-ad86-cbd2801b8de2 for range [(3074457345618258602,-9223372036854775808], (-9223372036854775808,-3074457345618258603], (-3074457345618258603,3074457345618258602]] failed with error Stream failed
> [2017-10-05 14:32:07,048] null
> [2017-10-05 14:32:07,050] Repair command #4 finished in 14 seconds
> error: Repair job has failed with the error message: [2017-10-05 14:32:07,048] null
> -- StackTrace --
> java.lang.RuntimeException: Repair job has failed with the error message: [2017-10-05 14:32:07,048] null
>     at org.apache.cassandra.tools.RepairRunner.progress(RepairRunner.java:122)
>     at org.apache.cassandra.utils.progress.jmx.JMXNotificationProgressListener.handleNotification(JMXNotificationProgressListener.java:77)
>     at com.sun.jmx.remote.internal.ClientNotifForwarder$NotifFetcher.dispatchNotification(ClientNotifForwarder.java:583)
>     at com.sun.jmx.remote.internal.ClientNotifForwarder$NotifFetcher.doRun(ClientNotifForwarder.java:533)
>     at com.sun.jmx.remote.internal.ClientNotifForwarder$NotifFetcher.run(ClientNotifForwarder.java:452)
>     at com.sun.jmx.remote.internal.ClientNotifForwarder$LinearExecutor$1.run(ClientNotifForwarder.java:108)
> {noformat}
> node1's {{system.log}}:
> {noformat}
> INFO  [Stream-Deserializer-/127.0.0.2:63069-e0af297f] 2017-10-05 14:32:07,037 StreamResultFuture.java:193 - [Stream #85d4b790-a98e-11e7-ad86-cbd2801b8de2] Session with /127.0.0.2 is complete
> INFO  [Stream-Deserializer-/127.0.0.3:63068-eb8f23bc] 2017-10-05 14:32:07,037 StreamResultFuture.java:193 - [Stream #85d3f440-a98e-11e7-ad86-cbd2801b8de2] Session with /127.0.0.3 is complete
> ERROR [Streaming-Netty-Thread-5-5] 2017-10-05 14:32:07,037 StreamSession.java:617 - [Stream #85d3f440-a98e-11e7-ad86-cbd2801b8de2] Streaming error occurred on session with peer 127.0.0.3
> java.nio.channels.ClosedChannelException: null
>         at io.netty.channel.AbstractChannel$AbstractUnsafe.write(...)(Unknown Source) ~[netty-all-4.1.14.Final.jar:4.1.14.Final]
> ERROR [Streaming-Netty-Thread-5-7] 2017-10-05 14:32:07,038 StreamSession.java:617 - [Stream #85d4b790-a98e-11e7-ad86-cbd2801b8de2] Streaming error occurred on session with peer 127.0.0.2
> java.nio.channels.ClosedChannelException: null
>         at io.netty.channel.AbstractChannel$AbstractUnsafe.write(...)(Unknown Source) ~[netty-all-4.1.14.Final.jar:4.1.14.Final]
> WARN  [Stream-Deserializer-/127.0.0.2:63069-e0af297f] 2017-10-05 14:32:07,038 StreamResultFuture.java:220 - [Stream #85d4b790-a98e-11e7-ad86-cbd2801b8de2] Stream failed
> WARN  [Stream-Deserializer-/127.0.0.3:63068-eb8f23bc] 2017-10-05 14:32:07,038 StreamResultFuture.java:220 - [Stream #85d3f440-a98e-11e7-ad86-cbd2801b8de2] Stream failed
> WARN  [RepairJobTask:1] 2017-10-05 14:32:07,038 RepairJob.java:176 - [repair #7e2e8e80-a98e-11e7-ad86-cbd2801b8de2] test_data sync failed
> ERROR [Stream-Deserializer-/127.0.0.3:7000-48246b87] 2017-10-05 14:32:07,041 StreamSession.java:757 - [Stream #85d3f440-a98e-11e7-ad86-cbd2801b8de2] Remote peer 127.0.0.3 failed stream session.
> ERROR [RepairJobTask:1] 2017-10-05 14:32:07,042 RepairSession.java:326 - [repair #7e2e8e80-a98e-11e7-ad86-cbd2801b8de2] Session completed with the following error
> org.apache.cassandra.streaming.StreamException: Stream failed
>         at org.apache.cassandra.streaming.management.StreamEventJMXNotifier.onFailure(StreamEventJMXNotifier.java:88) ~[main/:na]
>         at com.google.common.util.concurrent.Futures$6.run(Futures.java:1310) ~[guava-18.0.jar:na]
>         at com.google.common.util.concurrent.MoreExecutors$DirectExecutor.execute(MoreExecutors.java:457) ~[guava-18.0.jar:na]
>         at com.google.common.util.concurrent.ExecutionList.executeListener(ExecutionList.java:156) ~[guava-18.0.jar:na]
>         at com.google.common.util.concurrent.ExecutionList.execute(ExecutionList.java:145) ~[guava-18.0.jar:na]
>         at com.google.common.util.concurrent.AbstractFuture.setException(AbstractFuture.java:202) ~[guava-18.0.jar:na]
>         at org.apache.cassandra.streaming.StreamResultFuture.maybeComplete(StreamResultFuture.java:221) ~[main/:na]
>         at org.apache.cassandra.streaming.StreamResultFuture.handleSessionComplete(StreamResultFuture.java:197) ~[main/:na]
>         at org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:488) ~[main/:na]
>         at org.apache.cassandra.streaming.StreamSession.onError(StreamSession.java:601) ~[main/:na]
>         at org.apache.cassandra.streaming.async.StreamingInboundHandler$StreamDeserializingTask.run(StreamingInboundHandler.java:207) ~[main/:na]
>         at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_101]
> ERROR [RepairJobTask:1] 2017-10-05 14:32:07,043 RepairRunnable.java:564 - Repair session 7e2e8e80-a98e-11e7-ad86-cbd2801b8de2 for range [(3074457345618258602,-9223372036854775808], (-9223372036854775808,-3074457345618258603], (-3074457345618258603,3074457345618258602]] failed with error Stream failed
> org.apache.cassandra.streaming.StreamException: Stream failed
>         at org.apache.cassandra.streaming.management.StreamEventJMXNotifier.onFailure(StreamEventJMXNotifier.java:88) ~[main/:na]
>         at com.google.common.util.concurrent.Futures$6.run(Futures.java:1310) ~[guava-18.0.jar:na]
>         at com.google.common.util.concurrent.MoreExecutors$DirectExecutor.execute(MoreExecutors.java:457) ~[guava-18.0.jar:na]
>         at com.google.common.util.concurrent.ExecutionList.executeListener(ExecutionList.java:156) ~[guava-18.0.jar:na]
>         at com.google.common.util.concurrent.ExecutionList.execute(ExecutionList.java:145) ~[guava-18.0.jar:na]
>         at com.google.common.util.concurrent.AbstractFuture.setException(AbstractFuture.java:202) ~[guava-18.0.jar:na]
>         at org.apache.cassandra.streaming.StreamResultFuture.maybeComplete(StreamResultFuture.java:221) ~[main/:na]
>         at org.apache.cassandra.streaming.StreamResultFuture.handleSessionComplete(StreamResultFuture.java:197) ~[main/:na]
>         at org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:488) ~[main/:na]
>         at org.apache.cassandra.streaming.StreamSession.onError(StreamSession.java:601) ~[main/:na]
>         at org.apache.cassandra.streaming.async.StreamingInboundHandler$StreamDeserializingTask.run(StreamingInboundHandler.java:207) ~[main/:na]
>         at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_101]
> INFO  [RepairJobTask:1] 2017-10-05 14:32:07,045 CoordinatorSession.java:233 - Incremental repair session 7e1a9150-a98e-11e7-ad86-cbd2801b8de2 failed
> ERROR [Stream-Deserializer-/127.0.0.2:7000-4b83e3cb] 2017-10-05 14:32:07,045 StreamSession.java:757 - [Stream #85d4b790-a98e-11e7-ad86-cbd2801b8de2] Remote peer 127.0.0.2 failed stream session.
> INFO  [AntiEntropyStage:1] 2017-10-05 14:32:07,048 CoordinatorSession.java:233 - Incremental repair session 7e1a9150-a98e-11e7-ad86-cbd2801b8de2 failed
> INFO  [AntiEntropyStage:1] 2017-10-05 14:32:07,049 LocalSessions.java:501 - Failing local repair session 7e1a9150-a98e-11e7-ad86-cbd2801b8de2
> INFO  [RepairJobTask:1] 2017-10-05 14:32:07,049 RepairRunnable.java:647 - Repair command #4 finished in 14 seconds
> {noformat}
> node2's {{system.log}} (note the transport shutdowns at the end):
> {noformat}
> INFO  [AntiEntropyStage:1] 2017-10-05 18:31:52,521 LocalSessions.java:560 - Beginning local incremental repair session LocalSession{sessionID=7e1a9150-a98e-11e7-ad86-cbd2801b8de2, state=PREPARING, coordinator=/127.0.0.1, tableIds=[99d53860-a98d-11e7-9807-39cb3e573e5c], repairedAt=1507181512483, ranges=[(3074457345618258602,-9223372036854775808], (-9223372036854775808,-3074457345618258603], (-3074457345618258603,3074457345618258602]], participants=[/127.0.0.1, /127.0.0.2, /127.0.0.3], startedAt=1507181512, lastUpdate=1507181512}
> INFO  [CompactionExecutor:224] 2017-10-05 18:31:52,539 CompactionManager.java:642 - [repair #7e1a9150-a98e-11e7-ad86-cbd2801b8de2] Starting anticompaction for standard_long.test_data on 2/2 sstables
> INFO  [CompactionExecutor:224] 2017-10-05 18:31:52,539 CompactionManager.java:664 - [repair #7e1a9150-a98e-11e7-ad86-cbd2801b8de2] SSTable BigTableReader(path='/Users/zznate/.ccm/4_0_test/node2/data0/standard_long/test_data-99d53860a98d11e7980739cb3e573e5c/na-27-big-Data.db') fully contained in range (-9223372036854775808,-9223372036854775808], mutating repairedAt instead of anticompacting
> INFO  [CompactionExecutor:224] 2017-10-05 18:31:52,539 CompactionManager.java:664 - [repair #7e1a9150-a98e-11e7-ad86-cbd2801b8de2] SSTable BigTableReader(path='/Users/zznate/.ccm/4_0_test/node2/data0/standard_long/test_data-99d53860a98d11e7980739cb3e573e5c/na-26-big-Data.db') fully contained in range (-9223372036854775808,-9223372036854775808], mutating repairedAt instead of anticompacting
> INFO  [CompactionExecutor:224] 2017-10-05 18:31:52,547 CompactionManager.java:699 - [repair #7e1a9150-a98e-11e7-ad86-cbd2801b8de2] Completed anticompaction successfully
> INFO  [AntiEntropyStage:1] 2017-10-05 18:31:57,500 Validator.java:292 - [repair #7e2e8e80-a98e-11e7-ad86-cbd2801b8de2] Sending completed merkle tree to /127.0.0.1 for standard_long.test_data
> INFO  [Stream-Deserializer-/127.0.0.1:63064-3a39d969] 2017-10-05 18:32:05,417 StreamResultFuture.java:115 - [Stream #85d4b790-a98e-11e7-ad86-cbd2801b8de2 ID#0] Creating new streaming plan for Repair
> INFO  [Stream-Deserializer-/127.0.0.1:63064-3a39d969] 2017-10-05 18:32:05,418 StreamResultFuture.java:122 - [Stream #85d4b790-a98e-11e7-ad86-cbd2801b8de2, ID#0] Received streaming plan for Repair
> INFO  [NonPeriodicTasks:1] 2017-10-05 18:32:05,856 StreamResultFuture.java:179 - [Stream #85d4b790-a98e-11e7-ad86-cbd2801b8de2 ID#0] Prepare completed. Receiving 1 files(8.136MiB), sending 2 files(42.689MiB)
> INFO  [Stream-Deserializer-/127.0.0.1:63064-3a39d969] 2017-10-05 18:32:06,625 StreamResultFuture.java:179 - [Stream #85d4b790-a98e-11e7-ad86-cbd2801b8de2 ID#0] Prepare completed. Receiving 1 files(8.136MiB), sending 2 files(42.689MiB)
> WARN  [Stream-Deserializer-/127.0.0.1:63066-c7002e89] 2017-10-05 18:32:06,747 CompressedStreamReader.java:112 - [Stream 85d4b790-a98e-11e7-ad86-cbd2801b8de2] Error while reading partition DecoratedKey(-9060243433852736644, 5f1c6c5d747c) from stream on ks='standard_long' and table='test_data'.
> ERROR [Stream-Deserializer-/127.0.0.1:63066-c7002e89] 2017-10-05 18:32:06,759 StreamSession.java:617 - [Stream #85d4b790-a98e-11e7-ad86-cbd2801b8de2] Streaming error occurred on session with peer 127.0.0.1
> org.apache.cassandra.streaming.StreamReceiveException: java.lang.AssertionError: stream can only read forward.
>         at org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:63) ~[main/:na]
>         at org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:41) ~[main/:na]
>         at org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:55) ~[main/:na]
>         at org.apache.cassandra.streaming.async.StreamingInboundHandler$StreamDeserializingTask.run(StreamingInboundHandler.java:178) ~[main/:na]
>         at java.lang.Thread.run(Thread.java:745) [na:1.8.0_101]
> Caused by: java.lang.AssertionError: stream can only read forward.
>         at org.apache.cassandra.streaming.compress.CompressedInputStream.position(CompressedInputStream.java:108) ~[main/:na]
>         at org.apache.cassandra.streaming.compress.CompressedStreamReader.read(CompressedStreamReader.java:96) ~[main/:na]
>         at org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:58) ~[main/:na]
>         ... 4 common frames omitted
> INFO  [Stream-Deserializer-/127.0.0.1:63066-c7002e89] 2017-10-05 18:32:06,761 StreamResultFuture.java:193 - [Stream #85d4b790-a98e-11e7-ad86-cbd2801b8de2] Session with /127.0.0.1 is complete
> WARN  [Stream-Deserializer-/127.0.0.1:63066-c7002e89] 2017-10-05 18:32:06,762 StreamResultFuture.java:220 - [Stream #85d4b790-a98e-11e7-ad86-cbd2801b8de2] Stream failed
> ERROR [NettyStreaming-Outbound-/127.0.0.1:1] 2017-10-05 18:32:06,765 CassandraDaemon.java:211 - Exception in thread Thread[NettyStreaming-Outbound-/127.0.0.1:1,5,main]
> org.apache.cassandra.io.FSReadError: java.nio.channels.ClosedByInterruptException
>         at org.apache.cassandra.io.util.ChannelProxy.read(ChannelProxy.java:133) ~[main/:na]
>         at org.apache.cassandra.streaming.compress.CompressedStreamWriter.write(CompressedStreamWriter.java:94) ~[main/:na]
>         at org.apache.cassandra.streaming.messages.OutgoingFileMessage.serialize(OutgoingFileMessage.java:111) ~[main/:na]
>         at org.apache.cassandra.streaming.messages.OutgoingFileMessage$1.serialize(OutgoingFileMessage.java:53) ~[main/:na]
>         at org.apache.cassandra.streaming.messages.OutgoingFileMessage$1.serialize(OutgoingFileMessage.java:42) ~[main/:na]
>         at org.apache.cassandra.streaming.messages.StreamMessage.serialize(StreamMessage.java:41) ~[main/:na]
>         at org.apache.cassandra.streaming.async.NettyStreamingMessageSender$FileStreamTask.run(NettyStreamingMessageSender.java:324) ~[main/:na]
>         at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[na:1.8.0_101]
>         at java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[na:1.8.0_101]
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) ~[na:1.8.0_101]
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_101]
>         at org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81) [main/:na]
>         at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_101]
> Caused by: java.nio.channels.ClosedByInterruptException: null
>         at java.nio.channels.spi.AbstractInterruptibleChannel.end(AbstractInterruptibleChannel.java:202) ~[na:1.8.0_101]
>         at sun.nio.ch.FileChannelImpl.readInternal(FileChannelImpl.java:746) ~[na:1.8.0_101]
>         at sun.nio.ch.FileChannelImpl.read(FileChannelImpl.java:727) ~[na:1.8.0_101]
>         at org.apache.cassandra.io.util.ChannelProxy.read(ChannelProxy.java:129) ~[main/:na]
>         ... 12 common frames omitted
> ERROR [NettyStreaming-Outbound-/127.0.0.1:1] 2017-10-05 18:32:06,769 StorageService.java:393 - Stopping gossiper
> WARN  [NettyStreaming-Outbound-/127.0.0.1:1] 2017-10-05 18:32:06,769 StorageService.java:315 - Stopping gossip by operator request
> INFO  [NettyStreaming-Outbound-/127.0.0.1:1] 2017-10-05 18:32:06,769 Gossiper.java:1527 - Announcing shutdown
> INFO  [NettyStreaming-Outbound-/127.0.0.1:1] 2017-10-05 18:32:06,770 StorageService.java:2202 - Node /127.0.0.2 state jump to shutdown
> INFO  [AntiEntropyStage:1] 2017-10-05 18:32:07,049 LocalSessions.java:501 - Failing local repair session 7e1a9150-a98e-11e7-ad86-cbd2801b8de2
> ERROR [NettyStreaming-Outbound-/127.0.0.1:1] 2017-10-05 18:32:08,771 StorageService.java:398 - Stopping native transport
> INFO  [NettyStreaming-Outbound-/127.0.0.1:1] 2017-10-05 18:32:08,774 Server.java:180 - Stop listening for CQL clients
> {noformat}
> And node3 {{system.log}} (similar to node2):
> {noformat}
> INFO  [AntiEntropyStage:1] 2017-10-05 18:31:52,521 LocalSessions.java:560 - Beginning local incremental repair session LocalSession{sessionID=7e1a9150-a98e-11e7-ad86-cbd2801b8de2, state=PREPARING, coordinator=/127.0.0.1, tableIds=[99d53860-a98d-11e7-9807-39cb3e573e5c], repairedAt=1507181512483, ranges=[(3074457345618258602,-9223372036854775808], (-9223372036854775808,-3074457345618258603], (-3074457345618258603,3074457345618258602]], participants=[/127.0.0.1, /127.0.0.2, /127.0.0.3], startedAt=1507181512, lastUpdate=1507181512}
> INFO  [CompactionExecutor:249] 2017-10-05 18:31:52,542 CompactionManager.java:642 - [repair #7e1a9150-a98e-11e7-ad86-cbd2801b8de2] Starting anticompaction for standard_long.test_data on 2/2 sstables
> INFO  [CompactionExecutor:249] 2017-10-05 18:31:52,543 CompactionManager.java:664 - [repair #7e1a9150-a98e-11e7-ad86-cbd2801b8de2] SSTable BigTableReader(path='/Users/zznate/.ccm/4_0_test/node3/data0/standard_long/test_data-99d53860a98d11e7980739cb3e573e5c/na-27-big-Data.db') fully contained in range (-9223372036854775808,-9223372036854775808], mutating repairedAt instead of anticompacting
> INFO  [CompactionExecutor:249] 2017-10-05 18:31:52,543 CompactionManager.java:664 - [repair #7e1a9150-a98e-11e7-ad86-cbd2801b8de2] SSTable BigTableReader(path='/Users/zznate/.ccm/4_0_test/node3/data0/standard_long/test_data-99d53860a98d11e7980739cb3e573e5c/na-26-big-Data.db') fully contained in range (-9223372036854775808,-9223372036854775808], mutating repairedAt instead of anticompacting
> INFO  [CompactionExecutor:249] 2017-10-05 18:31:52,550 CompactionManager.java:699 - [repair #7e1a9150-a98e-11e7-ad86-cbd2801b8de2] Completed anticompaction successfully
> INFO  [AntiEntropyStage:1] 2017-10-05 18:31:57,918 Validator.java:292 - [repair #7e2e8e80-a98e-11e7-ad86-cbd2801b8de2] Sending completed merkle tree to /127.0.0.1 for standard_long.test_data
> INFO  [Stream-Deserializer-/127.0.0.1:63063-d6987513] 2017-10-05 18:32:05,817 StreamResultFuture.java:115 - [Stream #85d3f440-a98e-11e7-ad86-cbd2801b8de2 ID#0] Creating new streaming plan for Repair
> INFO  [Stream-Deserializer-/127.0.0.1:63063-d6987513] 2017-10-05 18:32:05,818 StreamResultFuture.java:122 - [Stream #85d3f440-a98e-11e7-ad86-cbd2801b8de2, ID#0] Received streaming plan for Repair
> INFO  [NonPeriodicTasks:1] 2017-10-05 18:32:05,866 StreamResultFuture.java:179 - [Stream #85d3f440-a98e-11e7-ad86-cbd2801b8de2 ID#0] Prepare completed. Receiving 1 files(8.136MiB), sending 2 files(42.679MiB)
> INFO  [Stream-Deserializer-/127.0.0.1:63063-d6987513] 2017-10-05 18:32:06,622 StreamResultFuture.java:179 - [Stream #85d3f440-a98e-11e7-ad86-cbd2801b8de2 ID#0] Prepare completed. Receiving 1 files(8.136MiB), sending 2 files(42.679MiB)
> WARN  [Stream-Deserializer-/127.0.0.1:63067-6347c9a8] 2017-10-05 18:32:06,759 CompressedStreamReader.java:112 - [Stream 85d3f440-a98e-11e7-ad86-cbd2801b8de2] Error while reading partition DecoratedKey(-9060243433852736644, 5f1c6c5d747c) from stream on ks='standard_long' and table='test_data'.
> ERROR [Stream-Deserializer-/127.0.0.1:63067-6347c9a8] 2017-10-05 18:32:06,773 StreamSession.java:617 - [Stream #85d3f440-a98e-11e7-ad86-cbd2801b8de2] Streaming error occurred on session with peer 127.0.0.1
> org.apache.cassandra.streaming.StreamReceiveException: java.lang.AssertionError: stream can only read forward.
>         at org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:63) ~[main/:na]
>         at org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:41) ~[main/:na]
>         at org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:55) ~[main/:na]
>         at org.apache.cassandra.streaming.async.StreamingInboundHandler$StreamDeserializingTask.run(StreamingInboundHandler.java:178) ~[main/:na]
>         at java.lang.Thread.run(Thread.java:745) [na:1.8.0_101]
> Caused by: java.lang.AssertionError: stream can only read forward.
>         at org.apache.cassandra.streaming.compress.CompressedInputStream.position(CompressedInputStream.java:108) ~[main/:na]
>         at org.apache.cassandra.streaming.compress.CompressedStreamReader.read(CompressedStreamReader.java:96) ~[main/:na]
>         at org.apache.cassandra.streaming.messages.IncomingFileMessage$1.deserialize(IncomingFileMessage.java:58) ~[main/:na]
>         ... 4 common frames omitted
> INFO  [GossipStage:1] 2017-10-05 18:32:06,774 Gossiper.java:1040 - InetAddress /127.0.0.2 is now DOWN
> INFO  [Stream-Deserializer-/127.0.0.1:63067-6347c9a8] 2017-10-05 18:32:06,775 StreamResultFuture.java:193 - [Stream #85d3f440-a98e-11e7-ad86-cbd2801b8de2] Session with /127.0.0.1 is complete
> WARN  [Stream-Deserializer-/127.0.0.1:63067-6347c9a8] 2017-10-05 18:32:06,775 StreamResultFuture.java:220 - [Stream #85d3f440-a98e-11e7-ad86-cbd2801b8de2] Stream failed
> ERROR [NettyStreaming-Outbound-/127.0.0.1:1] 2017-10-05 18:32:06,778 CassandraDaemon.java:211 - Exception in thread Thread[NettyStreaming-Outbound-/127.0.0.1:1,5,main]
> org.apache.cassandra.io.FSReadError: java.nio.channels.ClosedByInterruptException
>         at org.apache.cassandra.io.util.ChannelProxy.read(ChannelProxy.java:133) ~[main/:na]
>         at org.apache.cassandra.streaming.compress.CompressedStreamWriter.write(CompressedStreamWriter.java:94) ~[main/:na]
>         at org.apache.cassandra.streaming.messages.OutgoingFileMessage.serialize(OutgoingFileMessage.java:111) ~[main/:na]
>         at org.apache.cassandra.streaming.messages.OutgoingFileMessage$1.serialize(OutgoingFileMessage.java:53) ~[main/:na]
>         at org.apache.cassandra.streaming.messages.OutgoingFileMessage$1.serialize(OutgoingFileMessage.java:42) ~[main/:na]
>         at org.apache.cassandra.streaming.messages.StreamMessage.serialize(StreamMessage.java:41) ~[main/:na]
>         at org.apache.cassandra.streaming.async.NettyStreamingMessageSender$FileStreamTask.run(NettyStreamingMessageSender.java:324) ~[main/:na]
>         at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[na:1.8.0_101]
>         at java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[na:1.8.0_101]
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) ~[na:1.8.0_101]
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_101]
>         at org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81) [main/:na]
>         at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_101]
> Caused by: java.nio.channels.ClosedByInterruptException: null
>         at java.nio.channels.spi.AbstractInterruptibleChannel.end(AbstractInterruptibleChannel.java:202) ~[na:1.8.0_101]
>         at sun.nio.ch.FileChannelImpl.readInternal(FileChannelImpl.java:746) ~[na:1.8.0_101]
>         at sun.nio.ch.FileChannelImpl.read(FileChannelImpl.java:727) ~[na:1.8.0_101]
>         at org.apache.cassandra.io.util.ChannelProxy.read(ChannelProxy.java:129) ~[main/:na]
>         ... 12 common frames omitted
> ERROR [NettyStreaming-Outbound-/127.0.0.1:1] 2017-10-05 18:32:06,781 StorageService.java:393 - Stopping gossiper
> WARN  [NettyStreaming-Outbound-/127.0.0.1:1] 2017-10-05 18:32:06,781 StorageService.java:315 - Stopping gossip by operator request
> INFO  [NettyStreaming-Outbound-/127.0.0.1:1] 2017-10-05 18:32:06,781 Gossiper.java:1527 - Announcing shutdown
> INFO  [NettyStreaming-Outbound-/127.0.0.1:1] 2017-10-05 18:32:06,782 StorageService.java:2202 - Node /127.0.0.3 state jump to shutdown
> INFO  [AntiEntropyStage:1] 2017-10-05 18:32:07,049 LocalSessions.java:501 - Failing local repair session 7e1a9150-a98e-11e7-ad86-cbd2801b8de2
> ERROR [NettyStreaming-Outbound-/127.0.0.1:1] 2017-10-05 18:32:08,782 StorageService.java:398 - Stopping native transport
> INFO  [NettyStreaming-Outbound-/127.0.0.1:1] 2017-10-05 18:32:08,785 Server.java:180 - Stop listening for CQL clients
> {noformat}
> The final state of the cluster after running this repair command:
> {noformat}
> $ ccm node1 nodetool status
> Datacenter: datacenter1
> =======================
> Status=Up/Down
> |/ State=Normal/Leaving/Joining/Moving
> --  Address    Load       Tokens       Owns (effective)  Host ID                               Rack
> UN  127.0.0.1  8.62 MiB   1            100.0%            ffe7466b-2937-4322-a388-cca1819f6513  rack1
> DN  127.0.0.2  44.54 MiB  1            100.0%            e374f662-1da5-477d-b1fb-173b8311c4a9  rack1
> DN  127.0.0.3  44.53 MiB  1            100.0%            d8d99bd6-4b9f-4510-a4c3-62951be1b4d2  rack1
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org