git commit: Don't ignore IOException during compaction
Updated Branches: refs/heads/cassandra-1.0 add9372b0 - 6eac35cfd Don't ignore IOException during compaction patch by jbellis; reviewed by slebresne for CASSANDRA-3655 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/6eac35cf Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6eac35cf Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6eac35cf Branch: refs/heads/cassandra-1.0 Commit: 6eac35cfd0bfdf67e96f9147b4aad3ddd24c99c2 Parents: add9372 Author: Sylvain Lebresne sylv...@datastax.com Authored: Tue Jan 3 16:24:41 2012 +0100 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Tue Jan 3 16:24:41 2012 +0100 -- .../db/compaction/CompactionIterable.java |2 +- .../cassandra/db/compaction/PrecompactedRow.java | 10 -- .../cassandra/streaming/IncomingStreamReader.java |4 +--- 3 files changed, 10 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/6eac35cf/src/java/org/apache/cassandra/db/compaction/CompactionIterable.java -- diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionIterable.java b/src/java/org/apache/cassandra/db/compaction/CompactionIterable.java index bb8d2bd..5e0dfa7 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionIterable.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionIterable.java @@ -95,7 +95,7 @@ public class CompactionIterable extends AbstractCompactionIterable protected AbstractCompactedRow getReduced() { -assert rows.size() 0; +assert !rows.isEmpty(); try { http://git-wip-us.apache.org/repos/asf/cassandra/blob/6eac35cf/src/java/org/apache/cassandra/db/compaction/PrecompactedRow.java -- diff --git a/src/java/org/apache/cassandra/db/compaction/PrecompactedRow.java b/src/java/org/apache/cassandra/db/compaction/PrecompactedRow.java index 6363d9d..5b5ba8d 100644 --- a/src/java/org/apache/cassandra/db/compaction/PrecompactedRow.java +++ b/src/java/org/apache/cassandra/db/compaction/PrecompactedRow.java @@ -22,6 +22,7 @@ package org.apache.cassandra.db.compaction; import java.io.DataOutput; +import java.io.IOError; import java.io.IOException; import java.security.MessageDigest; import java.util.List; @@ -56,6 +57,10 @@ public class PrecompactedRow extends AbstractCompactedRow public static ColumnFamily removeDeletedAndOldShards(DecoratedKey? key, CompactionController controller, ColumnFamily cf) { +assert key != null; +assert controller != null; +assert cf != null; + // avoid calling shouldPurge unless we actually need to: it can be very expensive if LCS // gets behind and has hundreds of overlapping L0 sstables. Essentially, this method is an // ugly refactor of removeDeletedAndOldShards(controller.shouldPurge(key), controller, cf), @@ -95,6 +100,7 @@ public class PrecompactedRow extends AbstractCompactedRow private static ColumnFamily merge(ListSSTableIdentityIterator rows) { +assert !rows.isEmpty(); ColumnFamily cf = null; for (SSTableIdentityIterator row : rows) { @@ -105,9 +111,9 @@ public class PrecompactedRow extends AbstractCompactedRow } catch (IOException e) { -logger.error(Skipping row + row.getKey() + in + row.getPath(), e); -continue; +throw new IOError(e); } + if (cf == null) { cf = thisCF; http://git-wip-us.apache.org/repos/asf/cassandra/blob/6eac35cf/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java -- diff --git a/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java b/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java index f70f5ff..e2a618f 100644 --- a/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java +++ b/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java @@ -108,7 +108,7 @@ public class IncomingStreamReader ColumnFamilyStore cfs = Table.open(localFile.desc.ksname).getColumnFamilyStore(localFile.desc.cfname); DecoratedKey key; SSTableWriter writer = new SSTableWriter(localFile.getFilename(), remoteFile.estimatedKeys); -CompactionController controller = null; +CompactionController controller = new CompactionController(cfs, Collections.SSTableReaderemptyList(), Integer.MIN_VALUE, true); try { @@ -128,8 +128,6 @@ public class
[6/11] git commit: Don't ignore IOException during compaction
Don't ignore IOException during compaction patch by jbellis; reviewed by slebresne for CASSANDRA-3655 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/6eac35cf Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6eac35cf Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6eac35cf Branch: refs/heads/trunk Commit: 6eac35cfd0bfdf67e96f9147b4aad3ddd24c99c2 Parents: add9372 Author: Sylvain Lebresne sylv...@datastax.com Authored: Tue Jan 3 16:24:41 2012 +0100 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Tue Jan 3 16:24:41 2012 +0100 -- .../db/compaction/CompactionIterable.java |2 +- .../cassandra/db/compaction/PrecompactedRow.java | 10 -- .../cassandra/streaming/IncomingStreamReader.java |4 +--- 3 files changed, 10 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/6eac35cf/src/java/org/apache/cassandra/db/compaction/CompactionIterable.java -- diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionIterable.java b/src/java/org/apache/cassandra/db/compaction/CompactionIterable.java index bb8d2bd..5e0dfa7 100644 --- a/src/java/org/apache/cassandra/db/compaction/CompactionIterable.java +++ b/src/java/org/apache/cassandra/db/compaction/CompactionIterable.java @@ -95,7 +95,7 @@ public class CompactionIterable extends AbstractCompactionIterable protected AbstractCompactedRow getReduced() { -assert rows.size() 0; +assert !rows.isEmpty(); try { http://git-wip-us.apache.org/repos/asf/cassandra/blob/6eac35cf/src/java/org/apache/cassandra/db/compaction/PrecompactedRow.java -- diff --git a/src/java/org/apache/cassandra/db/compaction/PrecompactedRow.java b/src/java/org/apache/cassandra/db/compaction/PrecompactedRow.java index 6363d9d..5b5ba8d 100644 --- a/src/java/org/apache/cassandra/db/compaction/PrecompactedRow.java +++ b/src/java/org/apache/cassandra/db/compaction/PrecompactedRow.java @@ -22,6 +22,7 @@ package org.apache.cassandra.db.compaction; import java.io.DataOutput; +import java.io.IOError; import java.io.IOException; import java.security.MessageDigest; import java.util.List; @@ -56,6 +57,10 @@ public class PrecompactedRow extends AbstractCompactedRow public static ColumnFamily removeDeletedAndOldShards(DecoratedKey? key, CompactionController controller, ColumnFamily cf) { +assert key != null; +assert controller != null; +assert cf != null; + // avoid calling shouldPurge unless we actually need to: it can be very expensive if LCS // gets behind and has hundreds of overlapping L0 sstables. Essentially, this method is an // ugly refactor of removeDeletedAndOldShards(controller.shouldPurge(key), controller, cf), @@ -95,6 +100,7 @@ public class PrecompactedRow extends AbstractCompactedRow private static ColumnFamily merge(ListSSTableIdentityIterator rows) { +assert !rows.isEmpty(); ColumnFamily cf = null; for (SSTableIdentityIterator row : rows) { @@ -105,9 +111,9 @@ public class PrecompactedRow extends AbstractCompactedRow } catch (IOException e) { -logger.error(Skipping row + row.getKey() + in + row.getPath(), e); -continue; +throw new IOError(e); } + if (cf == null) { cf = thisCF; http://git-wip-us.apache.org/repos/asf/cassandra/blob/6eac35cf/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java -- diff --git a/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java b/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java index f70f5ff..e2a618f 100644 --- a/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java +++ b/src/java/org/apache/cassandra/streaming/IncomingStreamReader.java @@ -108,7 +108,7 @@ public class IncomingStreamReader ColumnFamilyStore cfs = Table.open(localFile.desc.ksname).getColumnFamilyStore(localFile.desc.cfname); DecoratedKey key; SSTableWriter writer = new SSTableWriter(localFile.getFilename(), remoteFile.estimatedKeys); -CompactionController controller = null; +CompactionController controller = new CompactionController(cfs, Collections.SSTableReaderemptyList(), Integer.MIN_VALUE, true); try { @@ -128,8 +128,6 @@ public class IncomingStreamReader if (cached != null remoteFile.type ==