Skip to content

Commit 373af6b

Browse files
committed
use fs channel to write to file to improve speed of adding to translog
1 parent 5c08271 commit 373af6b

File tree

2 files changed

+31
-13
lines changed

2 files changed

+31
-13
lines changed

modules/elasticsearch/src/main/java/org/elasticsearch/index/translog/fs/FsTranslog.java

Lines changed: 23 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -35,7 +35,9 @@
3535

3636
import java.io.File;
3737
import java.io.IOException;
38+
import java.nio.ByteBuffer;
3839
import java.util.concurrent.atomic.AtomicInteger;
40+
import java.util.concurrent.atomic.AtomicLong;
3941

4042
/**
4143
* @author kimchy (shay.banon)
@@ -54,7 +56,8 @@ public class FsTranslog extends AbstractIndexShardComponent implements Translog
5456

5557
private final AtomicInteger operationCounter = new AtomicInteger();
5658

57-
private long lastPosition = 0;
59+
private AtomicLong lastPosition = new AtomicLong(0);
60+
private AtomicLong lastWrittenPosition = new AtomicLong(0);
5861

5962
private RafReference raf;
6063

@@ -113,7 +116,8 @@ public File location() {
113116
@Override public void newTranslog() throws TranslogException {
114117
synchronized (mutex) {
115118
operationCounter.set(0);
116-
lastPosition = 0;
119+
lastPosition.set(0);
120+
lastWrittenPosition.set(0);
117121
this.id = id + 1;
118122
if (raf != null) {
119123
raf.decreaseRefCount(true);
@@ -131,7 +135,8 @@ public File location() {
131135
@Override public void newTranslog(long id) throws TranslogException {
132136
synchronized (mutex) {
133137
operationCounter.set(0);
134-
lastPosition = 0;
138+
lastPosition.set(0);
139+
lastWrittenPosition.set(0);
135140
this.id = id;
136141
if (raf != null) {
137142
raf.decreaseRefCount(true);
@@ -158,12 +163,15 @@ public File location() {
158163
out.seek(0);
159164
out.writeInt(size - 4);
160165

166+
long position = lastPosition.getAndAdd(size);
167+
// use channel#write and not raf#write since it allows for concurrent writes
168+
// with regards to positions
169+
raf.channel().write(ByteBuffer.wrap(out.unsafeByteArray(), 0, size), position);
170+
if (syncOnEachOperation) {
171+
raf.channel().force(false);
172+
}
161173
synchronized (mutex) {
162-
raf.raf().write(out.unsafeByteArray(), 0, size);
163-
if (syncOnEachOperation) {
164-
sync();
165-
}
166-
lastPosition += size;
174+
lastWrittenPosition.getAndAdd(size);
167175
operationCounter.incrementAndGet();
168176
}
169177
} catch (Exception e) {
@@ -175,10 +183,11 @@ public File location() {
175183
synchronized (mutex) {
176184
try {
177185
raf.increaseRefCount();
186+
raf.channel().force(true); // sync here, so we make sure we read back teh data?
178187
if (useStream) {
179-
return new FsStreamSnapshot(shardId, this.id, raf, lastPosition, operationCounter.get(), operationCounter.get());
188+
return new FsStreamSnapshot(shardId, this.id, raf, lastWrittenPosition.get(), operationCounter.get(), operationCounter.get());
180189
} else {
181-
return new FsChannelSnapshot(shardId, this.id, raf, lastPosition, operationCounter.get(), operationCounter.get());
190+
return new FsChannelSnapshot(shardId, this.id, raf, lastWrittenPosition.get(), operationCounter.get(), operationCounter.get());
182191
}
183192
} catch (Exception e) {
184193
throw new TranslogException(shardId, "Failed to snapshot", e);
@@ -193,12 +202,13 @@ public File location() {
193202
}
194203
try {
195204
raf.increaseRefCount();
205+
raf.channel().force(true); // sync here, so we make sure we read back teh data?
196206
if (useStream) {
197-
FsStreamSnapshot newSnapshot = new FsStreamSnapshot(shardId, id, raf, lastPosition, operationCounter.get(), operationCounter.get() - snapshot.totalOperations());
207+
FsStreamSnapshot newSnapshot = new FsStreamSnapshot(shardId, id, raf, lastWrittenPosition.get(), operationCounter.get(), operationCounter.get() - snapshot.totalOperations());
198208
newSnapshot.seekForward(snapshot.position());
199209
return newSnapshot;
200210
} else {
201-
FsChannelSnapshot newSnapshot = new FsChannelSnapshot(shardId, id, raf, lastPosition, operationCounter.get(), operationCounter.get() - snapshot.totalOperations());
211+
FsChannelSnapshot newSnapshot = new FsChannelSnapshot(shardId, id, raf, lastWrittenPosition.get(), operationCounter.get(), operationCounter.get() - snapshot.totalOperations());
202212
newSnapshot.seekForward(snapshot.position());
203213
return newSnapshot;
204214
}
@@ -212,7 +222,7 @@ public File location() {
212222
synchronized (mutex) {
213223
if (raf != null) {
214224
try {
215-
raf.raf().getFD().sync();
225+
raf.channel().force(true);
216226
} catch (Exception e) {
217227
// ignore
218228
}

modules/elasticsearch/src/main/java/org/elasticsearch/index/translog/fs/RafReference.java

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -25,6 +25,7 @@
2525
import java.io.FileNotFoundException;
2626
import java.io.IOException;
2727
import java.io.RandomAccessFile;
28+
import java.nio.channels.FileChannel;
2829
import java.util.concurrent.atomic.AtomicInteger;
2930

3031
/**
@@ -36,18 +37,25 @@ public class RafReference {
3637

3738
private final RandomAccessFile raf;
3839

40+
private final FileChannel channel;
41+
3942
private final AtomicInteger refCount = new AtomicInteger();
4043

4144
public RafReference(File file) throws FileNotFoundException {
4245
this.file = file;
4346
this.raf = new RandomAccessFile(file, "rw");
47+
this.channel = raf.getChannel();
4448
this.refCount.incrementAndGet();
4549
}
4650

4751
public File file() {
4852
return this.file;
4953
}
5054

55+
public FileChannel channel() {
56+
return this.channel;
57+
}
58+
5159
public RandomAccessFile raf() {
5260
return this.raf;
5361
}

0 commit comments

Comments
 (0)