You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@cassandra.apache.org by Yuji Ito <yu...@imagine-orb.com> on 2016/11/16 08:25:40 UTC

Does recovery continue after truncating a table?

Hi,

I could find stale data after truncating a table.
It seems that truncating starts while recovery is being executed just after
a node restarts.
After the truncating finishes, recovery still continues?
Is it expected?

I use C* 2.2.8 and can reproduce it as below.

==== [create table] ====
cqlsh $ip -e "drop keyspace testdb;"
cqlsh $ip -e "CREATE KEYSPACE testdb WITH replication = {'class':
'SimpleStrategy', 'replication_factor': '3'};"
cqlsh $ip -e "CREATE TABLE testdb.testtbl (key int PRIMARY KEY, val int);"

==== [script] ====
#!/bin/sh

node1_ip=<node1 IP address>
node2_ip=<node2 IP address>
node3_ip=<node3 IP address>
node3_user=<user name>
rows=10000

echo "consistency quorum;" > init_data.cql
for key in $(seq 0 $(expr $rows - 1))
do
    echo "insert into testdb.testtbl (key, val) values($key, 1111) IF NOT
EXISTS;" >> init_data.cql
done

while true
do
echo "truncate the table"
cqlsh $node1_ip -e "truncate table testdb.testtbl"
if [ $? -ne 0 ]; then
    echo "truncating failed"
    continue
else
    break
fi
done

echo "kill C* process on node3"
pdsh -l $node3_user -R ssh -w $node3_ip "ps auxww | grep CassandraDaemon |
awk '{if (\$13 ~ /cassand/) print \$2}' | xargs sudo kill -9"

echo "insert $rows rows"
cqlsh $node1_ip -f init_data.cql > insert_log 2>&1

echo "restart C* process on node3"
pdsh -l $node3_user -R ssh -w $node3_ip "sudo /etc/init.d/cassandra start"

while true
do
echo "truncate the table again"
cqlsh $node1_ip -e "truncate table testdb.testtbl"
if [ $? -ne 0 ]; then
    echo "truncating failed"
    continue
else
    break
fi
done

cqlsh $node1_ip --request-timeout 3600 -e "consistency serial; select
count(*) from testdb.testtbl;"
sleep 10
cqlsh $node1_ip --request-timeout 3600 -e "consistency serial; select
count(*) from testdb.testtbl;"


==== [result] ====
truncate the table
kill C* process on node3
insert 10000 rows
restart C* process on node3
10.91.145.27: Starting Cassandra: OK
truncate the table again
<stdin>:1:TruncateError: Error during truncate: Cannot achieve consistency
level ALL
truncating failed
truncate the table again
<stdin>:1:TruncateError: Error during truncate: Cannot achieve consistency
level ALL
truncating failed
truncate the table again
<stdin>:1:TruncateError: Error during truncate: Cannot achieve consistency
level ALL
truncating failed
truncate the table again
<stdin>:1:TruncateError: Error during truncate: Cannot achieve consistency
level ALL
truncating failed
truncate the table again
<stdin>:1:TruncateError: Error during truncate: Cannot achieve consistency
level ALL
truncating failed
truncate the table again
<stdin>:1:TruncateError: Error during truncate: Cannot achieve consistency
level ALL
truncating failed
truncate the table again
Consistency level set to SERIAL.

 count
-------
   300

(1 rows)

Warnings :
Aggregation query used without partition key

Consistency level set to SERIAL.

 count
-------
  2304

(1 rows)

Warnings :
Aggregation query used without partition key
====

