Author: slebresne
Date: Fri Nov 4 07:52:58 2011
New Revision: 1197417
URL: http://svn.apache.org/viewvc?rev=1197417&view=rev
Log:
Fix reference acquisition bug in sstableloader
patch by slebresne; reviewed by jbellis for CASSANDRA-3438
Modified:
cassandra/branches/cassandra-1.0/CHANGES.txt
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/tools/BulkLoader.java
Modified: cassandra/branches/cassandra-1.0/CHANGES.txt
URL:
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/CHANGES.txt?rev=1197417&r1=1197416&r2=1197417&view=diff
==============================================================================
--- cassandra/branches/cassandra-1.0/CHANGES.txt (original)
+++ cassandra/branches/cassandra-1.0/CHANGES.txt Fri Nov 4 07:52:58 2011
@@ -9,6 +9,7 @@
* fix bug that caused first column in per row indexes to be ignored
(CASSANDRA-3441)
* add JMX call to clean (failed) repair sessions (CASSANDRA-3316)
+ * Fix sstableloader reference acquisition bug (CASSANDRA-3438)
Merged from 0.8:
* acquire compactionlock during truncate (CASSANDRA-3399)
* fix displaying cfdef entries for super columnfamilies (CASSANDRA-3415)
Modified:
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
URL:
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java?rev=1197417&r1=1197416&r2=1197417&view=diff
==============================================================================
---
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
(original)
+++
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/SSTableLoader.java
Fri Nov 4 07:52:58 2011
@@ -125,6 +125,8 @@ public class SSTableLoader
}
Collection<Range> ranges = entry.getValue();
StreamOutSession session = StreamOutSession.create(keyspace,
remote, new CountDownCallback(future.latch, remote));
+ // transferSSTables assumes references have been acquired
+ SSTableReader.acquireReferences(sstables);
StreamOut.transferSSTables(session, sstables, ranges,
OperationType.BULK_LOAD);
future.setPendings(remote, session.getFiles());
}
Modified:
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/tools/BulkLoader.java
URL:
http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/tools/BulkLoader.java?rev=1197417&r1=1197416&r2=1197417&view=diff
==============================================================================
---
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/tools/BulkLoader.java
(original)
+++
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/tools/BulkLoader.java
Fri Nov 4 07:52:58 2011
@@ -70,12 +70,14 @@ public class BulkLoader
ProgressIndicator indicator = new
ProgressIndicator(future.getPendingFiles());
indicator.start();
System.out.println("");
+ boolean printEnd = false;
while (!future.isDone())
{
if (indicator.printProgress())
{
// We're done with streaming
System.out.println("\nWaiting for targets to rebuild
indexes ...");
+ printEnd = true;
future.get();
assert future.isDone();
}
@@ -84,6 +86,8 @@ public class BulkLoader
try { Thread.sleep(1000L); } catch (Exception e) {}
}
}
+ if (!printEnd)
+ indicator.printProgress();
}
System.exit(0); // We need that to stop non daemonized threads