use fs channel to write to file to improve speed of adding to translog

This commit is contained in:
kimchy 2011-01-09 20:53:40 +02:00
parent 5c08271e74
commit 373af6b1e0
2 changed files with 31 additions and 13 deletions

View File

@ -35,7 +35,9 @@ import org.elasticsearch.index.translog.TranslogStreams;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
/** /**
* @author kimchy (shay.banon) * @author kimchy (shay.banon)
@ -54,7 +56,8 @@ public class FsTranslog extends AbstractIndexShardComponent implements Translog
private final AtomicInteger operationCounter = new AtomicInteger(); private final AtomicInteger operationCounter = new AtomicInteger();
private long lastPosition = 0; private AtomicLong lastPosition = new AtomicLong(0);
private AtomicLong lastWrittenPosition = new AtomicLong(0);
private RafReference raf; private RafReference raf;
@ -113,7 +116,8 @@ public class FsTranslog extends AbstractIndexShardComponent implements Translog
@Override public void newTranslog() throws TranslogException { @Override public void newTranslog() throws TranslogException {
synchronized (mutex) { synchronized (mutex) {
operationCounter.set(0); operationCounter.set(0);
lastPosition = 0; lastPosition.set(0);
lastWrittenPosition.set(0);
this.id = id + 1; this.id = id + 1;
if (raf != null) { if (raf != null) {
raf.decreaseRefCount(true); raf.decreaseRefCount(true);
@ -131,7 +135,8 @@ public class FsTranslog extends AbstractIndexShardComponent implements Translog
@Override public void newTranslog(long id) throws TranslogException { @Override public void newTranslog(long id) throws TranslogException {
synchronized (mutex) { synchronized (mutex) {
operationCounter.set(0); operationCounter.set(0);
lastPosition = 0; lastPosition.set(0);
lastWrittenPosition.set(0);
this.id = id; this.id = id;
if (raf != null) { if (raf != null) {
raf.decreaseRefCount(true); raf.decreaseRefCount(true);
@ -158,12 +163,15 @@ public class FsTranslog extends AbstractIndexShardComponent implements Translog
out.seek(0); out.seek(0);
out.writeInt(size - 4); out.writeInt(size - 4);
long position = lastPosition.getAndAdd(size);
// use channel#write and not raf#write since it allows for concurrent writes
// with regards to positions
raf.channel().write(ByteBuffer.wrap(out.unsafeByteArray(), 0, size), position);
if (syncOnEachOperation) {
raf.channel().force(false);
}
synchronized (mutex) { synchronized (mutex) {
raf.raf().write(out.unsafeByteArray(), 0, size); lastWrittenPosition.getAndAdd(size);
if (syncOnEachOperation) {
sync();
}
lastPosition += size;
operationCounter.incrementAndGet(); operationCounter.incrementAndGet();
} }
} catch (Exception e) { } catch (Exception e) {
@ -175,10 +183,11 @@ public class FsTranslog extends AbstractIndexShardComponent implements Translog
synchronized (mutex) { synchronized (mutex) {
try { try {
raf.increaseRefCount(); raf.increaseRefCount();
raf.channel().force(true); // sync here, so we make sure we read back teh data?
if (useStream) { if (useStream) {
return new FsStreamSnapshot(shardId, this.id, raf, lastPosition, operationCounter.get(), operationCounter.get()); return new FsStreamSnapshot(shardId, this.id, raf, lastWrittenPosition.get(), operationCounter.get(), operationCounter.get());
} else { } else {
return new FsChannelSnapshot(shardId, this.id, raf, lastPosition, operationCounter.get(), operationCounter.get()); return new FsChannelSnapshot(shardId, this.id, raf, lastWrittenPosition.get(), operationCounter.get(), operationCounter.get());
} }
} catch (Exception e) { } catch (Exception e) {
throw new TranslogException(shardId, "Failed to snapshot", e); throw new TranslogException(shardId, "Failed to snapshot", e);
@ -193,12 +202,13 @@ public class FsTranslog extends AbstractIndexShardComponent implements Translog
} }
try { try {
raf.increaseRefCount(); raf.increaseRefCount();
raf.channel().force(true); // sync here, so we make sure we read back teh data?
if (useStream) { if (useStream) {
FsStreamSnapshot newSnapshot = new FsStreamSnapshot(shardId, id, raf, lastPosition, operationCounter.get(), operationCounter.get() - snapshot.totalOperations()); FsStreamSnapshot newSnapshot = new FsStreamSnapshot(shardId, id, raf, lastWrittenPosition.get(), operationCounter.get(), operationCounter.get() - snapshot.totalOperations());
newSnapshot.seekForward(snapshot.position()); newSnapshot.seekForward(snapshot.position());
return newSnapshot; return newSnapshot;
} else { } else {
FsChannelSnapshot newSnapshot = new FsChannelSnapshot(shardId, id, raf, lastPosition, operationCounter.get(), operationCounter.get() - snapshot.totalOperations()); FsChannelSnapshot newSnapshot = new FsChannelSnapshot(shardId, id, raf, lastWrittenPosition.get(), operationCounter.get(), operationCounter.get() - snapshot.totalOperations());
newSnapshot.seekForward(snapshot.position()); newSnapshot.seekForward(snapshot.position());
return newSnapshot; return newSnapshot;
} }
@ -212,7 +222,7 @@ public class FsTranslog extends AbstractIndexShardComponent implements Translog
synchronized (mutex) { synchronized (mutex) {
if (raf != null) { if (raf != null) {
try { try {
raf.raf().getFD().sync(); raf.channel().force(true);
} catch (Exception e) { } catch (Exception e) {
// ignore // ignore
} }

View File

@ -25,6 +25,7 @@ import java.io.File;
import java.io.FileNotFoundException; import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.io.RandomAccessFile; import java.io.RandomAccessFile;
import java.nio.channels.FileChannel;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
/** /**
@ -36,11 +37,14 @@ public class RafReference {
private final RandomAccessFile raf; private final RandomAccessFile raf;
private final FileChannel channel;
private final AtomicInteger refCount = new AtomicInteger(); private final AtomicInteger refCount = new AtomicInteger();
public RafReference(File file) throws FileNotFoundException { public RafReference(File file) throws FileNotFoundException {
this.file = file; this.file = file;
this.raf = new RandomAccessFile(file, "rw"); this.raf = new RandomAccessFile(file, "rw");
this.channel = raf.getChannel();
this.refCount.incrementAndGet(); this.refCount.incrementAndGet();
} }
@ -48,6 +52,10 @@ public class RafReference {
return this.file; return this.file;
} }
public FileChannel channel() {
return this.channel;
}
public RandomAccessFile raf() { public RandomAccessFile raf() {
return this.raf; return this.raf;
} }