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

Brandon Williams commented on CASSANDRA-3045:
---------------------------------------------

Specifically, on line 97 FileStreamTask has a dangling semicolon that causes 
the 'if' to always fire.  However, removing that still causes a secondary index 
check to fail in StreamingTransferTest.   Here is a debug log of a working test:
{noformat}
INFO [Thread-3] 2011-11-18 17:36:04,039 SecondaryIndexManager.java (line 115) 
Submitting index build of 626972746864617465, for data in 
SSTableReader(path='build/test/cassandra/data/Keyspace1/Indexed1-hb-4-Data.db')
DEBUG [CompactionExecutor:1] 2011-11-18 17:36:04,041 Table.java (line 515) 
Indexing row key1
DEBUG [CompactionExecutor:1] 2011-11-18 17:36:04,042 CollationController.java 
(line 76) collectTimeOrderedData
DEBUG [CompactionExecutor:1] 2011-11-18 17:36:04,042 KeysIndex.java (line 100) 
applying index row 3288498 in ColumnFamily(Indexed1.626972746864617465 
[6b657931:false:0@1234,])
DEBUG [CompactionExecutor:1] 2011-11-18 17:36:04,043 SlabAllocator.java (line 
105) 1 regions now allocated in org.apache.cassandra.utils.SlabAllocator@1716fa0
DEBUG [CompactionExecutor:1] 2011-11-18 17:36:04,043 Table.java (line 515) 
Indexing row key3
DEBUG [CompactionExecutor:1] 2011-11-18 17:36:04,043 CollationController.java 
(line 76) collectTimeOrderedData
DEBUG [CompactionExecutor:1] 2011-11-18 17:36:04,044 KeysIndex.java (line 100) 
applying index row 3288500 in ColumnFamily(Indexed1.626972746864617465 
[6b657933:false:0@1234,])
DEBUG [Thread-3] 2011-11-18 17:36:04,044 ColumnFamilyStore.java (line 671) 
flush position is ReplayPosition(segmentId=1321659363183, position=354)
 INFO [Thread-3] 2011-11-18 17:36:04,045 ColumnFamilyStore.java (line 685) 
Enqueuing flush of Memtable-Indexed1.626972746864617465@5734522(38/47 
serialized/live bytes, 2 ops)
 INFO [FlushWriter:1] 2011-11-18 17:36:04,045 Memtable.java (line 239) Writing 
Memtable-Indexed1.626972746864617465@5734522(38/47 serialized/live bytes, 2 ops)
DEBUG [FlushWriter:1] 2011-11-18 17:36:04,047 DatabaseDescriptor.java (line 
783) expected data files size is 84; largest free partition has 19441123328 
bytes free
 INFO [FlushWriter:1] 2011-11-18 17:36:04,062 Memtable.java (line 275) 
Completed flushing 
build/test/cassandra/data/Keyspace1/Indexed1.626972746864617465-hb-2-Data.db 
(154 bytes)
DEBUG [FlushWriter:1] 2011-11-18 17:36:04,063 IntervalNode.java (line 45) 
Creating IntervalNode from [Interval(DecoratedKey(3288498, 0000000000322db2), 
DecoratedKey(3288500, 0000000000322db4))]
DEBUG [FlushWriter:1] 2011-11-18 17:36:04,063 DataTracker.java (line 331) 
adding build/test/cassandra/data/Keyspace1/Indexed1.626972746864617465-hb-2 to 
list of files tracked for Keyspace1.Indexed1.626972746864617465
DEBUG [COMMIT-LOG-WRITER] 2011-11-18 17:36:04,064 CommitLog.java (line 459) 
discard completed log segments for ReplayPosition(segmentId=1321659363183, 
position=354), column family 1047.
DEBUG [COMMIT-LOG-WRITER] 2011-11-18 17:36:04,065 CommitLog.java (line 498) Not 
safe to delete commit log 
CommitLogSegment(/srv/encrypted/project/cassandra/build/test/cassandra/commitlog/CommitLog-1321659363183.log);
 dirty is ; hasNext: false
 INFO [Thread-3] 2011-11-18 17:36:04,065 SecondaryIndexManager.java (line 134) 
Index build of 626972746864617465, complete
 INFO [Thread-3] 2011-11-18 17:36:04,066 StreamInSession.java (line 162) 
Finished streaming session 778312411854932 from /127.0.0.1
DEBUG [MiscStage:1] 2011-11-18 17:36:04,066 StreamReplyVerbHandler.java (line 
47) Received StreamReply StreamReply(sessionId=778312411854932, file='', 
action=SESSION_FINISHED)
{noformat}

