[
https://issues.apache.org/jira/browse/CASSANDRA-3735?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13190984#comment-13190984
]
Sylvain Lebresne commented on CASSANDRA-3735:
---------------------------------------------
This seems to handle most of the warnings. However (on 1.0 branch):
* SSTableReaderTest doesn't pass on linux either. It ends up with the following
trace:
{noformat}
[junit] Testcase:
testPersistentStatisticsFromOlderIndexedSSTable(org.apache.cassandra.io.sstable.SSTableReaderTest):
Caused an ERROR
[junit] Failed to delete
/home/mcmanus/Git/cassandra/build/test/cassandra/data/Keyspace1/backups
[junit] java.io.IOException: Failed to delete
/home/mcmanus/Git/cassandra/build/test/cassandra/data/Keyspace1/backups
[junit] at
org.apache.cassandra.io.util.FileUtils.deleteWithConfirm(FileUtils.java:54)
[junit] at
org.apache.cassandra.io.util.FileUtils.deleteRecursive(FileUtils.java:237)
[junit] at
org.apache.cassandra.io.sstable.SSTableReaderTest.clearAndLoad(SSTableReaderTest.java:167)
[junit] at
org.apache.cassandra.io.sstable.SSTableReaderTest.assertIndexQueryWorks(SSTableReaderTest.java:260)
[junit] at
org.apache.cassandra.io.sstable.SSTableReaderTest.testPersistentStatisticsFromOlderIndexedSSTable(SSTableReaderTest.java:251)
{noformat}
* I get a "Largest generation seen in loaded sstables was 8, which may overlap
with native sstable files (generation 8)" warning during ColumnFamilyStoreTest,
followed by a (not surpising) 'Unable to create hard link', in
testSliceByNamesCommandOldMetatada. I believe a quick fix could be to have
clearUnsafe reset the generation for the CF. Another solution would be to make
loadNewSSTables smarter/less fragile by having it 'reserve' a generation for
each of the new file to load from the current generation and rename the loaded
files accordingly.
> Fix "Unable to create hard link" SSTableReaderTest error messages
> -----------------------------------------------------------------
>
> Key: CASSANDRA-3735
> URL: https://issues.apache.org/jira/browse/CASSANDRA-3735
> Project: Cassandra
> Issue Type: Bug
> Reporter: Jonathan Ellis
> Assignee: Jonathan Ellis
> Priority: Minor
> Attachments: 0001-fix-generation-update-in-loadNewSSTables.patch,
> 0002-remove-incremental-backups-before-reloading-sstables.patch
>
>
> Sample failure (on Windows):
> {noformat}
> [junit] java.io.IOException: Exception while executing the command: cmd
> /c mklink /H
> C:\Users\Jonathan\projects\cassandra\git\build\test\cassandra\data\Keyspace1\backups\Standard1-hc-1-Index.db
>
> c:\Users\Jonathan\projects\cassandra\git\build\test\cassandra\data\Keyspace1\Standard1-hc-1-Index.db,command
> error Code: 1, command output: Cannot create a file when that file already
> exists.
> [junit]
> [junit] at org.apache.cassandra.utils.CLibrary.exec(CLibrary.java:213)
> [junit] at
> org.apache.cassandra.utils.CLibrary.createHardLinkWithExec(CLibrary.java:188)
> [junit] at
> org.apache.cassandra.utils.CLibrary.createHardLink(CLibrary.java:151)
> [junit] at
> org.apache.cassandra.io.sstable.SSTableReader.createLinks(SSTableReader.java:833)
> [junit] at
> org.apache.cassandra.db.DataTracker$1.runMayThrow(DataTracker.java:161)
> [junit] at
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> [junit] at
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
> [junit] at
> java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
> [junit] at java.util.concurrent.FutureTask.run(FutureTask.java:138)
> [junit] at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:98)
> [junit] at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:206)
> [junit] at
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> [junit] at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> [junit] at java.lang.Thread.run(Thread.java:662)
> [junit] ERROR 17:10:17,111 Fatal exception in thread
> Thread[NonPeriodicTasks:1,5,main]
> {noformat}
--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators:
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira