[jira] [Commented] (CASSANDRA-5116) upgradesstables does not upgrade all sstables on a node

2013-03-23 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis commented on CASSANDRA-5116:
---

We skip (1) currently-being-compacted sstables and (2) sstables that are 
blacklisted for erroring out.

> upgradesstables does not upgrade all sstables on a node
> ---
>
> Key: CASSANDRA-5116
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5116
> Project: Cassandra
>  Issue Type: Bug
>Affects Versions: 1.2.0
> Environment: Ubuntu 12.04
>Reporter: Michael Kjellman
>
> upgradesstables appears to be skipping sstables randomly.
> finding a sstable with an mtime < the upgrade time and grepping through the 
> logs for a corresponding compaction log line, i find nothing. I have 
> reproduced this on all of my nodes across the cluster.
> is performAllSSTableOperation somehow skipping sstables?

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5116) upgradesstables does not upgrade all sstables on a node

2013-02-14 Thread Jonathan Ellis (JIRA)

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

Jonathan Ellis commented on CASSANDRA-5116:
---

Confused about #1 -- sounds like you're saying that we don't mark upgrade 
targets as compacting?  That's done by performAllSSTableOperation.  Probably 
I'm misunderstanding...

> upgradesstables does not upgrade all sstables on a node
> ---
>
> Key: CASSANDRA-5116
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5116
> Project: Cassandra
>  Issue Type: Bug
>Affects Versions: 1.2.0
> Environment: Ubuntu 12.04
>Reporter: Michael Kjellman
>Assignee: Yuki Morishita
>
> upgradesstables appears to be skipping sstables randomly.
> finding a sstable with an mtime < the upgrade time and grepping through the 
> logs for a corresponding compaction log line, i find nothing. I have 
> reproduced this on all of my nodes across the cluster.
> is performAllSSTableOperation somehow skipping sstables?

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5116) upgradesstables does not upgrade all sstables on a node

2013-01-07 Thread Yuki Morishita (JIRA)

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

Yuki Morishita commented on CASSANDRA-5116:
---

Couple of things I noticed during investigation:

  * As Michael commented above, there are some sstables that are not referenced 
by CF. That happens when those sstables are ancestors of other sstables(see 
https://github.com/apache/cassandra/blob/cassandra-1.2.0/src/java/org/apache/cassandra/db/ColumnFamilyStore.java#L267).
 Those sstables are supposed to be deleted through 
SSTableReader#releaseReference() from the comment below that line. But sstables 
would not be deleted since releaseReferece() checks isCompacted flag which, as 
the name suggests, only becomes true when compacted. There is no log about this 
behavior, so when that happens after upgradesstables, I think it is confusing. 
So I propose fix for actually deleting file + logging when that happens.
  * Maybe we can raise priority of updatesstables task among other compaction 
tasks to be processed as first as possible.
  * nodetool upgradesstables just submits task for upgrade but never knows when 
it finishes. It may be useful if we track the progress of upgrade like 
CASSANDRA-4767. (same can be said to cleanup, scrub, etc).

> upgradesstables does not upgrade all sstables on a node
> ---
>
> Key: CASSANDRA-5116
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5116
> Project: Cassandra
>  Issue Type: Bug
>Affects Versions: 1.2.0
> Environment: Ubuntu 12.04
>Reporter: Michael Kjellman
>Assignee: Yuki Morishita
>
> upgradesstables appears to be skipping sstables randomly.
> finding a sstable with an mtime < the upgrade time and grepping through the 
> logs for a corresponding compaction log line, i find nothing. I have 
> reproduced this on all of my nodes across the cluster.
> is performAllSSTableOperation somehow skipping sstables?

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5116) upgradesstables does not upgrade all sstables on a node

2013-01-07 Thread Michael Kjellman (JIRA)

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

Michael Kjellman commented on CASSANDRA-5116:
-

one other note:

if the Summary.db file has incorrect permissions, a restart of the node to 
re-read in the sstable is required before the sstable will get compacted.

> upgradesstables does not upgrade all sstables on a node
> ---
>
> Key: CASSANDRA-5116
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5116
> Project: Cassandra
>  Issue Type: Bug
>Affects Versions: 1.2.0
> Environment: Ubuntu 12.04
>Reporter: Michael Kjellman
>Assignee: Michael Kjellman
>
> upgradesstables appears to be skipping sstables randomly.
> finding a sstable with an mtime < the upgrade time and grepping through the 
> logs for a corresponding compaction log line, i find nothing. I have 
> reproduced this on all of my nodes across the cluster.
> is performAllSSTableOperation somehow skipping sstables?

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5116) upgradesstables does not upgrade all sstables on a node