and here is a failing one (with 3045 applied):
{noformat}
INFO [Thread-3] 2011-11-18 17:20:02,669 SecondaryIndexManager.java (line 117) 
Submitting index build of 626972746864617465, for data in 
SSTableReader(path='build/test/cassandra/data/Keyspace1/Indexed1-h-4-Data.db')
DEBUG [Streaming:1] 2011-11-18 17:20:02,669 MmappedSegmentedFile.java (line 
139) All segments have been unmapped successfully
DEBUG [NonPeriodicTasks:1] 2011-11-18 17:20:02,671 FileUtils.java (line 51) 
Deleting Indexed1-h-2-Statistics.db
DEBUG [NonPeriodicTasks:1] 2011-11-18 17:20:02,671 FileUtils.java (line 51) 
Deleting Indexed1-h-2-Filter.db
DEBUG [NonPeriodicTasks:1] 2011-11-18 17:20:02,671 FileUtils.java (line 51) 
Deleting Indexed1-h-2-Index.db
DEBUG [NonPeriodicTasks:1] 2011-11-18 17:20:02,672 SSTable.java (line 143) 
Deleted build/test/cassandra/data/Keyspace1/Indexed1-h-2
DEBUG [CompactionExecutor:1] 2011-11-18 17:20:02,674 Table.java (line 516) 
Indexing row key1
DEBUG [CompactionExecutor:1] 2011-11-18 17:20:02,674 CollationController.java 
(line 74) collectTimeOrderedData
DEBUG [CompactionExecutor:1] 2011-11-18 17:20:02,675 KeysIndex.java (line 100) 
applying index row 3288498 in ColumnFamily(Indexed1.626972746864617465 
[6b657931:false:0@1234,])
DEBUG [CompactionExecutor:1] 2011-11-18 17:20:02,676 SlabAllocator.java (line 
105) 1 regions now allocated in org.apache.cassandra.utils.SlabAllocator@1148603
DEBUG [CompactionExecutor:1] 2011-11-18 17:20:02,676 Table.java (line 516) 
Indexing row key3
DEBUG [CompactionExecutor:1] 2011-11-18 17:20:02,676 CollationController.java 
(line 74) collectTimeOrderedData
DEBUG [CompactionExecutor:1] 2011-11-18 17:20:02,677 KeysIndex.java (line 100) 
applying index row 3288500 in ColumnFamily(Indexed1.626972746864617465 
[6b657933:false:0@1234,])
DEBUG [Thread-3] 2011-11-18 17:20:02,677 ColumnFamilyStore.java (line 668) 
flush position is ReplayPosition(segmentId=1321658401840, position=354)
 INFO [Thread-3] 2011-11-18 17:20:02,678 ColumnFamilyStore.java (line 682) 
Enqueuing flush of Memtable-Indexed1.626972746864617465@6972371(38/47 
serialized/live bytes, 2 ops)
 INFO [FlushWriter:1] 2011-11-18 17:20:02,679 Memtable.java (line 237) Writing 
Memtable-Indexed1.626972746864617465@6972371(38/47 serialized/live bytes, 2 ops)
DEBUG [FlushWriter:1] 2011-11-18 17:20:02,679 DatabaseDescriptor.java (line 
791) expected data files size is 84; largest free partition has 19418710016 
bytes free
 INFO [FlushWriter:1] 2011-11-18 17:20:02,690 Memtable.java (line 273) 
Completed flushing 
build/test/cassandra/data/Keyspace1/Indexed1.626972746864617465-h-2-Data.db 
(154 bytes)
DEBUG [FlushWriter:1] 2011-11-18 17:20:02,690 IntervalNode.java (line 45) 
Creating IntervalNode from [Interval(DecoratedKey(3288498, 0000000000322db2), 
DecoratedKey(3288500, 0000000000322db4))]
DEBUG [FlushWriter:1] 2011-11-18 17:20:02,691 IntervalNode.java (line 45) 
Creating IntervalNode from []
DEBUG [COMMIT-LOG-WRITER] 2011-11-18 17:20:02,691 CommitLog.java (line 458) 
discard completed log segments for ReplayPosition(segmentId=1321658401840, 
position=354), column family 1046.
DEBUG [COMMIT-LOG-WRITER] 2011-11-18 17:20:02,691 CommitLog.java (line 497) Not 
safe to delete commit log 
CommitLogSegment(/srv/encrypted/project/cassandra/build/test/cassandra/commitlog/CommitLog-1321658401840.log);
 dirty is ; hasNext: false
 INFO [Thread-3] 2011-11-18 17:20:02,691 SecondaryIndexManager.java (line 136) 
Index build of 626972746864617465, complete
 INFO [Thread-3] 2011-11-18 17:20:02,692 StreamInSession.java (line 179) 
Finished streaming session 777351072779992 from /127.0.0.1
DEBUG [Streaming:1] 2011-11-18 17:20:02,692 StreamReplyVerbHandler.java (line 
47) Received StreamReply StreamReply(sessionId=777351072779992, file='', 
action=SESSION_FINISHED)
{noformat}

It looks like something is up with the "Creating IntervalNode from []" line, 
but I don't see how streaming could break that, especially since the other 
debug lines indicate the correct data was sent.
                
> Update ColumnFamilyOutputFormat to use new bulkload API
> -------------------------------------------------------
>
>                 Key: CASSANDRA-3045
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-3045
>             Project: Cassandra
>          Issue Type: Improvement
>          Components: Hadoop
>            Reporter: Jonathan Ellis
>            Assignee: Brandon Williams
>            Priority: Minor
>             Fix For: 1.1
>
>         Attachments: 0001-Remove-gossip-SS-requirement-from-BulkLoader.txt, 
> 0002-Allow-DD-loading-without-yaml.txt, 
> 0003-hadoop-output-support-for-bulk-loading.txt
>
>
> The bulk loading interface added in CASSANDRA-1278 is a great fit for Hadoop 
> jobs.

--
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

        

Reply via email to