package org.apache.cassandra.io.compress;
import java.io.DataOutputStream;
import java.io.EOFException;
import java.io.File;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.Channels;
import java.util.Optional;
import java.util.zip.CRC32;
import org.apache.cassandra.io.FSReadError;
import org.apache.cassandra.io.FSWriteError;
import org.apache.cassandra.io.sstable.CorruptSSTableException;
import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
import org.apache.cassandra.io.util.*;
import org.apache.cassandra.schema.CompressionParams;
import static org.apache.cassandra.utils.Throwables.merge;
public class CompressedSequentialWriter extends SequentialWriter
{
private final ChecksumWriter crcMetadata;
private long chunkOffset = 0;
private final CompressionMetadata.Writer metadataWriter;
private final ICompressor compressor;
private ByteBuffer compressed;
private int chunkCount = 0;
private long uncompressedSize = 0, compressedSize = 0;
private final MetadataCollector sstableMetadataCollector;
private final ByteBuffer crcCheckBuffer = ByteBuffer.allocate(4);
private final Optional<File> digestFile;
public CompressedSequentialWriter(File file,
String offsetsPath,
File digestFile,
SequentialWriterOption option,
CompressionParams parameters,
MetadataCollector sstableMetadataCollector)
{
super(file, SequentialWriterOption.newBuilder()
.bufferSize(option.bufferSize())
.bufferType(option.bufferType())
.bufferSize(parameters.chunkLength())
.bufferType(parameters.getSstableCompressor().preferredBufferType())
.finishOnClose(option.finishOnClose())
.build());
this.compressor = parameters.getSstableCompressor();
this.digestFile = Optional.ofNullable(digestFile);
compressed = compressor.preferredBufferType().allocate(compressor.initialCompressedBufferLength(buffer.capacity()));
metadataWriter = CompressionMetadata.Writer.open(parameters, offsetsPath);
this.sstableMetadataCollector = sstableMetadataCollector;
crcMetadata = new ChecksumWriter(new DataOutputStream(Channels.newOutputStream(channel)));
}
@Override
public long getOnDiskFilePointer()
{
try
{
return fchannel.position();
}
catch (IOException e)
{
throw new FSReadError(e, getPath());
}
}
@Override
public long getEstimatedOnDiskBytesWritten()
{
return chunkOffset;
}
@Override
public void flush()
{
throw new UnsupportedOperationException();
}
@Override
protected void flushData()
{
seekToChunkStart();
try
{
buffer.flip();
compressed.clear();
compressor.compress(buffer, compressed);
}
catch (IOException e)
{
throw new RuntimeException("Compression exception", e);
}
int compressedLength = compressed.position();
uncompressedSize += buffer.position();
compressedSize += compressedLength;
try
{
metadataWriter.addOffset(chunkOffset);
chunkCount++;
compressed.flip();
channel.write(compressed);
compressed.rewind();
crcMetadata.appendDirect(compressed, true);
lastFlushOffset = uncompressedSize;
}
catch (IOException e)
{
throw new FSWriteError(e, getPath());
}
chunkOffset += compressedLength + 4;
if (runPostFlush != null)
runPostFlush.run();
}
public CompressionMetadata open(long overrideLength)
{
if (overrideLength <= 0)
overrideLength = uncompressedSize;
return metadataWriter.open(overrideLength, chunkOffset);
}
@Override
public DataPosition mark()
{
if (!buffer.hasRemaining())
doFlush(0);
return new CompressedFileWriterMark(chunkOffset, current(), buffer.position(), chunkCount + 1);
}
@Override
public synchronized void resetAndTruncate(DataPosition mark)
{
assert mark instanceof CompressedFileWriterMark;
CompressedFileWriterMark realMark = (CompressedFileWriterMark) mark;
long truncateTarget = realMark.uncDataOffset;
if (realMark.chunkOffset == chunkOffset)
{
buffer.position(realMark.validBufferBytes);
return;
}
syncInternal();
chunkOffset = realMark.chunkOffset;
int chunkSize = (int) (metadataWriter.chunkOffsetBy(realMark.nextChunkIndex) - chunkOffset - 4);
if (compressed.capacity() < chunkSize)
{
FileUtils.clean(compressed);
compressed = compressor.preferredBufferType().allocate(chunkSize);
}
try
{
compressed.clear();
compressed.limit(chunkSize);
fchannel.position(chunkOffset);
fchannel.read(compressed);
try
{
buffer.clear();
compressed.flip();
compressor.uncompress(compressed, buffer);
}
catch (IOException e)
{
throw new CorruptBlockException(getPath(), chunkOffset, chunkSize, e);
}
CRC32 checksum = new CRC32();
compressed.rewind();
checksum.update(compressed);
crcCheckBuffer.clear();
fchannel.read(crcCheckBuffer);
crcCheckBuffer.flip();
if (crcCheckBuffer.getInt() != (int) checksum.getValue())
throw new CorruptBlockException(getPath(), chunkOffset, chunkSize);
}
catch (CorruptBlockException e)
{
throw new CorruptSSTableException(e, getPath());
}
catch (EOFException e)
{
throw new CorruptSSTableException(new CorruptBlockException(getPath(), chunkOffset, chunkSize), getPath());
}
catch (IOException e)
{
throw new FSReadError(e, getPath());
}
buffer.position(realMark.validBufferBytes);
bufferOffset = truncateTarget - buffer.position();
chunkCount = realMark.nextChunkIndex - 1;
truncate(chunkOffset, bufferOffset);
metadataWriter.resetAndTruncate(realMark.nextChunkIndex - 1);
}
private void truncate(long toFileSize, long toBufferOffset)
{
try
{
fchannel.truncate(toFileSize);
lastFlushOffset = toBufferOffset;
}
catch (IOException e)
{
throw new FSWriteError(e, getPath());
}
}
private void seekToChunkStart()
{
if (getOnDiskFilePointer() != chunkOffset)
{
try
{
fchannel.position(chunkOffset);
}
catch (IOException e)
{
throw new FSReadError(e, getPath());
}
}
}
protected class TransactionalProxy extends SequentialWriter.TransactionalProxy
{
@Override
protected Throwable doCommit(Throwable accumulate)
{
return super.doCommit(metadataWriter.commit(accumulate));
}
@Override
protected Throwable doAbort(Throwable accumulate)
{
return super.doAbort(metadataWriter.abort(accumulate));
}
@Override
protected void doPrepare()
{
syncInternal();
digestFile.ifPresent(crcMetadata::writeFullChecksum);
sstableMetadataCollector.addCompressionRatio(compressedSize, uncompressedSize);
metadataWriter.finalizeLength(current(), chunkCount).prepareToCommit();
}
@Override
protected Throwable doPreCleanup(Throwable accumulate)
{
accumulate = super.doPreCleanup(accumulate);
if (compressed != null)
{
try { FileUtils.clean(compressed); }
catch (Throwable t) { accumulate = merge(accumulate, t); }
compressed = null;
}
return accumulate;
}
}
@Override
protected SequentialWriter.TransactionalProxy txnProxy()
{
return new TransactionalProxy();
}
protected static class CompressedFileWriterMark implements DataPosition
{
final long chunkOffset;
final long uncDataOffset;
final int validBufferBytes;
final int nextChunkIndex;
public CompressedFileWriterMark(long chunkOffset, long uncDataOffset, int validBufferBytes, int nextChunkIndex)
{
this.chunkOffset = chunkOffset;
this.uncDataOffset = uncDataOffset;
this.validBufferBytes = validBufferBytes;
this.nextChunkIndex = nextChunkIndex;
}
}
}