2013-01-06 Thread Michael Kjellman (JIRA)

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

Michael Kjellman commented on CASSANDRA-5116:
-

during a repair -pr operation on a node that had a cleanup and upgrade i just 
saw an interesting log line

{code}
 INFO [CompactionExecutor:71] 2013-01-06 00:18:36,503 CompactionTask.java (line 
116) Compacting 
[SSTableReader(path='/data/cassandra/brts/domain_metadata/brts-domain_metadata-ia-1105-Data.db'),
 
SSTableReader(path='/data/cassandra/brts/domain_metadata/brts-domain_metadata-ia-1108-Data.db'),
 
SSTableReader(path='/data/cassandra/brts/domain_metadata/brts-domain_metadata-hf-693-Data.db'),
 
SSTableReader(path='/data/cassandra/brts/domain_metadata/brts-domain_metadata-hf-694-Data.db'),
 
SSTableReader(path='/data/cassandra/brts/domain_metadata/brts-domain_metadata-ia-1110-Data.db'),
 
SSTableReader(path='/data/cassandra/brts/domain_metadata/brts-domain_metadata-hf-1024-Data.db'),
 
SSTableReader(path='/data/cassandra/brts/domain_metadata/brts-domain_metadata-ia-1106-Data.db'),
 
SSTableReader(path='/data/cassandra/brts/domain_metadata/brts-domain_metadata-ia-1104-Data.db'),
 
SSTableReader(path='/data/cassandra/brts/domain_metadata/brts-domain_metadata-ia-1109-Data.db'),
 
SSTableReader(path='/data/cassandra/brts/domain_metadata/brts-domain_metadata-ia-1107-Data.db'),
 
SSTableReader(path='/data/cassandra/brts/domain_metadata/brts-domain_metadata-hf-1025-Data.db')]
{code}

as you can see "hf" is the 1.1.6 format and "ia" is the 1.2.0 format. A 
nodetool repair -pr has triggered a compaction of both formats.

So repair -pr somehow triggers a compaction on the non "active" sstables but 
standard cleanup and upgradesstables operations do not.

> upgradesstables does not upgrade all sstables on a node
> ---
>
> Key: CASSANDRA-5116
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5116
> Project: Cassandra
>  Issue Type: Bug
>Affects Versions: 1.2.0
> Environment: Ubuntu 12.04
>Reporter: Michael Kjellman
>Assignee: Yuki Morishita
>
> upgradesstables appears to be skipping sstables randomly.
> finding a sstable with an mtime < the upgrade time and grepping through the 
> logs for a corresponding compaction log line, i find nothing. I have 
> reproduced this on all of my nodes across the cluster.
> is performAllSSTableOperation somehow skipping sstables?

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5116) upgradesstables does not upgrade all sstables on a node

2013-01-06 Thread Michael Kjellman (JIRA)

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

Michael Kjellman commented on CASSANDRA-5116:
-

scrub, repair, and upgrade all seem to skip the same sstables which seems to go 
back to the original guess that performAllSSTableOperation is skipping random 
sstables.

when the node is restarted the sstables in question that have yet to be 
upgraded are logged as "Opening" and no exceptions are thrown. Additionally, 
other nodes attempt to stream these sstables which throws an exception as these 
sstables are not upgraded yet even when these other operations do not seem to 
act on these sstables at all.

> upgradesstables does not upgrade all sstables on a node
> ---
>
> Key: CASSANDRA-5116
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5116
> Project: Cassandra
>  Issue Type: Bug
>Affects Versions: 1.2.0
> Environment: Ubuntu 12.04
>Reporter: Michael Kjellman
>Assignee: Yuki Morishita
>
> upgradesstables appears to be skipping sstables randomly.
> finding a sstable with an mtime < the upgrade time and grepping through the 
> logs for a corresponding compaction log line, i find nothing. I have 
> reproduced this on all of my nodes across the cluster.
> is performAllSSTableOperation somehow skipping sstables?

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5116) upgradesstables does not upgrade all sstables on a node

