Repository: cassandra Updated Branches: refs/heads/cassandra-2.1 78b734e2e -> ddda54145
Fix minor potential leak in sstable2json patch by blerer; reviewed by slebresne for CASSANDRA-7709 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/f3f69cb4 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/f3f69cb4 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/f3f69cb4 Branch: refs/heads/cassandra-2.1 Commit: f3f69cb47319bbc7d3c1e1aa1d48076e4a155e8c Parents: b6b8a28 Author: Sylvain Lebresne <sylv...@datastax.com> Authored: Thu Aug 7 16:11:38 2014 +0200 Committer: Sylvain Lebresne <sylv...@datastax.com> Committed: Thu Aug 7 16:11:38 2014 +0200 ---------------------------------------------------------------------- CHANGES.txt | 1 + .../apache/cassandra/tools/SSTableExport.java | 139 +++++++++++-------- 2 files changed, 79 insertions(+), 61 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/f3f69cb4/CHANGES.txt ---------------------------------------------------------------------- diff --git a/CHANGES.txt b/CHANGES.txt index fdff490..4392159 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 2.0.10 + * Minor leak in sstable2jon (CASSANDRA-7709) * Add cassandra.auto_bootstrap system property (CASSANDRA-7650) * Remove CqlPagingRecordReader/CqlPagingInputFormat (CASSANDRA-7570) * Fix IncompatibleClassChangeError from hadoop2 (CASSANDRA-7229) http://git-wip-us.apache.org/repos/asf/cassandra/blob/f3f69cb4/src/java/org/apache/cassandra/tools/SSTableExport.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/tools/SSTableExport.java b/src/java/org/apache/cassandra/tools/SSTableExport.java index 197585b..f8b85c3 100644 --- a/src/java/org/apache/cassandra/tools/SSTableExport.java +++ b/src/java/org/apache/cassandra/tools/SSTableExport.java @@ -252,20 +252,26 @@ public class SSTableExport throws IOException { KeyIterator iter = new KeyIterator(desc); - DecoratedKey lastKey = null; - while (iter.hasNext()) + try { - DecoratedKey key = iter.next(); + DecoratedKey lastKey = null; + while (iter.hasNext()) + { + DecoratedKey key = iter.next(); - // validate order of the keys in the sstable - if (lastKey != null && lastKey.compareTo(key) > 0) - throw new IOException("Key out of order! " + lastKey + " > " + key); - lastKey = key; + // validate order of the keys in the sstable + if (lastKey != null && lastKey.compareTo(key) > 0) + throw new IOException("Key out of order! " + lastKey + " > " + key); + lastKey = key; - outs.println(bytesToHex(key.key)); - checkStream(outs); // flushes + outs.println(bytesToHex(key.key)); + checkStream(outs); // flushes + } + } + finally + { + iter.close(); } - iter.close(); } /** @@ -281,51 +287,59 @@ public class SSTableExport { SSTableReader sstable = SSTableReader.open(desc); RandomAccessReader dfile = sstable.openDataReader(); + try + { + IPartitioner<?> partitioner = sstable.partitioner; - IPartitioner<?> partitioner = sstable.partitioner; + if (excludes != null) + toExport.removeAll(Arrays.asList(excludes)); - if (excludes != null) - toExport.removeAll(Arrays.asList(excludes)); + outs.println("["); - outs.println("["); + int i = 0; - int i = 0; + // last key to compare order + DecoratedKey lastKey = null; - // last key to compare order - DecoratedKey lastKey = null; + for (String key : toExport) + { + DecoratedKey decoratedKey = partitioner.decorateKey(hexToBytes(key)); - for (String key : toExport) - { - DecoratedKey decoratedKey = partitioner.decorateKey(hexToBytes(key)); + if (lastKey != null && lastKey.compareTo(decoratedKey) > 0) + throw new IOException("Key out of order! " + lastKey + " > " + decoratedKey); - if (lastKey != null && lastKey.compareTo(decoratedKey) > 0) - throw new IOException("Key out of order! " + lastKey + " > " + decoratedKey); + lastKey = decoratedKey; - lastKey = decoratedKey; + RowIndexEntry entry = sstable.getPosition(decoratedKey, SSTableReader.Operator.EQ); + if (entry == null) + continue; - RowIndexEntry entry = sstable.getPosition(decoratedKey, SSTableReader.Operator.EQ); - if (entry == null) - continue; + dfile.seek(entry.position); + ByteBufferUtil.readWithShortLength(dfile); // row key + if (sstable.descriptor.version.hasRowSizeAndColumnCount) + dfile.readLong(); // row size + DeletionInfo deletionInfo = new DeletionInfo(DeletionTime.serializer.deserialize(dfile)); + int columnCount = sstable.descriptor.version.hasRowSizeAndColumnCount ? dfile.readInt() + : Integer.MAX_VALUE; - dfile.seek(entry.position); - ByteBufferUtil.readWithShortLength(dfile); // row key - if (sstable.descriptor.version.hasRowSizeAndColumnCount) - dfile.readLong(); // row size - DeletionInfo deletionInfo = new DeletionInfo(DeletionTime.serializer.deserialize(dfile)); - int columnCount = sstable.descriptor.version.hasRowSizeAndColumnCount ? dfile.readInt() : Integer.MAX_VALUE; + Iterator<OnDiskAtom> atomIterator = sstable.metadata.getOnDiskIterator(dfile, columnCount, + sstable.descriptor.version); - Iterator<OnDiskAtom> atomIterator = sstable.metadata.getOnDiskIterator(dfile, columnCount, sstable.descriptor.version); + checkStream(outs); - checkStream(outs); + if (i != 0) + outs.println(","); + i++; + serializeRow(deletionInfo, atomIterator, sstable.metadata, decoratedKey, outs); + } - if (i != 0) - outs.println(","); - i++; - serializeRow(deletionInfo, atomIterator, sstable.metadata, decoratedKey, outs); + outs.println("\n]"); + outs.flush(); + } + finally + { + dfile.close(); } - - outs.println("\n]"); - outs.flush(); } // This is necessary to accommodate the test suite since you cannot open a Reader more @@ -337,36 +351,39 @@ public class SSTableExport if (excludes != null) excludeSet = new HashSet<String>(Arrays.asList(excludes)); - SSTableIdentityIterator row; SSTableScanner scanner = reader.getScanner(); + try + { + outs.println("["); - outs.println("["); + int i = 0; - int i = 0; + // collecting keys to export + while (scanner.hasNext()) + { + row = (SSTableIdentityIterator) scanner.next(); - // collecting keys to export - while (scanner.hasNext()) - { - row = (SSTableIdentityIterator) scanner.next(); + String currentKey = bytesToHex(row.getKey().key); - String currentKey = bytesToHex(row.getKey().key); + if (excludeSet.contains(currentKey)) + continue; + else if (i != 0) + outs.println(","); - if (excludeSet.contains(currentKey)) - continue; - else if (i != 0) - outs.println(","); + serializeRow(row, row.getKey(), outs); + checkStream(outs); - serializeRow(row, row.getKey(), outs); - checkStream(outs); + i++; + } - i++; + outs.println("\n]"); + outs.flush(); + } + finally + { + scanner.close(); } - - outs.println("\n]"); - outs.flush(); - - scanner.close(); } /**