I found it when I was investigating data lost problem. (Ref. "failure node
rejoin" thread)
I'm not sure this problem is related to data lost.

Thanks,
yuji

Re: Does recovery continue after truncating a table?

Posted by Yuji Ito <yu...@imagine-orb.com>.
Thanks Ben and Hiro,

I've reported it at https://issues.apache.org/jira/browse/CASSANDRA-12960.

I'll use `truncatehints` or DROP command after this.


On Sun, Nov 27, 2016 at 12:33 PM, Ben Slater <be...@instaclustr.com>
wrote:

> By “undocumented limitation”, I meant “TRUNCATE” is mainly only used in
> development and testing, not production scenarios so a sufficient fix (and
> certainly a better than nothing fix) might be just to document that if you
> issue a TRUNCATE while there are still hinted hand-offs pending the hinted
> hand-offs replayed after the truncate will come back to life. Of course, an
> actual fix would be better.
>
> Cheers
> Ben
>
> On Sat, 26 Nov 2016 at 21:08 Hiroyuki Yamada <mo...@gmail.com> wrote:
>
>> Hi Yuji and Ben,
>>
>> I tried out this revised script and the same issue occurred to me, too.
>> I think it's definitely a bug to be solved asap.
>>
>> >Ben
>> What do you mean "an undocumented limitation" ?
>>
>> Thanks,
>> Hiro
>>
>> On Sat, Nov 26, 2016 at 3:13 PM, Ben Slater <be...@instaclustr.com>
>> wrote:
>> > Nice detective work! Seems to me that it’s a best an undocumented
>> limitation
>> > and potentially could be viewed as a bug - maybe log another JIRA?
>> >
>> > One node - there is a nodetool truncatehints command that could be used
>> to
>> > clear out the hints
>> > (http://cassandra.apache.org/doc/latest/tools/nodetool/
>> truncatehints.html?highlight=truncate)
>> > . However, it seems to clear all hints on particular endpoint, not just
>> for
>> > a specific table.
>> >
>> > Cheers
>> > Ben
>> >
>> > On Fri, 25 Nov 2016 at 17:42 Yuji Ito <yu...@imagine-orb.com> wrote:
>> >>
>> >> Hi all,
>> >>
>> >> I revised the script to reproduce the issue.
>> >> I think the issue happens more frequently than before.
>> >> Killing another node is added to the previous script.
>> >>
>> >> ==== [script] ====
>> >> #!/bin/sh
>> >>
>> >> node1_ip=<node1 IP address>
>> >> node2_ip=<node2 IP address>
>> >> node3_ip=<node3 IP address>
>> >> node2_user=<user name>
>> >> node3_user=<user name>
>> >> rows=10000
>> >>
>> >> echo "consistency quorum;" > init_data.cql
>> >> for key in $(seq 0 $(expr $rows - 1))
>> >> do
>> >>     echo "insert into testdb.testtbl (key, val) values($key, 1111) IF
>> NOT
>> >> EXISTS;" >> init_data.cql
>> >>     done
>> >>
>> >>     while true
>> >>     do
>> >>     echo "truncate the table"
>> >>     cqlsh $node1_ip -e "truncate table testdb.testtbl" > /dev/null 2>&1
>> >>     if [ $? -ne 0 ]; then
>> >>         echo "truncating failed"
>> >>     continue
>> >>     else
>> >>         break
>> >>     fi
>> >> done
>> >>
>> >> echo "kill C* process on node3"
>> >> pdsh -l $node3_user -R ssh -w $node3_ip "ps auxww | grep
>> CassandraDaemon |
>> >> awk '{if (\$13 ~ /cassand/) print \$2}' | xargs sudo kill -9"
>> >>
>> >> echo "insert $rows rows"
>> >> cqlsh $node1_ip -f init_data.cql > insert_log 2>&1
>> >>
>> >> echo "restart C* process on node3"
>> >> pdsh -l $node3_user -R ssh -w $node3_ip "sudo /etc/init.d/cassandra
>> start"
>> >>
>> >> while true
>> >> do
>> >> echo "truncate the table again"
>> >> cqlsh $node1_ip -e "truncate table testdb.testtbl"
>> >> if [ $? -ne 0 ]; then
>> >>     echo "truncating failed"
>> >>         continue
>> >> else
>> >>     echo "truncation succeeded!"
>> >>     break
>> >> fi
>> >> done
>> >>
>> >> echo "kill C* process on node2"
>> >> pdsh -l $node2_user -R ssh -w $node2_ip "ps auxww | grep
>> CassandraDaemon |
>> >> awk '{if (\$13 ~ /cassand/) print \$2}' | xargs sudo kill -9"
>> >>
>> >> cqlsh $node1_ip --request-timeout 3600 -e "consistency serial; select
>> >> count(*) from testdb.testtbl;"
>> >> sleep 10
>> >> cqlsh $node1_ip --request-timeout 3600 -e "consistency serial; select
>> >> count(*) from testdb.testtbl;"
>> >>
>> >> echo "restart C* process on node2"
>> >> pdsh -l $node2_user -R ssh -w $node2_ip "sudo /etc/init.d/cassandra
>> start"
>> >>
>> >>
>> >> Thanks,
>> >> yuji
>> >>
>> >>
>> >> On Fri, Nov 18, 2016 at 7:52 PM, Yuji Ito <yu...@imagine-orb.com>
>> wrote:
>> >>>
>> >>> I investigated source code and logs of killed node.
>> >>> I guess that unexpected writes are executed when truncation is being
>> >>> executed.
>> >>>
>> >>> Some writes were executed after flush (the first flush) in truncation
>> and
>> >>> these writes could be read.
>> >>> These writes were requested as MUTATION by another node for hinted
>> >>> handoff.
>> >>> Their data was stored to a new memtable and flushed (the second
>> flush) to
>> >>> a new SSTable before snapshot in truncation.
>> >>> So, the truncation discarded only old SSTables, not the new SSTable.
>> >>> That's because ReplayPosition which was used for discarding SSTable
>> was
>> >>> that of the first flush.
>> >>>
>> >>> I copied some parts of log as below.
>> >>> "##" line is my comment.
>> >>> The point is that the ReplayPosition is moved forward by the second
>> >>> flush.
>> >>> It means some writes are executed after the first flush.
>> >>>
>> >>> == log ==
>> >>> ## started truncation
>> >>> TRACE [SharedPool-Worker-16] 2016-11-17 08:36:04,612
>> >>> ColumnFamilyStore.java:2790 - truncating testtbl
>> >>> ## the first flush started before truncation
>> >>> DEBUG [SharedPool-Worker-16] 2016-11-17 08:36:04,612
>> >>> ColumnFamilyStore.java:952 - Enqueuing flush of testtbl: 591360 (0%)
>> >>> on-heap, 0 (0%) off-heap
>> >>> INFO  [MemtableFlushWriter:1] 2016-11-17 08:36:04,613
>> Memtable.java:352 -
>> >>> Writing Memtable-testtbl@1863835308(42.625KiB serialized bytes, 2816
>> ops,
>> >>> 0%/0% of on/off-heap limit)
>> >>> ...
>> >>> DEBUG [MemtableFlushWriter:1] 2016-11-17 08:36:04,973
>> Memtable.java:386 -
>> >>> Completed flushing
>> >>> /var/lib/cassandra/data/testdb/testtbl-562848f0a55611e68b1451065d58fd
>> fb/tmp-lb-1-big-Data.db
>> >>> (17.651KiB) for commitlog position ReplayPosition(segmentId=
>> 1479371760395,
>> >>> position=315867)
>> >>> ## this ReplayPosition was used for discarding SSTables
>> >>> ...
>> >>> TRACE [MemtablePostFlush:1] 2016-11-17 08:36:05,022
>> CommitLog.java:298 -
>> >>> discard completed log segments for ReplayPosition(segmentId=
>> 1479371760395,
>> >>> position=315867), table 562848f0-a556-11e6-8b14-51065d58fdfb
>> >>> ## end of the first flush
>> >>> DEBUG [SharedPool-Worker-16] 2016-11-17 08:36:05,028
>> >>> ColumnFamilyStore.java:2823 - Discarding sstable data for truncated
>> CF +
>> >>> indexes
>> >>> ## the second flush before snapshot
>> >>> DEBUG [SharedPool-Worker-16] 2016-11-17 08:36:05,028
>> >>> ColumnFamilyStore.java:952 - Enqueuing flush of testtbl: 698880 (0%)
>> >>> on-heap, 0 (0%) off-heap
>> >>> INFO  [MemtableFlushWriter:2] 2016-11-17 08:36:05,029
>> Memtable.java:352 -
>> >>> Writing Memtable-testtbl@1186728207(50.375KiB serialized bytes, 3328
>> ops,
>> >>> 0%/0% of on/off-heap limit)
>> >>> ...
>> >>> DEBUG [MemtableFlushWriter:2] 2016-11-17 08:36:05,258
>> Memtable.java:386 -
>> >>> Completed flushing
>> >>> /var/lib/cassandra/data/testdb/testtbl-562848f0a55611e68b1451065d58fd
>> fb/tmp-lb-2-big-Data.db
>> >>> (17.696KiB) for commitlog position ReplayPosition(segmentId=
>> 1479371760395,
>> >>> position=486627)
>> >>> ...
>> >>> TRACE [MemtablePostFlush:1] 2016-11-17 08:36:05,289
>> CommitLog.java:298 -
>> >>> discard completed log segments for ReplayPosition(segmentId=
>> 1479371760395,
>> >>> position=486627), table 562848f0-a556-11e6-8b14-51065d58fdfb
>> >>> ## end of the second flush: position was moved
>> >>> ...
>> >>> ## only old SSTable was deleted because this SSTable was older than
>> >>> ReplayPosition(segmentId=1479371760395, position=315867)
>> >>> TRACE [NonPeriodicTasks:1] 2016-11-17 08:36:05,303 SSTable.java:118 -
>> >>> Deleted
>> >>> /var/lib/cassandra/data/testdb/testtbl-562848f0a55611e68b1451065d58fd
>> fb/lb-1-big
>> >>> ...
>> >>> TRACE [SharedPool-Worker-16] 2016-11-17 08:36:05,320
>> >>> ColumnFamilyStore.java:2841 - truncate complete
>> >>> TRACE [SharedPool-Worker-16] 2016-11-17 08:36:05,320
>> >>> TruncateVerbHandler.java:53 - Truncation(keyspace='testdb',
>> cf='testtbl')
>> >>> applied.  Enqueuing response to 36512@/10.91.145.7
>> >>> TRACE [SharedPool-Worker-16] 2016-11-17 08:36:05,320
>> >>> MessagingService.java:728 - /10.91.145.27 sending REQUEST_RESPONSE to
>> >>> 36512@/10.91.145.7
>> >>> ## end of truncation
>> >>> ====
>> >>>
>> >>> Actually, "truncated_at" of the table on the system.local after
>> running
>> >>> the script was 0x00000158716da30b0004d1db00000158716db524.
>> >>> It means segmentId=1479371760395, position=315867
>> >>> truncated_at=1479371765028 (2016-11-17 08:36:05,028)
>> >>>
>> >>> thanks,
>> >>> yuji
>> >>>
>> >>>
>> >>> On Wed, Nov 16, 2016 at 5:25 PM, Yuji Ito <yu...@imagine-orb.com>
>> wrote:
>> >>>>
>> >>>> Hi,
>> >>>>
>> >>>> I could find stale data after truncating a table.
>> >>>> It seems that truncating starts while recovery is being executed just
>> >>>> after a node restarts.
>> >>>> After the truncating finishes, recovery still continues?
>> >>>> Is it expected?
>> >>>>
>> >>>> I use C* 2.2.8 and can reproduce it as below.
>> >>>>
>> >>>> ==== [create table] ====
>> >>>> cqlsh $ip -e "drop keyspace testdb;"
>> >>>> cqlsh $ip -e "CREATE KEYSPACE testdb WITH replication = {'class':
>> >>>> 'SimpleStrategy', 'replication_factor': '3'};"
>> >>>> cqlsh $ip -e "CREATE TABLE testdb.testtbl (key int PRIMARY KEY, val
>> >>>> int);"
>> >>>>
>> >>>> ==== [script] ====
>> >>>> #!/bin/sh
>> >>>>
>> >>>> node1_ip=<node1 IP address>
>> >>>> node2_ip=<node2 IP address>
>> >>>> node3_ip=<node3 IP address>
>> >>>> node3_user=<user name>
>> >>>> rows=10000
>> >>>>
>> >>>> echo "consistency quorum;" > init_data.cql
>> >>>> for key in $(seq 0 $(expr $rows - 1))
>> >>>> do
>> >>>>     echo "insert into testdb.testtbl (key, val) values($key, 1111) IF
>> >>>> NOT EXISTS;" >> init_data.cql
>> >>>> done
>> >>>>
>> >>>> while true
>> >>>> do
>> >>>> echo "truncate the table"
>> >>>> cqlsh $node1_ip -e "truncate table testdb.testtbl"
>> >>>> if [ $? -ne 0 ]; then
>> >>>>     echo "truncating failed"
>> >>>>     continue
>> >>>> else
>> >>>>     break
>> >>>> fi
>> >>>> done
>> >>>>
>> >>>> echo "kill C* process on node3"
>> >>>> pdsh -l $node3_user -R ssh -w $node3_ip "ps auxww | grep
>> CassandraDaemon
>> >>>> | awk '{if (\$13 ~ /cassand/) print \$2}' | xargs sudo kill -9"
>> >>>>
>> >>>> echo "insert $rows rows"
>> >>>> cqlsh $node1_ip -f init_data.cql > insert_log 2>&1
>> >>>>
>> >>>> echo "restart C* process on node3"
>> >>>> pdsh -l $node3_user -R ssh -w $node3_ip "sudo /etc/init.d/cassandra
>> >>>> start"
>> >>>>
>> >>>> while true
>> >>>> do
>> >>>> echo "truncate the table again"
>> >>>> cqlsh $node1_ip -e "truncate table testdb.testtbl"
>> >>>> if [ $? -ne 0 ]; then
>> >>>>     echo "truncating failed"
>> >>>>     continue
>> >>>> else
>> >>>>     break
>> >>>> fi
>> >>>> done
>> >>>>
>> >>>> cqlsh $node1_ip --request-timeout 3600 -e "consistency serial; select
>> >>>> count(*) from testdb.testtbl;"
>> >>>> sleep 10
>> >>>> cqlsh $node1_ip --request-timeout 3600 -e "consistency serial; select
>> >>>> count(*) from testdb.testtbl;"
>> >>>>
>> >>>>
>> >>>> ==== [result] ====
>> >>>> truncate the table
>> >>>> kill C* process on node3
>> >>>> insert 10000 rows
>> >>>> restart C* process on node3
>> >>>> 10.91.145.27: Starting Cassandra: OK
>> >>>> truncate the table again
>> >>>> <stdin>:1:TruncateError: Error during truncate: Cannot achieve
>> >>>> consistency level ALL
>> >>>> truncating failed
>> >>>> truncate the table again
>> >>>> <stdin>:1:TruncateError: Error during truncate: Cannot achieve
>> >>>> consistency level ALL
>> >>>> truncating failed
>> >>>> truncate the table again
>> >>>> <stdin>:1:TruncateError: Error during truncate: Cannot achieve
>> >>>> consistency level ALL
>> >>>> truncating failed
>> >>>> truncate the table again
>> >>>> <stdin>:1:TruncateError: Error during truncate: Cannot achieve
>> >>>> consistency level ALL
>> >>>> truncating failed
>> >>>> truncate the table again
>> >>>> <stdin>:1:TruncateError: Error during truncate: Cannot achieve
>> >>>> consistency level ALL
>> >>>> truncating failed
>> >>>> truncate the table again
>> >>>> <stdin>:1:TruncateError: Error during truncate: Cannot achieve
>> >>>> consistency level ALL
>> >>>> truncating failed
>> >>>> truncate the table again
>> >>>> Consistency level set to SERIAL.
>> >>>>
>> >>>>  count
>> >>>> -------
>> >>>>    300
>> >>>>
>> >>>> (1 rows)
>> >>>>
>> >>>> Warnings :
>> >>>> Aggregation query used without partition key
>> >>>>
>> >>>> Consistency level set to SERIAL.
>> >>>>
>> >>>>  count
>> >>>> -------
>> >>>>   2304
>> >>>>
>> >>>> (1 rows)
>> >>>>
>> >>>> Warnings :
>> >>>> Aggregation query used without partition key
>> >>>> ====
>> >>>>
>> >>>> I found it when I was investigating data lost problem. (Ref. "failure
>> >>>> node rejoin" thread)
>> >>>> I'm not sure this problem is related to data lost.
>> >>>>
>> >>>> Thanks,
>> >>>> yuji
>> >>>
>> >>>
>> >>
>> >
>>
>

Re: Does recovery continue after truncating a table?

Posted by Ben Slater <be...@instaclustr.com>.
By “undocumented limitation”, I meant “TRUNCATE” is mainly only used in
development and testing, not production scenarios so a sufficient fix (and
certainly a better than nothing fix) might be just to document that if you
issue a TRUNCATE while there are still hinted hand-offs pending the hinted
hand-offs replayed after the truncate will come back to life. Of course, an
actual fix would be better.

Cheers
Ben

On Sat, 26 Nov 2016 at 21:08 Hiroyuki Yamada <mo...@gmail.com> wrote:

> Hi Yuji and Ben,
>
> I tried out this revised script and the same issue occurred to me, too.
> I think it's definitely a bug to be solved asap.
>
> >Ben
> What do you mean "an undocumented limitation" ?
>
> Thanks,
> Hiro
>
> On Sat, Nov 26, 2016 at 3:13 PM, Ben Slater <be...@instaclustr.com>
> wrote:
> > Nice detective work! Seems to me that it’s a best an undocumented
> limitation
> > and potentially could be viewed as a bug - maybe log another JIRA?
> >
> > One node - there is a nodetool truncatehints command that could be used
> to
> > clear out the hints
> > (
> http://cassandra.apache.org/doc/latest/tools/nodetool/truncatehints.html?highlight=truncate
> )
> > . However, it seems to clear all hints on particular endpoint, not just
> for
> > a specific table.
> >
> > Cheers
> > Ben
> >
> > On Fri, 25 Nov 2016 at 17:42 Yuji Ito <yu...@imagine-orb.com> wrote:
> >>
> >> Hi all,
> >>
> >> I revised the script to reproduce the issue.
> >> I think the issue happens more frequently than before.
> >> Killing another node is added to the previous script.
> >>
> >> ==== [script] ====
> >> #!/bin/sh
> >>
> >> node1_ip=<node1 IP address>
> >> node2_ip=<node2 IP address>
> >> node3_ip=<node3 IP address>
> >> node2_user=<user name>
> >> node3_user=<user name>
> >> rows=10000
> >>
> >> echo "consistency quorum;" > init_data.cql
> >> for key in $(seq 0 $(expr $rows - 1))
> >> do
> >>     echo "insert into testdb.testtbl (key, val) values($key, 1111) IF
> NOT
> >> EXISTS;" >> init_data.cql
> >>     done
> >>
> >>     while true
> >>     do
> >>     echo "truncate the table"
> >>     cqlsh $node1_ip -e "truncate table testdb.testtbl" > /dev/null 2>&1
> >>     if [ $? -ne 0 ]; then
> >>         echo "truncating failed"
> >>     continue
> >>     else
> >>         break
> >>     fi
> >> done
> >>
> >> echo "kill C* process on node3"
> >> pdsh -l $node3_user -R ssh -w $node3_ip "ps auxww | grep
> CassandraDaemon |
> >> awk '{if (\$13 ~ /cassand/) print \$2}' | xargs sudo kill -9"
> >>
> >> echo "insert $rows rows"
> >> cqlsh $node1_ip -f init_data.cql > insert_log 2>&1
> >>
> >> echo "restart C* process on node3"
> >> pdsh -l $node3_user -R ssh -w $node3_ip "sudo /etc/init.d/cassandra
> start"
> >>
> >> while true
> >> do
> >> echo "truncate the table again"
> >> cqlsh $node1_ip -e "truncate table testdb.testtbl"
> >> if [ $? -ne 0 ]; then
> >>     echo "truncating failed"
> >>         continue
> >> else
> >>     echo "truncation succeeded!"
> >>     break
> >> fi
> >> done
> >>
> >> echo "kill C* process on node2"
> >> pdsh -l $node2_user -R ssh -w $node2_ip "ps auxww | grep
> CassandraDaemon |
> >> awk '{if (\$13 ~ /cassand/) print \$2}' | xargs sudo kill -9"
> >>
> >> cqlsh $node1_ip --request-timeout 3600 -e "consistency serial; select
> >> count(*) from testdb.testtbl;"
> >> sleep 10
> >> cqlsh $node1_ip --request-timeout 3600 -e "consistency serial; select
> >> count(*) from testdb.testtbl;"
> >>
> >> echo "restart C* process on node2"
> >> pdsh -l $node2_user -R ssh -w $node2_ip "sudo /etc/init.d/cassandra
> start"
> >>
> >>
> >> Thanks,
> >> yuji
> >>
> >>
> >> On Fri, Nov 18, 2016 at 7:52 PM, Yuji Ito <yu...@imagine-orb.com> wrote:
> >>>
> >>> I investigated source code and logs of killed node.
> >>> I guess that unexpected writes are executed when truncation is being
> >>> executed.
> >>>
> >>> Some writes were executed after flush (the first flush) in truncation
> and
> >>> these writes could be read.
> >>> These writes were requested as MUTATION by another node for hinted
> >>> handoff.
> >>> Their data was stored to a new memtable and flushed (the second flush)
> to
> >>> a new SSTable before snapshot in truncation.
> >>> So, the truncation discarded only old SSTables, not the new SSTable.
> >>> That's because ReplayPosition which was used for discarding SSTable was
> >>> that of the first flush.
> >>>
> >>> I copied some parts of log as below.
> >>> "##" line is my comment.
> >>> The point is that the ReplayPosition is moved forward by the second
> >>> flush.
> >>> It means some writes are executed after the first flush.
> >>>
> >>> == log ==
> >>> ## started truncation
> >>> TRACE [SharedPool-Worker-16] 2016-11-17 08:36:04,612
> >>> ColumnFamilyStore.java:2790 - truncating testtbl
> >>> ## the first flush started before truncation
> >>> DEBUG [SharedPool-Worker-16] 2016-11-17 08:36:04,612
> >>> ColumnFamilyStore.java:952 - Enqueuing flush of testtbl: 591360 (0%)
> >>> on-heap, 0 (0%) off-heap
> >>> INFO  [MemtableFlushWriter:1] 2016-11-17 08:36:04,613
> Memtable.java:352 -
> >>> Writing Memtable-testtbl@1863835308(42.625KiB serialized bytes, 2816
> ops,
> >>> 0%/0% of on/off-heap limit)
> >>> ...
> >>> DEBUG [MemtableFlushWriter:1] 2016-11-17 08:36:04,973
> Memtable.java:386 -
> >>> Completed flushing
> >>>
> /var/lib/cassandra/data/testdb/testtbl-562848f0a55611e68b1451065d58fdfb/tmp-lb-1-big-Data.db
> >>> (17.651KiB) for commitlog position
> ReplayPosition(segmentId=1479371760395,
> >>> position=315867)
> >>> ## this ReplayPosition was used for discarding SSTables
> >>> ...
> >>> TRACE [MemtablePostFlush:1] 2016-11-17 08:36:05,022 CommitLog.java:298
> -
> >>> discard completed log segments for
> ReplayPosition(segmentId=1479371760395,
> >>> position=315867), table 562848f0-a556-11e6-8b14-51065d58fdfb
> >>> ## end of the first flush
> >>> DEBUG [SharedPool-Worker-16] 2016-11-17 08:36:05,028
> >>> ColumnFamilyStore.java:2823 - Discarding sstable data for truncated CF
> +
> >>> indexes
> >>> ## the second flush before snapshot
> >>> DEBUG [SharedPool-Worker-16] 2016-11-17 08:36:05,028
> >>> ColumnFamilyStore.java:952 - Enqueuing flush of testtbl: 698880 (0%)
> >>> on-heap, 0 (0%) off-heap
> >>> INFO  [MemtableFlushWriter:2] 2016-11-17 08:36:05,029
> Memtable.java:352 -
> >>> Writing Memtable-testtbl@1186728207(50.375KiB serialized bytes, 3328
> ops,
> >>> 0%/0% of on/off-heap limit)
> >>> ...
> >>> DEBUG [MemtableFlushWriter:2] 2016-11-17 08:36:05,258
> Memtable.java:386 -
> >>> Completed flushing
> >>>
> /var/lib/cassandra/data/testdb/testtbl-562848f0a55611e68b1451065d58fdfb/tmp-lb-2-big-Data.db
> >>> (17.696KiB) for commitlog position
> ReplayPosition(segmentId=1479371760395,
> >>> position=486627)
> >>> ...
> >>> TRACE [MemtablePostFlush:1] 2016-11-17 08:36:05,289 CommitLog.java:298
> -
> >>> discard completed log segments for
> ReplayPosition(segmentId=1479371760395,
> >>> position=486627), table 562848f0-a556-11e6-8b14-51065d58fdfb
> >>> ## end of the second flush: position was moved
> >>> ...
> >>> ## only old SSTable was deleted because this SSTable was older than
> >>> ReplayPosition(segmentId=1479371760395, position=315867)
> >>> TRACE [NonPeriodicTasks:1] 2016-11-17 08:36:05,303 SSTable.java:118 -
> >>> Deleted
> >>>
> /var/lib/cassandra/data/testdb/testtbl-562848f0a55611e68b1451065d58fdfb/lb-1-big
> >>> ...
> >>> TRACE [SharedPool-Worker-16] 2016-11-17 08:36:05,320
> >>> ColumnFamilyStore.java:2841 - truncate complete
> >>> TRACE [SharedPool-Worker-16] 2016-11-17 08:36:05,320
> >>> TruncateVerbHandler.java:53 - Truncation(keyspace='testdb',
> cf='testtbl')
> >>> applied.  Enqueuing response to 36512@/10.91.145.7
> >>> TRACE [SharedPool-Worker-16] 2016-11-17 08:36:05,320
> >>> MessagingService.java:728 - /10.91.145.27 sending REQUEST_RESPONSE to
> >>> 36512@/10.91.145.7
> >>> ## end of truncation
> >>> ====
> >>>
> >>> Actually, "truncated_at" of the table on the system.local after running
> >>> the script was 0x00000158716da30b0004d1db00000158716db524.
> >>> It means segmentId=1479371760395, position=315867
> >>> truncated_at=1479371765028 (2016-11-17 08:36:05,028)
> >>>
> >>> thanks,
> >>> yuji
> >>>
> >>>
> >>> On Wed, Nov 16, 2016 at 5:25 PM, Yuji Ito <yu...@imagine-orb.com>
> wrote:
> >>>>
> >>>> Hi,
> >>>>
> >>>> I could find stale data after truncating a table.
> >>>> It seems that truncating starts while recovery is being executed just
> >>>> after a node restarts.
> >>>> After the truncating finishes, recovery still continues?
> >>>> Is it expected?
> >>>>
> >>>> I use C* 2.2.8 and can reproduce it as below.
> >>>>
> >>>> ==== [create table] ====
> >>>> cqlsh $ip -e "drop keyspace testdb;"
> >>>> cqlsh $ip -e "CREATE KEYSPACE testdb WITH replication = {'class':
> >>>> 'SimpleStrategy', 'replication_factor': '3'};"
> >>>> cqlsh $ip -e "CREATE TABLE testdb.testtbl (key int PRIMARY KEY, val
> >>>> int);"
> >>>>
> >>>> ==== [script] ====
> >>>> #!/bin/sh
> >>>>
> >>>> node1_ip=<node1 IP address>
> >>>> node2_ip=<node2 IP address>
> >>>> node3_ip=<node3 IP address>
> >>>> node3_user=<user name>
> >>>> rows=10000
> >>>>
> >>>> echo "consistency quorum;" > init_data.cql
> >>>> for key in $(seq 0 $(expr $rows - 1))
> >>>> do
> >>>>     echo "insert into testdb.testtbl (key, val) values($key, 1111) IF
> >>>> NOT EXISTS;" >> init_data.cql
> >>>> done
> >>>>
> >>>> while true
> >>>> do
> >>>> echo "truncate the table"
> >>>> cqlsh $node1_ip -e "truncate table testdb.testtbl"
> >>>> if [ $? -ne 0 ]; then
> >>>>     echo "truncating failed"
> >>>>     continue
> >>>> else
> >>>>     break
> >>>> fi
> >>>> done
> >>>>
> >>>> echo "kill C* process on node3"
> >>>> pdsh -l $node3_user -R ssh -w $node3_ip "ps auxww | grep
> CassandraDaemon
> >>>> | awk '{if (\$13 ~ /cassand/) print \$2}' | xargs sudo kill -9"
> >>>>
> >>>> echo "insert $rows rows"
> >>>> cqlsh $node1_ip -f init_data.cql > insert_log 2>&1
> >>>>
> >>>> echo "restart C* process on node3"
> >>>> pdsh -l $node3_user -R ssh -w $node3_ip "sudo /etc/init.d/cassandra
> >>>> start"
> >>>>
> >>>> while true
> >>>> do
> >>>> echo "truncate the table again"
> >>>> cqlsh $node1_ip -e "truncate table testdb.testtbl"
> >>>> if [ $? -ne 0 ]; then
> >>>>     echo "truncating failed"
> >>>>     continue
> >>>> else
> >>>>     break
> >>>> fi
> >>>> done
> >>>>
> >>>> cqlsh $node1_ip --request-timeout 3600 -e "consistency serial; select
> >>>> count(*) from testdb.testtbl;"
> >>>> sleep 10
> >>>> cqlsh $node1_ip --request-timeout 3600 -e "consistency serial; select
> >>>> count(*) from testdb.testtbl;"
> >>>>
> >>>>
> >>>> ==== [result] ====
> >>>> truncate the table
> >>>> kill C* process on node3
> >>>> insert 10000 rows
> >>>> restart C* process on node3
> >>>> 10.91.145.27: Starting Cassandra: OK
> >>>> truncate the table again
> >>>> <stdin>:1:TruncateError: Error during truncate: Cannot achieve
> >>>> consistency level ALL
> >>>> truncating failed
> >>>> truncate the table again
> >>>> <stdin>:1:TruncateError: Error during truncate: Cannot achieve
> >>>> consistency level ALL
> >>>> truncating failed
> >>>> truncate the table again
> >>>> <stdin>:1:TruncateError: Error during truncate: Cannot achieve
> >>>> consistency level ALL
> >>>> truncating failed
> >>>> truncate the table again
> >>>> <stdin>:1:TruncateError: Error during truncate: Cannot achieve
> >>>> consistency level ALL
> >>>> truncating failed
> >>>> truncate the table again
> >>>> <stdin>:1:TruncateError: Error during truncate: Cannot achieve
> >>>> consistency level ALL
> >>>> truncating failed
> >>>> truncate the table again
> >>>> <stdin>:1:TruncateError: Error during truncate: Cannot achieve
> >>>> consistency level ALL
> >>>> truncating failed
> >>>> truncate the table again
> >>>> Consistency level set to SERIAL.
> >>>>
> >>>>  count
> >>>> -------
> >>>>    300
> >>>>
> >>>> (1 rows)
> >>>>
> >>>> Warnings :
> >>>> Aggregation query used without partition key
> >>>>
> >>>> Consistency level set to SERIAL.
> >>>>
> >>>>  count
> >>>> -------
> >>>>   2304
> >>>>
> >>>> (1 rows)
> >>>>
> >>>> Warnings :
> >>>> Aggregation query used without partition key
> >>>> ====
> >>>>
> >>>> I found it when I was investigating data lost problem. (Ref. "failure
> >>>> node rejoin" thread)
> >>>> I'm not sure this problem is related to data lost.
> >>>>
> >>>> Thanks,
> >>>> yuji
> >>>
> >>>
> >>
> >
>

Re: Does recovery continue after truncating a table?

Posted by Hiroyuki Yamada <mo...@gmail.com>.
Hi Yuji and Ben,

I tried out this revised script and the same issue occurred to me, too.
I think it's definitely a bug to be solved asap.

>Ben
What do you mean "an undocumented limitation" ?

Thanks,
Hiro

On Sat, Nov 26, 2016 at 3:13 PM, Ben Slater <be...@instaclustr.com> wrote:
> Nice detective work! Seems to me that it’s a best an undocumented limitation
> and potentially could be viewed as a bug - maybe log another JIRA?
>
> One node - there is a nodetool truncatehints command that could be used to
> clear out the hints
> (http://cassandra.apache.org/doc/latest/tools/nodetool/truncatehints.html?highlight=truncate)
> . However, it seems to clear all hints on particular endpoint, not just for
> a specific table.
>
> Cheers
> Ben
>
> On Fri, 25 Nov 2016 at 17:42 Yuji Ito <yu...@imagine-orb.com> wrote:
>>
>> Hi all,
>>
>> I revised the script to reproduce the issue.
>> I think the issue happens more frequently than before.
>> Killing another node is added to the previous script.
>>
>> ==== [script] ====
>> #!/bin/sh
>>
>> node1_ip=<node1 IP address>
>> node2_ip=<node2 IP address>
>> node3_ip=<node3 IP address>
>> node2_user=<user name>
>> node3_user=<user name>
>> rows=10000
>>
>> echo "consistency quorum;" > init_data.cql
>> for key in $(seq 0 $(expr $rows - 1))
>> do
>>     echo "insert into testdb.testtbl (key, val) values($key, 1111) IF NOT
>> EXISTS;" >> init_data.cql
>>     done
>>
>>     while true
>>     do
>>     echo "truncate the table"
>>     cqlsh $node1_ip -e "truncate table testdb.testtbl" > /dev/null 2>&1
>>     if [ $? -ne 0 ]; then
>>         echo "truncating failed"
>>     continue
>>     else
>>         break
>>     fi
>> done
>>
>> echo "kill C* process on node3"
>> pdsh -l $node3_user -R ssh -w $node3_ip "ps auxww | grep CassandraDaemon |
>> awk '{if (\$13 ~ /cassand/) print \$2}' | xargs sudo kill -9"
>>
>> echo "insert $rows rows"
>> cqlsh $node1_ip -f init_data.cql > insert_log 2>&1
>>
>> echo "restart C* process on node3"
>> pdsh -l $node3_user -R ssh -w $node3_ip "sudo /etc/init.d/cassandra start"
>>
>> while true
>> do
>> echo "truncate the table again"
>> cqlsh $node1_ip -e "truncate table testdb.testtbl"
>> if [ $? -ne 0 ]; then
>>     echo "truncating failed"
>>         continue
>> else
>>     echo "truncation succeeded!"
>>     break
>> fi
>> done
>>
>> echo "kill C* process on node2"
>> pdsh -l $node2_user -R ssh -w $node2_ip "ps auxww | grep CassandraDaemon |
>> awk '{if (\$13 ~ /cassand/) print \$2}' | xargs sudo kill -9"
>>
>> cqlsh $node1_ip --request-timeout 3600 -e "consistency serial; select
>> count(*) from testdb.testtbl;"
>> sleep 10
>> cqlsh $node1_ip --request-timeout 3600 -e "consistency serial; select
>> count(*) from testdb.testtbl;"
>>
>> echo "restart C* process on node2"
>> pdsh -l $node2_user -R ssh -w $node2_ip "sudo /etc/init.d/cassandra start"
>>
>>
>> Thanks,
>> yuji
>>
>>
>> On Fri, Nov 18, 2016 at 7:52 PM, Yuji Ito <yu...@imagine-orb.com> wrote:
>>>
>>> I investigated source code and logs of killed node.
>>> I guess that unexpected writes are executed when truncation is being
>>> executed.
>>>
>>> Some writes were executed after flush (the first flush) in truncation and
>>> these writes could be read.
>>> These writes were requested as MUTATION by another node for hinted
>>> handoff.
>>> Their data was stored to a new memtable and flushed (the second flush) to
>>> a new SSTable before snapshot in truncation.
>>> So, the truncation discarded only old SSTables, not the new SSTable.
>>> That's because ReplayPosition which was used for discarding SSTable was
>>> that of the first flush.
>>>
>>> I copied some parts of log as below.
>>> "##" line is my comment.
>>> The point is that the ReplayPosition is moved forward by the second
>>> flush.
>>> It means some writes are executed after the first flush.
>>>
>>> == log ==
>>> ## started truncation
>>> TRACE [SharedPool-Worker-16] 2016-11-17 08:36:04,612
>>> ColumnFamilyStore.java:2790 - truncating testtbl
>>> ## the first flush started before truncation
>>> DEBUG [SharedPool-Worker-16] 2016-11-17 08:36:04,612
>>> ColumnFamilyStore.java:952 - Enqueuing flush of testtbl: 591360 (0%)
>>> on-heap, 0 (0%) off-heap
>>> INFO  [MemtableFlushWriter:1] 2016-11-17 08:36:04,613 Memtable.java:352 -
>>> Writing Memtable-testtbl@1863835308(42.625KiB serialized bytes, 2816 ops,
>>> 0%/0% of on/off-heap limit)
>>> ...
>>> DEBUG [MemtableFlushWriter:1] 2016-11-17 08:36:04,973 Memtable.java:386 -
>>> Completed flushing
>>> /var/lib/cassandra/data/testdb/testtbl-562848f0a55611e68b1451065d58fdfb/tmp-lb-1-big-Data.db
>>> (17.651KiB) for commitlog position ReplayPosition(segmentId=1479371760395,
>>> position=315867)
>>> ## this ReplayPosition was used for discarding SSTables
>>> ...
>>> TRACE [MemtablePostFlush:1] 2016-11-17 08:36:05,022 CommitLog.java:298 -
>>> discard completed log segments for ReplayPosition(segmentId=1479371760395,
>>> position=315867), table 562848f0-a556-11e6-8b14-51065d58fdfb
>>> ## end of the first flush
>>> DEBUG [SharedPool-Worker-16] 2016-11-17 08:36:05,028
>>> ColumnFamilyStore.java:2823 - Discarding sstable data for truncated CF +
>>> indexes
>>> ## the second flush before snapshot
>>> DEBUG [SharedPool-Worker-16] 2016-11-17 08:36:05,028
>>> ColumnFamilyStore.java:952 - Enqueuing flush of testtbl: 698880 (0%)
>>> on-heap, 0 (0%) off-heap
>>> INFO  [MemtableFlushWriter:2] 2016-11-17 08:36:05,029 Memtable.java:352 -
>>> Writing Memtable-testtbl@1186728207(50.375KiB serialized bytes, 3328 ops,
>>> 0%/0% of on/off-heap limit)
>>> ...
>>> DEBUG [MemtableFlushWriter:2] 2016-11-17 08:36:05,258 Memtable.java:386 -
>>> Completed flushing
>>> /var/lib/cassandra/data/testdb/testtbl-562848f0a55611e68b1451065d58fdfb/tmp-lb-2-big-Data.db
>>> (17.696KiB) for commitlog position ReplayPosition(segmentId=1479371760395,
>>> position=486627)
>>> ...
>>> TRACE [MemtablePostFlush:1] 2016-11-17 08:36:05,289 CommitLog.java:298 -
>>> discard completed log segments for ReplayPosition(segmentId=1479371760395,
>>> position=486627), table 562848f0-a556-11e6-8b14-51065d58fdfb
>>> ## end of the second flush: position was moved
>>> ...
>>> ## only old SSTable was deleted because this SSTable was older than
>>> ReplayPosition(segmentId=1479371760395, position=315867)
>>> TRACE [NonPeriodicTasks:1] 2016-11-17 08:36:05,303 SSTable.java:118 -
>>> Deleted
>>> /var/lib/cassandra/data/testdb/testtbl-562848f0a55611e68b1451065d58fdfb/lb-1-big
>>> ...
>>> TRACE [SharedPool-Worker-16] 2016-11-17 08:36:05,320
>>> ColumnFamilyStore.java:2841 - truncate complete
>>> TRACE [SharedPool-Worker-16] 2016-11-17 08:36:05,320
>>> TruncateVerbHandler.java:53 - Truncation(keyspace='testdb', cf='testtbl')
>>> applied.  Enqueuing response to 36512@/10.91.145.7
>>> TRACE [SharedPool-Worker-16] 2016-11-17 08:36:05,320
>>> MessagingService.java:728 - /10.91.145.27 sending REQUEST_RESPONSE to
>>> 36512@/10.91.145.7
>>> ## end of truncation
>>> ====
>>>
>>> Actually, "truncated_at" of the table on the system.local after running
>>> the script was 0x00000158716da30b0004d1db00000158716db524.
>>> It means segmentId=1479371760395, position=315867
>>> truncated_at=1479371765028 (2016-11-17 08:36:05,028)
>>>
>>> thanks,
>>> yuji
>>>
>>>
>>> On Wed, Nov 16, 2016 at 5:25 PM, Yuji Ito <yu...@imagine-orb.com> wrote:
>>>>
>>>> Hi,
>>>>
>>>> I could find stale data after truncating a table.
>>>> It seems that truncating starts while recovery is being executed just
>>>> after a node restarts.
>>>> After the truncating finishes, recovery still continues?
>>>> Is it expected?
>>>>
>>>> I use C* 2.2.8 and can reproduce it as below.
>>>>
>>>> ==== [create table] ====
>>>> cqlsh $ip -e "drop keyspace testdb;"
>>>> cqlsh $ip -e "CREATE KEYSPACE testdb WITH replication = {'class':
>>>> 'SimpleStrategy', 'replication_factor': '3'};"
>>>> cqlsh $ip -e "CREATE TABLE testdb.testtbl (key int PRIMARY KEY, val
>>>> int);"
>>>>
>>>> ==== [script] ====
>>>> #!/bin/sh
>>>>
>>>> node1_ip=<node1 IP address>
>>>> node2_ip=<node2 IP address>
>>>> node3_ip=<node3 IP address>
>>>> node3_user=<user name>
>>>> rows=10000
>>>>
>>>> echo "consistency quorum;" > init_data.cql
>>>> for key in $(seq 0 $(expr $rows - 1))
>>>> do
>>>>     echo "insert into testdb.testtbl (key, val) values($key, 1111) IF
>>>> NOT EXISTS;" >> init_data.cql
>>>> done
>>>>
>>>> while true
>>>> do
>>>> echo "truncate the table"
>>>> cqlsh $node1_ip -e "truncate table testdb.testtbl"
>>>> if [ $? -ne 0 ]; then
>>>>     echo "truncating failed"
>>>>     continue
>>>> else
>>>>     break
>>>> fi
>>>> done
>>>>
>>>> echo "kill C* process on node3"
>>>> pdsh -l $node3_user -R ssh -w $node3_ip "ps auxww | grep CassandraDaemon
>>>> | awk '{if (\$13 ~ /cassand/) print \$2}' | xargs sudo kill -9"
>>>>
>>>> echo "insert $rows rows"
>>>> cqlsh $node1_ip -f init_data.cql > insert_log 2>&1
>>>>
>>>> echo "restart C* process on node3"
>>>> pdsh -l $node3_user -R ssh -w $node3_ip "sudo /etc/init.d/cassandra
>>>> start"
>>>>
>>>> while true
>>>> do
>>>> echo "truncate the table again"
>>>> cqlsh $node1_ip -e "truncate table testdb.testtbl"
>>>> if [ $? -ne 0 ]; then
>>>>     echo "truncating failed"
>>>>     continue
>>>> else
>>>>     break
>>>> fi
>>>> done
>>>>
>>>> cqlsh $node1_ip --request-timeout 3600 -e "consistency serial; select
>>>> count(*) from testdb.testtbl;"
>>>> sleep 10
>>>> cqlsh $node1_ip --request-timeout 3600 -e "consistency serial; select
>>>> count(*) from testdb.testtbl;"
>>>>
>>>>
>>>> ==== [result] ====
>>>> truncate the table
>>>> kill C* process on node3
>>>> insert 10000 rows
>>>> restart C* process on node3
>>>> 10.91.145.27: Starting Cassandra: OK
>>>> truncate the table again
>>>> <stdin>:1:TruncateError: Error during truncate: Cannot achieve
>>>> consistency level ALL
>>>> truncating failed
>>>> truncate the table again
>>>> <stdin>:1:TruncateError: Error during truncate: Cannot achieve
>>>> consistency level ALL
>>>> truncating failed
>>>> truncate the table again
>>>> <stdin>:1:TruncateError: Error during truncate: Cannot achieve
>>>> consistency level ALL
>>>> truncating failed
>>>> truncate the table again
>>>> <stdin>:1:TruncateError: Error during truncate: Cannot achieve
>>>> consistency level ALL
>>>> truncating failed
>>>> truncate the table again
>>>> <stdin>:1:TruncateError: Error during truncate: Cannot achieve
>>>> consistency level ALL
>>>> truncating failed
>>>> truncate the table again
>>>> <stdin>:1:TruncateError: Error during truncate: Cannot achieve
>>>> consistency level ALL
>>>> truncating failed
>>>> truncate the table again
>>>> Consistency level set to SERIAL.
>>>>
>>>>  count
>>>> -------
>>>>    300
>>>>
>>>> (1 rows)
>>>>
>>>> Warnings :
>>>> Aggregation query used without partition key
>>>>
>>>> Consistency level set to SERIAL.
>>>>
>>>>  count
>>>> -------
>>>>   2304
>>>>
>>>> (1 rows)
>>>>
>>>> Warnings :
>>>> Aggregation query used without partition key
>>>> ====
>>>>
>>>> I found it when I was investigating data lost problem. (Ref. "failure
>>>> node rejoin" thread)
>>>> I'm not sure this problem is related to data lost.
>>>>
>>>> Thanks,
>>>> yuji
>>>
>>>
>>
>

Re: Does recovery continue after truncating a table?

Posted by Ben Slater <be...@instaclustr.com>.
Nice detective work! Seems to me that it’s a best an undocumented
limitation and potentially could be viewed as a bug - maybe log another
JIRA?

One node - there is a nodetool truncatehints command that could be used to
clear out the hints (
http://cassandra.apache.org/doc/latest/tools/nodetool/truncatehints.html?highlight=truncate)
.
However, it seems to clear all hints on particular endpoint, not just for a
specific table.

Cheers
Ben

On Fri, 25 Nov 2016 at 17:42 Yuji Ito <yu...@imagine-orb.com> wrote:

> Hi all,
>
> I revised the script to reproduce the issue.
> I think the issue happens more frequently than before.
> Killing another node is added to the previous script.
>
> ==== [script] ====
> #!/bin/sh
>
> node1_ip=<node1 IP address>
> node2_ip=<node2 IP address>
> node3_ip=<node3 IP address>
> node2_user=<user name>
> node3_user=<user name>
> rows=10000
>
> echo "consistency quorum;" > init_data.cql
> for key in $(seq 0 $(expr $rows - 1))
> do
>     echo "insert into testdb.testtbl (key, val) values($key, 1111) IF NOT
> EXISTS;" >> init_data.cql
>     done
>
>     while true
>     do
>     echo "truncate the table"
>     cqlsh $node1_ip -e "truncate table testdb.testtbl" > /dev/null 2>&1
>     if [ $? -ne 0 ]; then
>         echo "truncating failed"
>     continue
>     else
>         break
>     fi
> done
>
> echo "kill C* process on node3"
> pdsh -l $node3_user -R ssh -w $node3_ip "ps auxww | grep CassandraDaemon |
> awk '{if (\$13 ~ /cassand/) print \$2}' | xargs sudo kill -9"
>
> echo "insert $rows rows"
> cqlsh $node1_ip -f init_data.cql > insert_log 2>&1
>
> echo "restart C* process on node3"
> pdsh -l $node3_user -R ssh -w $node3_ip "sudo /etc/init.d/cassandra start"
>
> while true
> do
> echo "truncate the table again"
> cqlsh $node1_ip -e "truncate table testdb.testtbl"
> if [ $? -ne 0 ]; then
>     echo "truncating failed"
>         continue
> else
>     echo "truncation succeeded!"
>     break
> fi
> done
>
> echo "kill C* process on node2"
> pdsh -l $node2_user -R ssh -w $node2_ip "ps auxww | grep CassandraDaemon |
> awk '{if (\$13 ~ /cassand/) print \$2}' | xargs sudo kill -9"
>
> cqlsh $node1_ip --request-timeout 3600 -e "consistency serial; select
> count(*) from testdb.testtbl;"
> sleep 10
> cqlsh $node1_ip --request-timeout 3600 -e "consistency serial; select
> count(*) from testdb.testtbl;"
>
> echo "restart C* process on node2"
> pdsh -l $node2_user -R ssh -w $node2_ip "sudo /etc/init.d/cassandra start"
>
>
> Thanks,
> yuji
>
>
> On Fri, Nov 18, 2016 at 7:52 PM, Yuji Ito <yu...@imagine-orb.com> wrote:
>
> I investigated source code and logs of killed node.
> I guess that unexpected writes are executed when truncation is being
> executed.
>
> Some writes were executed after flush (the first flush) in truncation and
> these writes could be read.
> These writes were requested as MUTATION by another node for hinted handoff.
> Their data was stored to a new memtable and flushed (the second flush) to
> a new SSTable before snapshot in truncation.
> So, the truncation discarded only old SSTables, not the new SSTable.
> That's because ReplayPosition which was used for discarding SSTable was
> that of the first flush.
>
> I copied some parts of log as below.
> "##" line is my comment.
> The point is that the ReplayPosition is moved forward by the second flush.
> It means some writes are executed after the first flush.
>
> == log ==
> ## started truncation
> TRACE [SharedPool-Worker-16] 2016-11-17 08:36:04,612
> ColumnFamilyStore.java:2790 - truncating testtbl
> ## the first flush started before truncation
> DEBUG [SharedPool-Worker-16] 2016-11-17 08:36:04,612
> ColumnFamilyStore.java:952 - Enqueuing flush of testtbl: 591360 (0%)
> on-heap, 0 (0%) off-heap
> INFO  [MemtableFlushWriter:1] 2016-11-17 08:36:04,613 Memtable.java:352 -
> Writing Memtable-testtbl@1863835308(42.625KiB serialized bytes, 2816 ops,
> 0%/0% of on/off-heap limit)
> ...
> DEBUG [MemtableFlushWriter:1] 2016-11-17 08:36:04,973 Memtable.java:386 -
> Completed flushing
> /var/lib/cassandra/data/testdb/testtbl-562848f0a55611e68b1451065d58fdfb/tmp-lb-1-big-Data.db
> (17.651KiB) for commitlog position ReplayPosition(segmentId=1479371760395,
> position=315867)
> ## this ReplayPosition was used for discarding SSTables
> ...
> TRACE [MemtablePostFlush:1] 2016-11-17 08:36:05,022 CommitLog.java:298 -
> discard completed log segments for ReplayPosition(segmentId=1479371760395,
> position=315867), table 562848f0-a556-11e6-8b14-51065d58fdfb
> ## end of the first flush
> DEBUG [SharedPool-Worker-16] 2016-11-17 08:36:05,028
> ColumnFamilyStore.java:2823 - Discarding sstable data for truncated CF +
> indexes
> ## the second flush before snapshot
> DEBUG [SharedPool-Worker-16] 2016-11-17 08:36:05,028
> ColumnFamilyStore.java:952 - Enqueuing flush of testtbl: 698880 (0%)
> on-heap, 0 (0%) off-heap
> INFO  [MemtableFlushWriter:2] 2016-11-17 08:36:05,029 Memtable.java:352 -
> Writing Memtable-testtbl@1186728207(50.375KiB serialized bytes, 3328 ops,
> 0%/0% of on/off-heap limit)
> ...
> DEBUG [MemtableFlushWriter:2] 2016-11-17 08:36:05,258 Memtable.java:386 -
> Completed flushing
> /var/lib/cassandra/data/testdb/testtbl-562848f0a55611e68b1451065d58fdfb/tmp-lb-2-big-Data.db
> (17.696KiB) for commitlog position ReplayPosition(segmentId=1479371760395,
> position=486627)
> ...
> TRACE [MemtablePostFlush:1] 2016-11-17 08:36:05,289 CommitLog.java:298 -
> discard completed log segments for ReplayPosition(segmentId=1479371760395,
> position=486627), table 562848f0-a556-11e6-8b14-51065d58fdfb
> ## end of the second flush: position was moved
> ...
> ## only old SSTable was deleted because this SSTable was older than
> ReplayPosition(segmentId=1479371760395, position=315867)
> TRACE [NonPeriodicTasks:1] 2016-11-17 08:36:05,303 SSTable.java:118 -
> Deleted
> /var/lib/cassandra/data/testdb/testtbl-562848f0a55611e68b1451065d58fdfb/lb-1-big
> ...
> TRACE [SharedPool-Worker-16] 2016-11-17 08:36:05,320
> ColumnFamilyStore.java:2841 - truncate complete
> TRACE [SharedPool-Worker-16] 2016-11-17 08:36:05,320
> TruncateVerbHandler.java:53 - Truncation(keyspace='testdb', cf='testtbl')
> applied.  Enqueuing response to 36512@/10.91.145.7
> TRACE [SharedPool-Worker-16] 2016-11-17 08:36:05,320
> MessagingService.java:728 - /10.91.145.27 sending REQUEST_RESPONSE to
> 36512@/10.91.145.7
> ## end of truncation
> ====
>
> Actually, "truncated_at" of the table on the system.local after running
> the script was 0x00000158716da30b0004d1db00000158716db524.
> It means segmentId=1479371760395, position=315867
> truncated_at=1479371765028 (2016-11-17 08:36:05,028)
>
> thanks,
> yuji
>
>
> On Wed, Nov 16, 2016 at 5:25 PM, Yuji Ito <yu...@imagine-orb.com> wrote:
>
> Hi,
>
> I could find stale data after truncating a table.
> It seems that truncating starts while recovery is being executed just
> after a node restarts.
> After the truncating finishes, recovery still continues?
> Is it expected?
>
> I use C* 2.2.8 and can reproduce it as below.
>
> ==== [create table] ====
> cqlsh $ip -e "drop keyspace testdb;"
> cqlsh $ip -e "CREATE KEYSPACE testdb WITH replication = {'class':
> 'SimpleStrategy', 'replication_factor': '3'};"
> cqlsh $ip -e "CREATE TABLE testdb.testtbl (key int PRIMARY KEY, val int);"
>
> ==== [script] ====
> #!/bin/sh
>
> node1_ip=<node1 IP address>
> node2_ip=<node2 IP address>
> node3_ip=<node3 IP address>
> node3_user=<user name>
> rows=10000
>
> echo "consistency quorum;" > init_data.cql
> for key in $(seq 0 $(expr $rows - 1))
> do
>     echo "insert into testdb.testtbl (key, val) values($key, 1111) IF NOT
> EXISTS;" >> init_data.cql
> done
>
> while true
> do
> echo "truncate the table"
> cqlsh $node1_ip -e "truncate table testdb.testtbl"
> if [ $? -ne 0 ]; then
>     echo "truncating failed"
>     continue
> else
>     break
> fi
> done
>
> echo "kill C* process on node3"
> pdsh -l $node3_user -R ssh -w $node3_ip "ps auxww | grep CassandraDaemon |
> awk '{if (\$13 ~ /cassand/) print \$2}' | xargs sudo kill -9"
>
> echo "insert $rows rows"
> cqlsh $node1_ip -f init_data.cql > insert_log 2>&1
>
> echo "restart C* process on node3"
> pdsh -l $node3_user -R ssh -w $node3_ip "sudo /etc/init.d/cassandra start"
>
> while true
> do
> echo "truncate the table again"
> cqlsh $node1_ip -e "truncate table testdb.testtbl"
> if [ $? -ne 0 ]; then
>     echo "truncating failed"
>     continue
> else
>     break
> fi
> done
>
> cqlsh $node1_ip --request-timeout 3600 -e "consistency serial; select
> count(*) from testdb.testtbl;"
> sleep 10
> cqlsh $node1_ip --request-timeout 3600 -e "consistency serial; select
> count(*) from testdb.testtbl;"
>
>
> ==== [result] ====
> truncate the table
> kill C* process on node3
> insert 10000 rows
> restart C* process on node3
> 10.91.145.27: Starting Cassandra: OK
> truncate the table again
> <stdin>:1:TruncateError: Error during truncate: Cannot achieve consistency
> level ALL
> truncating failed
> truncate the table again
> <stdin>:1:TruncateError: Error during truncate: Cannot achieve consistency
> level ALL
> truncating failed
> truncate the table again
> <stdin>:1:TruncateError: Error during truncate: Cannot achieve consistency
> level ALL
> truncating failed
> truncate the table again
> <stdin>:1:TruncateError: Error during truncate: Cannot achieve consistency
> level ALL
> truncating failed
> truncate the table again
> <stdin>:1:TruncateError: Error during truncate: Cannot achieve consistency
> level ALL
> truncating failed
> truncate the table again
> <stdin>:1:TruncateError: Error during truncate: Cannot achieve consistency
> level ALL
> truncating failed
> truncate the table again
> Consistency level set to SERIAL.
>
>  count
> -------
>    300
>
> (1 rows)
>
> Warnings :
> Aggregation query used without partition key
>
> Consistency level set to SERIAL.
>
>  count
> -------
>   2304
>
> (1 rows)
>
> Warnings :
> Aggregation query used without partition key
> ====
>
> I found it when I was investigating data lost problem. (Ref. "failure node
> rejoin" thread)
> I'm not sure this problem is related to data lost.
>
> Thanks,
> yuji
>
>
>
>

Re: Does recovery continue after truncating a table?

Posted by Yuji Ito <yu...@imagine-orb.com>.
Hi all,

I revised the script to reproduce the issue.
I think the issue happens more frequently than before.
Killing another node is added to the previous script.

==== [script] ====
#!/bin/sh

node1_ip=<node1 IP address>
node2_ip=<node2 IP address>
node3_ip=<node3 IP address>
node2_user=<user name>
node3_user=<user name>
rows=10000

echo "consistency quorum;" > init_data.cql
for key in $(seq 0 $(expr $rows - 1))
do
    echo "insert into testdb.testtbl (key, val) values($key, 1111) IF NOT
EXISTS;" >> init_data.cql
    done

    while true
    do
    echo "truncate the table"
    cqlsh $node1_ip -e "truncate table testdb.testtbl" > /dev/null 2>&1
    if [ $? -ne 0 ]; then
        echo "truncating failed"
    continue
    else
        break
    fi
done

echo "kill C* process on node3"
pdsh -l $node3_user -R ssh -w $node3_ip "ps auxww | grep CassandraDaemon |
awk '{if (\$13 ~ /cassand/) print \$2}' | xargs sudo kill -9"

echo "insert $rows rows"
cqlsh $node1_ip -f init_data.cql > insert_log 2>&1

echo "restart C* process on node3"
pdsh -l $node3_user -R ssh -w $node3_ip "sudo /etc/init.d/cassandra start"

while true
do
echo "truncate the table again"
cqlsh $node1_ip -e "truncate table testdb.testtbl"
if [ $? -ne 0 ]; then
    echo "truncating failed"
        continue
else
    echo "truncation succeeded!"
    break
fi
done

echo "kill C* process on node2"
pdsh -l $node2_user -R ssh -w $node2_ip "ps auxww | grep CassandraDaemon |
awk '{if (\$13 ~ /cassand/) print \$2}' | xargs sudo kill -9"

cqlsh $node1_ip --request-timeout 3600 -e "consistency serial; select
count(*) from testdb.testtbl;"
sleep 10
cqlsh $node1_ip --request-timeout 3600 -e "consistency serial; select
count(*) from testdb.testtbl;"

echo "restart C* process on node2"
pdsh -l $node2_user -R ssh -w $node2_ip "sudo /etc/init.d/cassandra start"


Thanks,
yuji


On Fri, Nov 18, 2016 at 7:52 PM, Yuji Ito <yu...@imagine-orb.com> wrote:

> I investigated source code and logs of killed node.
> I guess that unexpected writes are executed when truncation is being
> executed.
>
> Some writes were executed after flush (the first flush) in truncation and
> these writes could be read.
> These writes were requested as MUTATION by another node for hinted handoff.
> Their data was stored to a new memtable and flushed (the second flush) to
> a new SSTable before snapshot in truncation.
> So, the truncation discarded only old SSTables, not the new SSTable.
> That's because ReplayPosition which was used for discarding SSTable was
> that of the first flush.
>
> I copied some parts of log as below.
> "##" line is my comment.
> The point is that the ReplayPosition is moved forward by the second flush.
> It means some writes are executed after the first flush.
>
> == log ==
> ## started truncation
> TRACE [SharedPool-Worker-16] 2016-11-17 08:36:04,612
> ColumnFamilyStore.java:2790 - truncating testtbl
> ## the first flush started before truncation
> DEBUG [SharedPool-Worker-16] 2016-11-17 08:36:04,612
> ColumnFamilyStore.java:952 - Enqueuing flush of testtbl: 591360 (0%)
> on-heap, 0 (0%) off-heap
> INFO  [MemtableFlushWriter:1] 2016-11-17 08:36:04,613 Memtable.java:352 -
> Writing Memtable-testtbl@1863835308(42.625KiB serialized bytes, 2816 ops,
> 0%/0% of on/off-heap limit)
> ...
> DEBUG [MemtableFlushWriter:1] 2016-11-17 08:36:04,973 Memtable.java:386 -
> Completed flushing /var/lib/cassandra/data/testdb
> /testtbl-562848f0a55611e68b1451065d58fdfb/tmp-lb-1-big-Data.db
> (17.651KiB) for commitlog position ReplayPosition(segmentId=1479371760395,
> position=315867)
> ## this ReplayPosition was used for discarding SSTables
> ...
> TRACE [MemtablePostFlush:1] 2016-11-17 08:36:05,022 CommitLog.java:298 -
> discard completed log segments for ReplayPosition(segmentId=1479371760395,
> position=315867), table 562848f0-a556-11e6-8b14-51065d58fdfb
> ## end of the first flush
> DEBUG [SharedPool-Worker-16] 2016-11-17 08:36:05,028
> ColumnFamilyStore.java:2823 - Discarding sstable data for truncated CF +
> indexes
> ## the second flush before snapshot
> DEBUG [SharedPool-Worker-16] 2016-11-17 08:36:05,028
> ColumnFamilyStore.java:952 - Enqueuing flush of testtbl: 698880 (0%)
> on-heap, 0 (0%) off-heap
> INFO  [MemtableFlushWriter:2] 2016-11-17 08:36:05,029 Memtable.java:352 -
> Writing Memtable-testtbl@1186728207(50.375KiB serialized bytes, 3328 ops,
> 0%/0% of on/off-heap limit)
> ...
> DEBUG [MemtableFlushWriter:2] 2016-11-17 08:36:05,258 Memtable.java:386 -
> Completed flushing /var/lib/cassandra/data/testdb
> /testtbl-562848f0a55611e68b1451065d58fdfb/tmp-lb-2-big-Data.db
> (17.696KiB) for commitlog position ReplayPosition(segmentId=1479371760395,
> position=486627)
> ...
> TRACE [MemtablePostFlush:1] 2016-11-17 08:36:05,289 CommitLog.java:298 -
> discard completed log segments for ReplayPosition(segmentId=1479371760395,
> position=486627), table 562848f0-a556-11e6-8b14-51065d58fdfb
> ## end of the second flush: position was moved
> ...
> ## only old SSTable was deleted because this SSTable was older than
> ReplayPosition(segmentId=1479371760395, position=315867)
> TRACE [NonPeriodicTasks:1] 2016-11-17 08:36:05,303 SSTable.java:118 -
> Deleted /var/lib/cassandra/data/testdb/testtbl-562848f0a55611e68b145
> 1065d58fdfb/lb-1-big
> ...
> TRACE [SharedPool-Worker-16] 2016-11-17 08:36:05,320
> ColumnFamilyStore.java:2841 - truncate complete
> TRACE [SharedPool-Worker-16] 2016-11-17 08:36:05,320
> TruncateVerbHandler.java:53 - Truncation(keyspace='testdb', cf='testtbl')
> applied.  Enqueuing response to 36512@/10.91.145.7
> TRACE [SharedPool-Worker-16] 2016-11-17 08:36:05,320
> MessagingService.java:728 - /10.91.145.27 sending REQUEST_RESPONSE to
> 36512@/10.91.145.7
> ## end of truncation
> ====
>
> Actually, "truncated_at" of the table on the system.local after running
> the script was 0x00000158716da30b0004d1db00000158716db524.
> It means segmentId=1479371760395, position=315867
> truncated_at=1479371765028 (2016-11-17 08:36:05,028)
>
> thanks,
> yuji
>
>
> On Wed, Nov 16, 2016 at 5:25 PM, Yuji Ito <yu...@imagine-orb.com> wrote:
>
>> Hi,
>>
>> I could find stale data after truncating a table.
>> It seems that truncating starts while recovery is being executed just
>> after a node restarts.
>> After the truncating finishes, recovery still continues?
>> Is it expected?
>>
>> I use C* 2.2.8 and can reproduce it as below.
>>
>> ==== [create table] ====
>> cqlsh $ip -e "drop keyspace testdb;"
>> cqlsh $ip -e "CREATE KEYSPACE testdb WITH replication = {'class':
>> 'SimpleStrategy', 'replication_factor': '3'};"
>> cqlsh $ip -e "CREATE TABLE testdb.testtbl (key int PRIMARY KEY, val int);"
>>
>> ==== [script] ====
>> #!/bin/sh
>>
>> node1_ip=<node1 IP address>
>> node2_ip=<node2 IP address>
>> node3_ip=<node3 IP address>
>> node3_user=<user name>
>> rows=10000
>>
>> echo "consistency quorum;" > init_data.cql
>> for key in $(seq 0 $(expr $rows - 1))
>> do
>>     echo "insert into testdb.testtbl (key, val) values($key, 1111) IF NOT
>> EXISTS;" >> init_data.cql
>> done
>>
>> while true
>> do
>> echo "truncate the table"
>> cqlsh $node1_ip -e "truncate table testdb.testtbl"
>> if [ $? -ne 0 ]; then
>>     echo "truncating failed"
>>     continue
>> else
>>     break
>> fi
>> done
>>
>> echo "kill C* process on node3"
>> pdsh -l $node3_user -R ssh -w $node3_ip "ps auxww | grep CassandraDaemon
>> | awk '{if (\$13 ~ /cassand/) print \$2}' | xargs sudo kill -9"
>>
>> echo "insert $rows rows"
>> cqlsh $node1_ip -f init_data.cql > insert_log 2>&1
>>
>> echo "restart C* process on node3"
>> pdsh -l $node3_user -R ssh -w $node3_ip "sudo /etc/init.d/cassandra start"
>>
>> while true
>> do
>> echo "truncate the table again"
>> cqlsh $node1_ip -e "truncate table testdb.testtbl"
>> if [ $? -ne 0 ]; then
>>     echo "truncating failed"
>>     continue
>> else
>>     break
>> fi
>> done
>>
>> cqlsh $node1_ip --request-timeout 3600 -e "consistency serial; select
>> count(*) from testdb.testtbl;"
>> sleep 10
>> cqlsh $node1_ip --request-timeout 3600 -e "consistency serial; select
>> count(*) from testdb.testtbl;"
>>
>>
>> ==== [result] ====
>> truncate the table
>> kill C* process on node3
>> insert 10000 rows
>> restart C* process on node3
>> 10.91.145.27: Starting Cassandra: OK
>> truncate the table again
>> <stdin>:1:TruncateError: Error during truncate: Cannot achieve
>> consistency level ALL
>> truncating failed
>> truncate the table again
>> <stdin>:1:TruncateError: Error during truncate: Cannot achieve
>> consistency level ALL
>> truncating failed
>> truncate the table again
>> <stdin>:1:TruncateError: Error during truncate: Cannot achieve
>> consistency level ALL
>> truncating failed
>> truncate the table again
>> <stdin>:1:TruncateError: Error during truncate: Cannot achieve
>> consistency level ALL
>> truncating failed
>> truncate the table again
>> <stdin>:1:TruncateError: Error during truncate: Cannot achieve
>> consistency level ALL
>> truncating failed
>> truncate the table again
>> <stdin>:1:TruncateError: Error during truncate: Cannot achieve
>> consistency level ALL
>> truncating failed
>> truncate the table again
>> Consistency level set to SERIAL.
>>
>>  count
>> -------
>>    300
>>
>> (1 rows)
>>
>> Warnings :
>> Aggregation query used without partition key
>>
>> Consistency level set to SERIAL.
>>
>>  count
>> -------
>>   2304
>>
>> (1 rows)
>>
>> Warnings :
>> Aggregation query used without partition key
>> ====
>>
>> I found it when I was investigating data lost problem. (Ref. "failure
>> node rejoin" thread)
>> I'm not sure this problem is related to data lost.
>>
>> Thanks,
>> yuji
>>
>
>

Re: Does recovery continue after truncating a table?

Posted by Yuji Ito <yu...@imagine-orb.com>.
I investigated source code and logs of killed node.
I guess that unexpected writes are executed when truncation is being
executed.

Some writes were executed after flush (the first flush) in truncation and
these writes could be read.
These writes were requested as MUTATION by another node for hinted handoff.
Their data was stored to a new memtable and flushed (the second flush) to a
new SSTable before snapshot in truncation.
So, the truncation discarded only old SSTables, not the new SSTable.
That's because ReplayPosition which was used for discarding SSTable was
that of the first flush.

I copied some parts of log as below.
"##" line is my comment.
The point is that the ReplayPosition is moved forward by the second flush.
It means some writes are executed after the first flush.

== log ==
## started truncation
TRACE [SharedPool-Worker-16] 2016-11-17 08:36:04,612
ColumnFamilyStore.java:2790 - truncating testtbl
## the first flush started before truncation
DEBUG [SharedPool-Worker-16] 2016-11-17 08:36:04,612
ColumnFamilyStore.java:952 - Enqueuing flush of testtbl: 591360 (0%)
on-heap, 0 (0%) off-heap
INFO  [MemtableFlushWriter:1] 2016-11-17 08:36:04,613 Memtable.java:352 -
Writing Memtable-testtbl@1863835308(42.625KiB serialized bytes, 2816 ops,
0%/0% of on/off-heap limit)
...
DEBUG [MemtableFlushWriter:1] 2016-11-17 08:36:04,973 Memtable.java:386 -
Completed flushing /var/lib/cassandra/data/testdb/testtbl-
562848f0a55611e68b1451065d58fdfb/tmp-lb-1-big-Data.db (17.651KiB) for
commitlog position ReplayPosition(segmentId=1479371760395, position=315867)
## this ReplayPosition was used for discarding SSTables
...
TRACE [MemtablePostFlush:1] 2016-11-17 08:36:05,022 CommitLog.java:298 -
discard completed log segments for ReplayPosition(segmentId=1479371760395,
position=315867), table 562848f0-a556-11e6-8b14-51065d58fdfb
## end of the first flush
DEBUG [SharedPool-Worker-16] 2016-11-17 08:36:05,028
ColumnFamilyStore.java:2823 - Discarding sstable data for truncated CF +
indexes
## the second flush before snapshot
DEBUG [SharedPool-Worker-16] 2016-11-17 08:36:05,028
ColumnFamilyStore.java:952 - Enqueuing flush of testtbl: 698880 (0%)
on-heap, 0 (0%) off-heap
INFO  [MemtableFlushWriter:2] 2016-11-17 08:36:05,029 Memtable.java:352 -
Writing Memtable-testtbl@1186728207(50.375KiB serialized bytes, 3328 ops,
0%/0% of on/off-heap limit)
...
DEBUG [MemtableFlushWriter:2] 2016-11-17 08:36:05,258 Memtable.java:386 -
Completed flushing /var/lib/cassandra/data/testdb/testtbl-
562848f0a55611e68b1451065d58fdfb/tmp-lb-2-big-Data.db (17.696KiB) for
commitlog position ReplayPosition(segmentId=1479371760395, position=486627)
...
TRACE [MemtablePostFlush:1] 2016-11-17 08:36:05,289 CommitLog.java:298 -
discard completed log segments for ReplayPosition(segmentId=1479371760395,
position=486627), table 562848f0-a556-11e6-8b14-51065d58fdfb
## end of the second flush: position was moved
...
## only old SSTable was deleted because this SSTable was older than
ReplayPosition(segmentId=1479371760395, position=315867)
TRACE [NonPeriodicTasks:1] 2016-11-17 08:36:05,303 SSTable.java:118 -
Deleted /var/lib/cassandra/data/testdb/testtbl-
562848f0a55611e68b1451065d58fdfb/lb-1-big
...
TRACE [SharedPool-Worker-16] 2016-11-17 08:36:05,320
ColumnFamilyStore.java:2841 - truncate complete
TRACE [SharedPool-Worker-16] 2016-11-17 08:36:05,320
TruncateVerbHandler.java:53 - Truncation(keyspace='testdb', cf='testtbl')
applied.  Enqueuing response to 36512@/10.91.145.7
TRACE [SharedPool-Worker-16] 2016-11-17 08:36:05,320
MessagingService.java:728 - /10.91.145.27 sending REQUEST_RESPONSE to 36512@
/10.91.145.7
## end of truncation
====

Actually, "truncated_at" of the table on the system.local after running the
script was 0x00000158716da30b0004d1db00000158716db524.
It means segmentId=1479371760395, position=315867
truncated_at=1479371765028 (2016-11-17 08:36:05,028)

thanks,
yuji


On Wed, Nov 16, 2016 at 5:25 PM, Yuji Ito <yu...@imagine-orb.com> wrote:

> Hi,
>
> I could find stale data after truncating a table.
> It seems that truncating starts while recovery is being executed just
> after a node restarts.
> After the truncating finishes, recovery still continues?
> Is it expected?
>
> I use C* 2.2.8 and can reproduce it as below.
>
> ==== [create table] ====
> cqlsh $ip -e "drop keyspace testdb;"
> cqlsh $ip -e "CREATE KEYSPACE testdb WITH replication = {'class':
> 'SimpleStrategy', 'replication_factor': '3'};"
> cqlsh $ip -e "CREATE TABLE testdb.testtbl (key int PRIMARY KEY, val int);"
>
> ==== [script] ====
> #!/bin/sh
>
> node1_ip=<node1 IP address>
> node2_ip=<node2 IP address>
> node3_ip=<node3 IP address>
> node3_user=<user name>
> rows=10000
>
> echo "consistency quorum;" > init_data.cql
> for key in $(seq 0 $(expr $rows - 1))
> do
>     echo "insert into testdb.testtbl (key, val) values($key, 1111) IF NOT
> EXISTS;" >> init_data.cql
> done
>
> while true
> do
> echo "truncate the table"
> cqlsh $node1_ip -e "truncate table testdb.testtbl"
> if [ $? -ne 0 ]; then
>     echo "truncating failed"
>     continue
> else
>     break
> fi
> done
>
> echo "kill C* process on node3"
> pdsh -l $node3_user -R ssh -w $node3_ip "ps auxww | grep CassandraDaemon |
> awk '{if (\$13 ~ /cassand/) print \$2}' | xargs sudo kill -9"
>
> echo "insert $rows rows"
> cqlsh $node1_ip -f init_data.cql > insert_log 2>&1
>
> echo "restart C* process on node3"
> pdsh -l $node3_user -R ssh -w $node3_ip "sudo /etc/init.d/cassandra start"
>
> while true
> do
> echo "truncate the table again"
> cqlsh $node1_ip -e "truncate table testdb.testtbl"
> if [ $? -ne 0 ]; then
>     echo "truncating failed"
>     continue
> else
>     break
> fi
> done
>
> cqlsh $node1_ip --request-timeout 3600 -e "consistency serial; select
> count(*) from testdb.testtbl;"
> sleep 10
> cqlsh $node1_ip --request-timeout 3600 -e "consistency serial; select
> count(*) from testdb.testtbl;"
>
>
> ==== [result] ====
> truncate the table
> kill C* process on node3
> insert 10000 rows
> restart C* process on node3
> 10.91.145.27: Starting Cassandra: OK
> truncate the table again
> <stdin>:1:TruncateError: Error during truncate: Cannot achieve consistency
> level ALL
> truncating failed
> truncate the table again
> <stdin>:1:TruncateError: Error during truncate: Cannot achieve consistency
> level ALL
> truncating failed
> truncate the table again
> <stdin>:1:TruncateError: Error during truncate: Cannot achieve consistency
> level ALL
> truncating failed
> truncate the table again
> <stdin>:1:TruncateError: Error during truncate: Cannot achieve consistency
> level ALL
> truncating failed
> truncate the table again
> <stdin>:1:TruncateError: Error during truncate: Cannot achieve consistency
> level ALL
> truncating failed
> truncate the table again
> <stdin>:1:TruncateError: Error during truncate: Cannot achieve consistency
> level ALL
> truncating failed
> truncate the table again
> Consistency level set to SERIAL.
>
>  count
> -------
>    300
>
> (1 rows)
>
> Warnings :
> Aggregation query used without partition key
>
> Consistency level set to SERIAL.
>
>  count
> -------
>   2304
>
> (1 rows)
>
> Warnings :
> Aggregation query used without partition key
> ====
>
> I found it when I was investigating data lost problem. (Ref. "failure node
> rejoin" thread)
> I'm not sure this problem is related to data lost.
>
> Thanks,
> yuji
>