2013-01-05 Thread Michael Kjellman (JIRA)

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

Michael Kjellman commented on CASSANDRA-5116:
-

doing a cleanup prior to a upgradesstables still results in sstables that were 
not upgraded

> upgradesstables does not upgrade all sstables on a node
> ---
>
> Key: CASSANDRA-5116
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5116
> Project: Cassandra
>  Issue Type: Bug
>Affects Versions: 1.2.0
> Environment: Ubuntu 12.04
>Reporter: Michael Kjellman
>Assignee: Yuki Morishita
>
> upgradesstables appears to be skipping sstables randomly.
> finding a sstable with an mtime < the upgrade time and grepping through the 
> logs for a corresponding compaction log line, i find nothing. I have 
> reproduced this on all of my nodes across the cluster.
> is performAllSSTableOperation somehow skipping sstables?

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5116) upgradesstables does not upgrade all sstables on a node

2013-01-05 Thread Michael Kjellman (JIRA)

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

Michael Kjellman commented on CASSANDRA-5116:
-

caches and the system cache had no effect on this so not sure if the cleanup 
was the reason more hf files were compacted during the upgrade. I did notice 
that after i restarted the node, caches etc there were 4 out of 20-30+ pending 
sstables that did get upgraded. so it's not like certain sstables refuse to get 
upgraded.

> upgradesstables does not upgrade all sstables on a node
> ---
>
> Key: CASSANDRA-5116
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5116
> Project: Cassandra
>  Issue Type: Bug
>Affects Versions: 1.2.0
> Environment: Ubuntu 12.04
>Reporter: Michael Kjellman
>Assignee: Yuki Morishita
>
> upgradesstables appears to be skipping sstables randomly.
> finding a sstable with an mtime < the upgrade time and grepping through the 
> logs for a corresponding compaction log line, i find nothing. I have 
> reproduced this on all of my nodes across the cluster.
> is performAllSSTableOperation somehow skipping sstables?

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5116) upgradesstables does not upgrade all sstables on a node

2013-01-04 Thread Michael Kjellman (JIRA)

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

Michael Kjellman commented on CASSANDRA-5116:
-

more on this:

1) did a cleanup on the node...the sstables that weren't upgraded were left 
alone by the cleanup operation
2) the log files show cassandra opening the sstable
3) i set row and key caches to 0, cleared everything in saved_caches and 
physically restarted the node to clear the kernel cache, i've now seen at least 
one of the sstables getting upgraded in my 4th attempt on this particular node 
to upgrade sstables.

developing... :)

> upgradesstables does not upgrade all sstables on a node
> ---
>
> Key: CASSANDRA-5116
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5116
> Project: Cassandra
>  Issue Type: Bug
>Affects Versions: 1.2.0
> Environment: Ubuntu 12.04
>Reporter: Michael Kjellman
>Assignee: Yuki Morishita
>
> upgradesstables appears to be skipping sstables randomly.
> finding a sstable with an mtime < the upgrade time and grepping through the 
> logs for a corresponding compaction log line, i find nothing. I have 
> reproduced this on all of my nodes across the cluster.
> is performAllSSTableOperation somehow skipping sstables?

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5116) upgradesstables does not upgrade all sstables on a node

2013-01-04 Thread Michael Kjellman (JIRA)

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

Michael Kjellman commented on CASSANDRA-5116:
-

when attempting to do a forceUserDefinedCompaction on the sstable's in question 
this is logged:

{code}
 INFO 12:50:39,748 Will not compact ks/ks-evidence_index-hf-70: it is not an 
active sstable
 INFO 12:50:39,749 No file to compact for user defined compaction
{code}

which according to the source looks like lookupSSTable() returned null.

which begs the question then, why are these sstables still getting streamed 
then?

> upgradesstables does not upgrade all sstables on a node
> ---
>
> Key: CASSANDRA-5116
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5116
> Project: Cassandra
>  Issue Type: Bug
>Affects Versions: 1.2.0
> Environment: Ubuntu 12.04
>Reporter: Michael Kjellman
>
> upgradesstables appears to be skipping sstables randomly.
> finding a sstable with an mtime < the upgrade time and grepping through the 
> logs for a corresponding compaction log line, i find nothing. I have 
> reproduced this on all of my nodes across the cluster.
> is performAllSSTableOperation somehow skipping sstables?

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira