Return-Path: X-Original-To: apmail-cassandra-commits-archive@www.apache.org Delivered-To: apmail-cassandra-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id C54F6174B9 for ; Thu, 12 Feb 2015 14:02:53 +0000 (UTC) Received: (qmail 38548 invoked by uid 500); 12 Feb 2015 14:02:53 -0000 Delivered-To: apmail-cassandra-commits-archive@cassandra.apache.org Received: (qmail 38380 invoked by uid 500); 12 Feb 2015 14:02:53 -0000 Mailing-List: contact commits-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@cassandra.apache.org Delivered-To: mailing list commits@cassandra.apache.org Received: (qmail 38208 invoked by uid 99); 12 Feb 2015 14:02:53 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 12 Feb 2015 14:02:53 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 2F300E0540; Thu, 12 Feb 2015 14:02:53 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: benedict@apache.org To: commits@cassandra.apache.org Date: Thu, 12 Feb 2015 14:02:55 -0000 Message-Id: <0736dcc4038e45d99f4b5f31e38575c6@git.apache.org> In-Reply-To: <5fed4fc68122463487d98b0929043e73@git.apache.org> References: <5fed4fc68122463487d98b0929043e73@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [3/5] cassandra git commit: Merge branch 'cassandra-2.1' into trunk http://git-wip-us.apache.org/repos/asf/cassandra/blob/36729b94/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java index 89ecd99,0000000..d411faf mode 100644,000000..100644 --- a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java +++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java @@@ -1,583 -1,0 +1,583 @@@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.io.sstable.format.big; + +import java.io.DataInput; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.cassandra.db.*; +import org.apache.cassandra.io.sstable.*; +import org.apache.cassandra.io.sstable.format.SSTableReader; +import org.apache.cassandra.io.sstable.format.SSTableWriter; +import org.apache.cassandra.io.sstable.format.Version; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.config.CFMetaData; +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.compaction.AbstractCompactedRow; +import org.apache.cassandra.dht.IPartitioner; +import org.apache.cassandra.io.FSWriteError; +import org.apache.cassandra.io.compress.CompressedSequentialWriter; +import org.apache.cassandra.io.sstable.metadata.MetadataCollector; +import org.apache.cassandra.io.sstable.metadata.MetadataComponent; +import org.apache.cassandra.io.sstable.metadata.MetadataType; +import org.apache.cassandra.io.sstable.metadata.StatsMetadata; +import org.apache.cassandra.io.util.DataOutputPlus; +import org.apache.cassandra.io.util.DataOutputStreamAndChannel; +import org.apache.cassandra.io.util.FileMark; +import org.apache.cassandra.io.util.FileUtils; +import org.apache.cassandra.io.util.SegmentedFile; +import org.apache.cassandra.io.util.SequentialWriter; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.cassandra.utils.FBUtilities; +import org.apache.cassandra.utils.FilterFactory; +import org.apache.cassandra.utils.IFilter; +import org.apache.cassandra.utils.Pair; +import org.apache.cassandra.utils.StreamingHistogram; + +public class BigTableWriter extends SSTableWriter +{ + private static final Logger logger = LoggerFactory.getLogger(BigTableWriter.class); + + // not very random, but the only value that can't be mistaken for a legal column-name length + public static final int END_OF_ROW = 0x0000; + + private IndexWriter iwriter; + private SegmentedFile.Builder dbuilder; + private final SequentialWriter dataFile; + private DecoratedKey lastWrittenKey; + private FileMark dataMark; + + BigTableWriter(Descriptor descriptor, Long keyCount, Long repairedAt, CFMetaData metadata, IPartitioner partitioner, MetadataCollector metadataCollector) + { + super(descriptor, keyCount, repairedAt, metadata, partitioner, metadataCollector); + + if (compression) + { + dataFile = SequentialWriter.open(getFilename(), + descriptor.filenameFor(Component.COMPRESSION_INFO), + metadata.compressionParameters(), + metadataCollector); + dbuilder = SegmentedFile.getCompressedBuilder((CompressedSequentialWriter) dataFile); + } + else + { + dataFile = SequentialWriter.open(new File(getFilename()), new File(descriptor.filenameFor(Component.CRC))); + dbuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getDiskAccessMode()); + } + iwriter = new IndexWriter(keyCount, dataFile); + } + + public void mark() + { + dataMark = dataFile.mark(); + iwriter.mark(); + } + + public void resetAndTruncate() + { + dataFile.resetAndTruncate(dataMark); + iwriter.resetAndTruncate(); + } + + /** + * Perform sanity checks on @param decoratedKey and @return the position in the data file before any data is written + */ + private long beforeAppend(DecoratedKey decoratedKey) + { + assert decoratedKey != null : "Keys must not be null"; // empty keys ARE allowed b/c of indexed column values + if (lastWrittenKey != null && lastWrittenKey.compareTo(decoratedKey) >= 0) + throw new RuntimeException("Last written key " + lastWrittenKey + " >= current key " + decoratedKey + " writing into " + getFilename()); + return (lastWrittenKey == null) ? 0 : dataFile.getFilePointer(); + } + + private void afterAppend(DecoratedKey decoratedKey, long dataPosition, RowIndexEntry index) + { + metadataCollector.addKey(decoratedKey.getKey()); + lastWrittenKey = decoratedKey; + last = lastWrittenKey; + if (first == null) + first = lastWrittenKey; + + if (logger.isTraceEnabled()) + logger.trace("wrote {} at {}", decoratedKey, dataPosition); + iwriter.append(decoratedKey, index, dataPosition); + dbuilder.addPotentialBoundary(dataPosition); + } + + /** + * @param row + * @return null if the row was compacted away entirely; otherwise, the PK index entry for this row + */ + public RowIndexEntry append(AbstractCompactedRow row) + { + long startPosition = beforeAppend(row.key); + RowIndexEntry entry; + try + { + entry = row.write(startPosition, dataFile); + if (entry == null) + return null; + } + catch (IOException e) + { + throw new FSWriteError(e, dataFile.getPath()); + } + long endPosition = dataFile.getFilePointer(); + metadataCollector.update(endPosition - startPosition, row.columnStats()); + afterAppend(row.key, endPosition, entry); + return entry; + } + + public void append(DecoratedKey decoratedKey, ColumnFamily cf) + { + if (decoratedKey.getKey().remaining() > FBUtilities.MAX_UNSIGNED_SHORT) + { + logger.error("Key size {} exceeds maximum of {}, skipping row", + decoratedKey.getKey().remaining(), + FBUtilities.MAX_UNSIGNED_SHORT); + return; + } + + long startPosition = beforeAppend(decoratedKey); + try + { + RowIndexEntry entry = rawAppend(cf, startPosition, decoratedKey, dataFile.stream); + afterAppend(decoratedKey, startPosition, entry); + } + catch (IOException e) + { + throw new FSWriteError(e, dataFile.getPath()); + } + metadataCollector.update(dataFile.getFilePointer() - startPosition, cf.getColumnStats()); + } + + private static RowIndexEntry rawAppend(ColumnFamily cf, long startPosition, DecoratedKey key, DataOutputPlus out) throws IOException + { + assert cf.hasColumns() || cf.isMarkedForDelete(); + + ColumnIndex.Builder builder = new ColumnIndex.Builder(cf, key.getKey(), out); + ColumnIndex index = builder.build(cf); + + out.writeShort(END_OF_ROW); + return RowIndexEntry.create(startPosition, cf.deletionInfo().getTopLevelDeletion(), index); + } + + /** + * @throws IOException if a read from the DataInput fails + * @throws FSWriteError if a write to the dataFile fails + */ + public long appendFromStream(DecoratedKey key, CFMetaData metadata, DataInput in, Version version) throws IOException + { + long currentPosition = beforeAppend(key); + + ColumnStats.MaxLongTracker maxTimestampTracker = new ColumnStats.MaxLongTracker(Long.MAX_VALUE); + ColumnStats.MinLongTracker minTimestampTracker = new ColumnStats.MinLongTracker(Long.MIN_VALUE); + ColumnStats.MaxIntTracker maxDeletionTimeTracker = new ColumnStats.MaxIntTracker(Integer.MAX_VALUE); + List minColumnNames = Collections.emptyList(); + List maxColumnNames = Collections.emptyList(); + StreamingHistogram tombstones = new StreamingHistogram(TOMBSTONE_HISTOGRAM_BIN_SIZE); + boolean hasLegacyCounterShards = false; + + ColumnFamily cf = ArrayBackedSortedColumns.factory.create(metadata); + cf.delete(DeletionTime.serializer.deserialize(in)); + + ColumnIndex.Builder columnIndexer = new ColumnIndex.Builder(cf, key.getKey(), dataFile.stream); + + if (cf.deletionInfo().getTopLevelDeletion().localDeletionTime < Integer.MAX_VALUE) + { + tombstones.update(cf.deletionInfo().getTopLevelDeletion().localDeletionTime); + maxDeletionTimeTracker.update(cf.deletionInfo().getTopLevelDeletion().localDeletionTime); + minTimestampTracker.update(cf.deletionInfo().getTopLevelDeletion().markedForDeleteAt); + maxTimestampTracker.update(cf.deletionInfo().getTopLevelDeletion().markedForDeleteAt); + } + + Iterator rangeTombstoneIterator = cf.deletionInfo().rangeIterator(); + while (rangeTombstoneIterator.hasNext()) + { + RangeTombstone rangeTombstone = rangeTombstoneIterator.next(); + tombstones.update(rangeTombstone.getLocalDeletionTime()); + minTimestampTracker.update(rangeTombstone.timestamp()); + maxTimestampTracker.update(rangeTombstone.timestamp()); + maxDeletionTimeTracker.update(rangeTombstone.getLocalDeletionTime()); + minColumnNames = ColumnNameHelper.minComponents(minColumnNames, rangeTombstone.min, metadata.comparator); + maxColumnNames = ColumnNameHelper.maxComponents(maxColumnNames, rangeTombstone.max, metadata.comparator); + } + + Iterator iter = AbstractCell.onDiskIterator(in, ColumnSerializer.Flag.PRESERVE_SIZE, Integer.MIN_VALUE, version, metadata.comparator); + try + { + while (iter.hasNext()) + { + OnDiskAtom atom = iter.next(); + if (atom == null) + break; + + if (atom instanceof CounterCell) + { + atom = ((CounterCell) atom).markLocalToBeCleared(); + hasLegacyCounterShards = hasLegacyCounterShards || ((CounterCell) atom).hasLegacyShards(); + } + + int deletionTime = atom.getLocalDeletionTime(); + if (deletionTime < Integer.MAX_VALUE) + tombstones.update(deletionTime); + minTimestampTracker.update(atom.timestamp()); + maxTimestampTracker.update(atom.timestamp()); + minColumnNames = ColumnNameHelper.minComponents(minColumnNames, atom.name(), metadata.comparator); + maxColumnNames = ColumnNameHelper.maxComponents(maxColumnNames, atom.name(), metadata.comparator); + maxDeletionTimeTracker.update(atom.getLocalDeletionTime()); + + columnIndexer.add(atom); // This write the atom on disk too + } + + columnIndexer.maybeWriteEmptyRowHeader(); + dataFile.stream.writeShort(END_OF_ROW); + } + catch (IOException e) + { + throw new FSWriteError(e, dataFile.getPath()); + } + + metadataCollector.updateMinTimestamp(minTimestampTracker.get()) + .updateMaxTimestamp(maxTimestampTracker.get()) + .updateMaxLocalDeletionTime(maxDeletionTimeTracker.get()) + .addRowSize(dataFile.getFilePointer() - currentPosition) + .addColumnCount(columnIndexer.writtenAtomCount()) + .mergeTombstoneHistogram(tombstones) + .updateMinColumnNames(minColumnNames) + .updateMaxColumnNames(maxColumnNames) + .updateHasLegacyCounterShards(hasLegacyCounterShards); + + afterAppend(key, currentPosition, RowIndexEntry.create(currentPosition, cf.deletionInfo().getTopLevelDeletion(), columnIndexer.build())); + return currentPosition; + } + + /** + * After failure, attempt to close the index writer and data file before deleting all temp components for the sstable + */ + public void abort() + { + assert descriptor.type.isTemporary; + if (iwriter == null && dataFile == null) + return; + + if (iwriter != null) + iwriter.abort(); + + if (dataFile!= null) + dataFile.abort(); + + Set components = SSTable.componentsFor(descriptor); + try + { + if (!components.isEmpty()) + SSTable.delete(descriptor, components); + } + catch (FSWriteError e) + { + logger.error(String.format("Failed deleting temp components for %s", descriptor), e); + throw e; + } + } + + // we use this method to ensure any managed data we may have retained references to during the write are no + // longer referenced, so that we do not need to enclose the expensive call to closeAndOpenReader() in a transaction + public void isolateReferences() + { + // currently we only maintain references to first/last/lastWrittenKey from the data provided; all other + // data retention is done through copying + first = getMinimalKey(first); + last = lastWrittenKey = getMinimalKey(last); + } + + private Descriptor makeTmpLinks() + { + // create temp links if they don't already exist + Descriptor link = descriptor.asType(Descriptor.Type.TEMPLINK); + if (!new File(link.filenameFor(Component.PRIMARY_INDEX)).exists()) + { + FileUtils.createHardLink(new File(descriptor.filenameFor(Component.PRIMARY_INDEX)), new File(link.filenameFor(Component.PRIMARY_INDEX))); + FileUtils.createHardLink(new File(descriptor.filenameFor(Component.DATA)), new File(link.filenameFor(Component.DATA))); + } + return link; + } + + public SSTableReader openEarly(long maxDataAge) + { + StatsMetadata sstableMetadata = (StatsMetadata) metadataCollector.finalizeMetadata(partitioner.getClass().getCanonicalName(), + metadata.getBloomFilterFpChance(), + repairedAt).get(MetadataType.STATS); + + // find the max (exclusive) readable key + IndexSummaryBuilder.ReadableBoundary boundary = iwriter.getMaxReadable(); + if (boundary == null) + return null; + + assert boundary.indexLength > 0 && boundary.dataLength > 0; + Descriptor link = makeTmpLinks(); + // open the reader early, giving it a FINAL descriptor type so that it is indistinguishable for other consumers - SegmentedFile ifile = iwriter.builder.complete(link.filenameFor(Component.PRIMARY_INDEX), FinishType.EARLY); - SegmentedFile dfile = dbuilder.complete(link.filenameFor(Component.DATA), FinishType.EARLY); ++ SegmentedFile ifile = iwriter.builder.complete(link.filenameFor(Component.PRIMARY_INDEX), boundary.indexLength); ++ SegmentedFile dfile = dbuilder.complete(link.filenameFor(Component.DATA), boundary.dataLength); + SSTableReader sstable = SSTableReader.internalOpen(descriptor.asType(Descriptor.Type.FINAL), + components, metadata, + partitioner, ifile, + dfile, iwriter.summary.build(partitioner, boundary.lastKey), + iwriter.bf.sharedCopy(), maxDataAge, sstableMetadata, SSTableReader.OpenReason.EARLY); + + // now it's open, find the ACTUAL last readable key (i.e. for which the data file has also been flushed) + sstable.first = getMinimalKey(first); + sstable.last = getMinimalKey(boundary.lastKey); + return sstable; + } + + public SSTableReader closeAndOpenReader() + { + return closeAndOpenReader(System.currentTimeMillis()); + } + + public SSTableReader closeAndOpenReader(long maxDataAge) + { + return finish(FinishType.NORMAL, maxDataAge, this.repairedAt); + } + + public SSTableReader finish(FinishType finishType, long maxDataAge, long repairedAt) + { + assert finishType != FinishType.CLOSE; + Pair p; + + p = close(finishType, repairedAt < 0 ? this.repairedAt : repairedAt); + Descriptor desc = p.left; + StatsMetadata metadata = p.right; + + if (finishType == FinishType.EARLY) + desc = makeTmpLinks(); + + // finalize in-memory state for the reader - SegmentedFile ifile = iwriter.builder.complete(desc.filenameFor(Component.PRIMARY_INDEX), finishType); - SegmentedFile dfile = dbuilder.complete(desc.filenameFor(Component.DATA), finishType); ++ SegmentedFile ifile = iwriter.builder.complete(desc.filenameFor(Component.PRIMARY_INDEX), finishType.isFinal); ++ SegmentedFile dfile = dbuilder.complete(desc.filenameFor(Component.DATA), finishType.isFinal); + SSTableReader sstable = SSTableReader.internalOpen(desc.asType(Descriptor.Type.FINAL), + components, + this.metadata, + partitioner, + ifile, + dfile, + iwriter.summary.build(partitioner), + iwriter.bf.sharedCopy(), + maxDataAge, + metadata, + finishType.openReason); + sstable.first = getMinimalKey(first); + sstable.last = getMinimalKey(last); + + if (finishType.isFinal) + { + iwriter.bf.close(); + // try to save the summaries to disk + sstable.saveSummary(iwriter.builder, dbuilder); + iwriter = null; + dbuilder = null; + } + return sstable; + } + + // Close the writer and return the descriptor to the new sstable and it's metadata + public Pair close() + { + return close(FinishType.CLOSE, this.repairedAt); + } + + private Pair close(FinishType type, long repairedAt) + { + switch (type) + { + case EARLY: case CLOSE: case NORMAL: + iwriter.close(); + dataFile.close(); + if (type == FinishType.CLOSE) + iwriter.bf.close(); + } + + // write sstable statistics + Map metadataComponents; + metadataComponents = metadataCollector + .finalizeMetadata(partitioner.getClass().getCanonicalName(), + metadata.getBloomFilterFpChance(),repairedAt); + + // remove the 'tmp' marker from all components + Descriptor descriptor = this.descriptor; + if (type.isFinal) + { + dataFile.writeFullChecksum(descriptor); + writeMetadata(descriptor, metadataComponents); + // save the table of components + SSTable.appendTOC(descriptor, components); + descriptor = rename(descriptor, components); + } + + return Pair.create(descriptor, (StatsMetadata) metadataComponents.get(MetadataType.STATS)); + } + + private static void writeMetadata(Descriptor desc, Map components) + { + SequentialWriter out = SequentialWriter.open(new File(desc.filenameFor(Component.STATS))); + try + { + desc.getMetadataSerializer().serialize(components, out.stream); + } + catch (IOException e) + { + throw new FSWriteError(e, out.getPath()); + } + finally + { + out.close(); + } + } + + public long getFilePointer() + { + return dataFile.getFilePointer(); + } + + public long getOnDiskFilePointer() + { + return dataFile.getOnDiskFilePointer(); + } + + /** + * Encapsulates writing the index and filter for an SSTable. The state of this object is not valid until it has been closed. + */ + class IndexWriter + { + private final SequentialWriter indexFile; + public final SegmentedFile.Builder builder; + public final IndexSummaryBuilder summary; + public final IFilter bf; + private FileMark mark; + + IndexWriter(long keyCount, final SequentialWriter dataFile) + { + indexFile = SequentialWriter.open(new File(descriptor.filenameFor(Component.PRIMARY_INDEX))); + builder = SegmentedFile.getBuilder(DatabaseDescriptor.getIndexAccessMode()); + summary = new IndexSummaryBuilder(keyCount, metadata.getMinIndexInterval(), Downsampling.BASE_SAMPLING_LEVEL); + bf = FilterFactory.getFilter(keyCount, metadata.getBloomFilterFpChance(), true); + // register listeners to be alerted when the data files are flushed + indexFile.setPostFlushListener(new Runnable() + { + public void run() + { + summary.markIndexSynced(indexFile.getLastFlushOffset()); + } + }); + dataFile.setPostFlushListener(new Runnable() + { + public void run() + { + summary.markDataSynced(dataFile.getLastFlushOffset()); + } + }); + } + + // finds the last (-offset) decorated key that can be guaranteed to occur fully in the flushed portion of the index file + IndexSummaryBuilder.ReadableBoundary getMaxReadable() + { + return summary.getLastReadableBoundary(); + } + + public void append(DecoratedKey key, RowIndexEntry indexEntry, long dataEnd) + { + bf.add(key.getKey()); + long indexStart = indexFile.getFilePointer(); + try + { + ByteBufferUtil.writeWithShortLength(key.getKey(), indexFile.stream); + rowIndexEntrySerializer.serialize(indexEntry, indexFile.stream); + } + catch (IOException e) + { + throw new FSWriteError(e, indexFile.getPath()); + } + long indexEnd = indexFile.getFilePointer(); + + if (logger.isTraceEnabled()) + logger.trace("wrote index entry: {} at {}", indexEntry, indexStart); + + summary.maybeAddEntry(key, indexStart, indexEnd, dataEnd); + builder.addPotentialBoundary(indexStart); + } + + public void abort() + { + indexFile.abort(); + bf.close(); + } + + /** + * Closes the index and bloomfilter, making the public state of this writer valid for consumption. + */ + public void close() + { + if (components.contains(Component.FILTER)) + { + String path = descriptor.filenameFor(Component.FILTER); + try + { + // bloom filter + FileOutputStream fos = new FileOutputStream(path); + DataOutputStreamAndChannel stream = new DataOutputStreamAndChannel(fos); + FilterFactory.serialize(bf, stream); + stream.flush(); + fos.getFD().sync(); + stream.close(); + } + catch (IOException e) + { + throw new FSWriteError(e, path); + } + } + + // index + long position = indexFile.getFilePointer(); + indexFile.close(); // calls force + FileUtils.truncate(indexFile.getPath(), position); + } + + public void mark() + { + mark = indexFile.mark(); + } + + public void resetAndTruncate() + { + // we can't un-set the bloom filter addition, but extra keys in there are harmless. + // we can't reset dbuilder either, but that is the last thing called in afterappend so + // we assume that if that worked then we won't be trying to reset. + indexFile.resetAndTruncate(mark); + } + } +} http://git-wip-us.apache.org/repos/asf/cassandra/blob/36729b94/src/java/org/apache/cassandra/io/util/BufferedPoolingSegmentedFile.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/36729b94/src/java/org/apache/cassandra/io/util/BufferedSegmentedFile.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/36729b94/src/java/org/apache/cassandra/io/util/CompressedSegmentedFile.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/36729b94/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/36729b94/src/java/org/apache/cassandra/io/util/RandomAccessReader.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/io/util/RandomAccessReader.java index 6bff378,df68ca3..1f2d395 --- a/src/java/org/apache/cassandra/io/util/RandomAccessReader.java +++ b/src/java/org/apache/cassandra/io/util/RandomAccessReader.java @@@ -53,52 -55,44 +53,56 @@@ public class RandomAccessReader extend protected RandomAccessReader(File file, int bufferSize, PoolingSegmentedFile owner) throws FileNotFoundException { - this(file, bufferSize, false, owner); - this(file, bufferSize, -1, owner); ++ this(file, bufferSize, -1, false, owner); } - - protected RandomAccessReader(File file, int bufferSize, boolean useDirectBuffer, PoolingSegmentedFile owner) throws FileNotFoundException - protected RandomAccessReader(File file, int bufferSize, long overrideLength, PoolingSegmentedFile owner) throws FileNotFoundException ++ protected RandomAccessReader(File file, int bufferSize, long overrideLength, boolean useDirectBuffer, PoolingSegmentedFile owner) throws FileNotFoundException { - super(file, "r"); - this.owner = owner; - channel = super.getChannel(); filePath = file.getAbsolutePath(); + try + { + channel = FileChannel.open(file.toPath(), StandardOpenOption.READ); + } + catch (IOException e) + { + throw new FileNotFoundException(filePath); + } + // allocating required size of the buffer if (bufferSize <= 0) throw new IllegalArgumentException("bufferSize must be positive"); - buffer = new byte[bufferSize]; - // we can cache file length in read-only mode - try - { - fileLength = channel.size(); - } - catch (IOException e) + long fileLength = overrideLength; + if (fileLength <= 0) { - throw new FSReadError(e, filePath); + try + { + fileLength = channel.size(); + } + catch (IOException e) + { + throw new FSReadError(e, filePath); + } } - + this.fileLength = fileLength; - validBufferBytes = -1; // that will trigger reBuffer() on demand by read/seek operations + buffer = allocateBuffer(bufferSize, useDirectBuffer); + buffer.limit(0); + } + + protected ByteBuffer allocateBuffer(int bufferSize, boolean useDirectBuffer) + { + int size = (int) Math.min(fileLength, bufferSize); + return useDirectBuffer + ? ByteBuffer.allocate(size) + : ByteBuffer.allocateDirect(size); } - public static RandomAccessReader open(File file, PoolingSegmentedFile owner) + public static RandomAccessReader open(File file, long overrideSize, PoolingSegmentedFile owner) { - return open(file, DEFAULT_BUFFER_SIZE, owner); + return open(file, DEFAULT_BUFFER_SIZE, overrideSize, owner); } public static RandomAccessReader open(File file) @@@ -109,11 -108,16 +118,16 @@@ @VisibleForTesting static RandomAccessReader open(File file, int bufferSize, PoolingSegmentedFile owner) { + return open(file, bufferSize, -1L, owner); + } + + private static RandomAccessReader open(File file, int bufferSize, long overrideSize, PoolingSegmentedFile owner) + { try { - return new RandomAccessReader(file, bufferSize, owner); - return new RandomAccessReader(file, bufferSize, overrideSize, owner); ++ return new RandomAccessReader(file, bufferSize, overrideSize, false, owner); } - catch (FileNotFoundException e) + catch (IOException e) { throw new RuntimeException(e); } @@@ -142,14 -138,27 +156,18 @@@ try { - int read = buffer.length; - if (bufferOffset + read > fileLength) - { - if (bufferOffset >= fileLength) - return; - read = (int) (fileLength - bufferOffset); - } - channel.position(bufferOffset); // setting channel position - while (buffer.hasRemaining()) - - int offset = 0; - while (read > 0) ++ long limit = bufferOffset; ++ while (buffer.hasRemaining() && limit < fileLength) { - int n = super.read(buffer, offset, read); + int n = channel.read(buffer); if (n < 0) - throw new IllegalStateException(); - read -= n; - offset += n; + break; ++ limit = bufferOffset + buffer.position(); } - - validBufferBytes = offset; ++ if (limit > fileLength) ++ buffer.position((int)(fileLength - bufferOffset)); + buffer.flip(); } catch (IOException e) { http://git-wip-us.apache.org/repos/asf/cassandra/blob/36729b94/src/java/org/apache/cassandra/io/util/SegmentedFile.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/io/util/SegmentedFile.java index 8f760da,146494d..1341fa9 --- a/src/java/org/apache/cassandra/io/util/SegmentedFile.java +++ b/src/java/org/apache/cassandra/io/util/SegmentedFile.java @@@ -25,6 -25,6 +25,7 @@@ import java.nio.MappedByteBuffer import java.util.Iterator; import java.util.NoSuchElementException; ++import com.google.common.base.Throwables; import com.google.common.util.concurrent.RateLimiter; import org.apache.cassandra.config.Config; http://git-wip-us.apache.org/repos/asf/cassandra/blob/36729b94/src/java/org/apache/cassandra/io/util/ThrottledReader.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/io/util/ThrottledReader.java index b9b645a,c4845c5..570d777 --- a/src/java/org/apache/cassandra/io/util/ThrottledReader.java +++ b/src/java/org/apache/cassandra/io/util/ThrottledReader.java @@@ -30,9 -30,9 +30,9 @@@ public class ThrottledReader extends Ra { private final RateLimiter limiter; - protected ThrottledReader(File file, RateLimiter limiter) throws FileNotFoundException + protected ThrottledReader(File file, long overrideLength, RateLimiter limiter) throws FileNotFoundException { - super(file, RandomAccessReader.DEFAULT_BUFFER_SIZE, null); - super(file, RandomAccessReader.DEFAULT_BUFFER_SIZE, overrideLength, null); ++ super(file, RandomAccessReader.DEFAULT_BUFFER_SIZE, overrideLength, false, null); this.limiter = limiter; }