You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by dw...@apache.org on 2019/01/15 12:44:25 UTC
[lucene-solr] branch master updated: LUCENE-8474: (partial) removal
of accesses to RAMFile and RAMDirectory streams. Removal of
GrowableByteArrayDataOutput.
This is an automated email from the ASF dual-hosted git repository.
dweiss pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git
The following commit(s) were added to refs/heads/master by this push:
new d4e016a LUCENE-8474: (partial) removal of accesses to RAMFile and RAMDirectory streams. Removal of GrowableByteArrayDataOutput.
d4e016a is described below
commit d4e016afdf41baf0104f79e82d953a4650df42aa
Author: Dawid Weiss <dw...@apache.org>
AuthorDate: Tue Jan 15 13:42:25 2019 +0100
LUCENE-8474: (partial) removal of accesses to RAMFile and RAMDirectory streams. Removal of GrowableByteArrayDataOutput.
---
.../lucene/codecs/blockterms/BlockTermsWriter.java | 22 +--
.../blockterms/FixedGapTermsIndexWriter.java | 23 +--
.../blocktreeords/OrdsBlockTreeTermsWriter.java | 36 ++--
.../lucene/codecs/memory/DirectPostingsFormat.java | 14 +-
.../lucene/codecs/memory/FSTOrdTermsWriter.java | 49 +++---
.../lucene/codecs/memory/FSTTermsWriter.java | 12 +-
.../lucene/codecs/MultiLevelSkipListWriter.java | 19 ++-
.../codecs/blocktree/BlockTreeTermsWriter.java | 35 ++--
.../compressing/CompressingStoredFieldsWriter.java | 36 ++--
.../compressing/CompressingTermVectorsWriter.java | 20 ++-
.../lucene/codecs/lucene50/Lucene50SkipWriter.java | 13 +-
.../codecs/lucene70/Lucene70DocValuesConsumer.java | 31 ++--
.../org/apache/lucene/index/PrefixCodedTerms.java | 69 ++++----
.../org/apache/lucene/index/SortingLeafReader.java | 29 ++--
.../lucene/store/GrowableByteArrayDataOutput.java | 102 -----------
.../src/java/org/apache/lucene/util/IOUtils.java | 3 +-
.../java/org/apache/lucene/util/bkd/BKDWriter.java | 38 ++---
.../src/java/org/apache/lucene/util/fst/FST.java | 10 +-
.../org/apache/lucene/codecs/TestCodecUtil.java | 102 +++++------
.../lucene70/TestLucene70DocValuesFormat.java | 35 ++--
.../lucene/index/TestIndexWriterOnDiskFull.java | 2 +-
.../apache/lucene/search/TermInSetQueryTest.java | 4 +-
.../lucene/store/BaseDataOutputTestCase.java | 1 -
.../lucene/store/TestByteBuffersDataInput.java | 3 +-
.../lucene/store/TestByteBuffersDataOutput.java | 1 -
.../store/TestGrowableByteArrayDataOutput.java | 80 ---------
.../org/apache/lucene/store/TestHugeRamFile.java | 102 -----------
.../org/apache/lucene/store/TestRAMDirectory.java | 189 ---------------------
.../apache/lucene/replicator/nrt/PrimaryNode.java | 24 +--
.../lucene/replicator/nrt/SimpleTransLog.java | 15 +-
.../idversion/VersionBlockTreeTermsWriter.java | 34 ++--
.../apache/lucene/store/BaseDirectoryTestCase.java | 26 +--
.../apache/lucene/store/MockDirectoryWrapper.java | 47 +----
.../lucene/store/MockIndexOutputWrapper.java | 6 +-
.../org/apache/lucene/util/RamUsageTester.java | 3 +
35 files changed, 364 insertions(+), 871 deletions(-)
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsWriter.java b/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsWriter.java
index 9de9d73..f620bd8 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsWriter.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsWriter.java
@@ -28,16 +28,16 @@ import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.PostingsWriterBase;
import org.apache.lucene.codecs.TermStats;
-import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.ByteBuffersDataOutput;
import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
@@ -306,8 +306,8 @@ public class BlockTermsWriter extends FieldsConsumer implements Closeable {
return pos1;
}
- private final RAMOutputStream bytesWriter = new RAMOutputStream();
- private final RAMOutputStream bufferWriter = new RAMOutputStream();
+ private final ByteBuffersDataOutput bytesWriter = ByteBuffersDataOutput.newResettableInstance();
+ private final ByteBuffersDataOutput bufferWriter = ByteBuffersDataOutput.newResettableInstance();
private void flushBlock() throws IOException {
//System.out.println("BTW.flushBlock seg=" + segment + " pendingCount=" + pendingCount + " fp=" + out.getFilePointer());
@@ -333,8 +333,8 @@ public class BlockTermsWriter extends FieldsConsumer implements Closeable {
bytesWriter.writeVInt(suffix);
bytesWriter.writeBytes(pendingTerms[termCount].term.bytes(), commonPrefix, suffix);
}
- out.writeVInt((int) bytesWriter.getFilePointer());
- bytesWriter.writeTo(out);
+ out.writeVInt(Math.toIntExact(bytesWriter.size()));
+ bytesWriter.copyTo(out);
bytesWriter.reset();
// 3rd pass: write the freqs as byte[] blob
@@ -348,8 +348,8 @@ public class BlockTermsWriter extends FieldsConsumer implements Closeable {
bytesWriter.writeVLong(state.totalTermFreq-state.docFreq);
}
}
- out.writeVInt((int) bytesWriter.getFilePointer());
- bytesWriter.writeTo(out);
+ out.writeVInt(Math.toIntExact(bytesWriter.size()));
+ bytesWriter.copyTo(out);
bytesWriter.reset();
// 4th pass: write the metadata
@@ -361,12 +361,12 @@ public class BlockTermsWriter extends FieldsConsumer implements Closeable {
for (int i = 0; i < longsSize; i++) {
bytesWriter.writeVLong(longs[i]);
}
- bufferWriter.writeTo(bytesWriter);
+ bufferWriter.copyTo(bytesWriter);
bufferWriter.reset();
absolute = false;
}
- out.writeVInt((int) bytesWriter.getFilePointer());
- bytesWriter.writeTo(out);
+ out.writeVInt(Math.toIntExact(bytesWriter.size()));
+ bytesWriter.copyTo(out);
bytesWriter.reset();
lastPrevTerm.copyBytes(pendingTerms[pendingCount-1].term);
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/FixedGapTermsIndexWriter.java b/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/FixedGapTermsIndexWriter.java
index abb4cb0..7bbded8 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/FixedGapTermsIndexWriter.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/FixedGapTermsIndexWriter.java
@@ -17,13 +17,17 @@
package org.apache.lucene.codecs.blockterms;
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.TermStats;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.IOUtils;
@@ -31,10 +35,6 @@ import org.apache.lucene.util.StringHelper;
import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
import org.apache.lucene.util.packed.PackedInts;
-import java.util.List;
-import java.util.ArrayList;
-import java.io.IOException;
-
/**
* Selects every Nth term as and index term, and hold term
* bytes (mostly) fully expanded in memory. This terms index
@@ -111,11 +111,11 @@ public class FixedGapTermsIndexWriter extends TermsIndexWriterBase {
long packedOffsetsStart;
private long numTerms;
- private RAMOutputStream offsetsBuffer = new RAMOutputStream();
+ private ByteBuffersDataOutput offsetsBuffer = ByteBuffersDataOutput.newResettableInstance();
private MonotonicBlockPackedWriter termOffsets = new MonotonicBlockPackedWriter(offsetsBuffer, BLOCKSIZE);
private long currentOffset;
- private RAMOutputStream addressBuffer = new RAMOutputStream();
+ private ByteBuffersDataOutput addressBuffer = ByteBuffersDataOutput.newResettableInstance();
private MonotonicBlockPackedWriter termAddresses = new MonotonicBlockPackedWriter(addressBuffer, BLOCKSIZE);
private final BytesRefBuilder lastTerm = new BytesRefBuilder();
@@ -183,18 +183,19 @@ public class FixedGapTermsIndexWriter extends TermsIndexWriterBase {
// relative to our indexStart
termAddresses.finish();
- addressBuffer.writeTo(out);
+ addressBuffer.copyTo(out);
packedOffsetsStart = out.getFilePointer();
// write offsets into the byte[] terms
termOffsets.finish();
- offsetsBuffer.writeTo(out);
+ offsetsBuffer.copyTo(out);
// our referrer holds onto us, while other fields are
// being written, so don't tie up this RAM:
termOffsets = termAddresses = null;
- addressBuffer = offsetsBuffer = null;
+ addressBuffer = null;
+ offsetsBuffer = null;
}
}
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsBlockTreeTermsWriter.java b/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsBlockTreeTermsWriter.java
index bdc0738..769c732 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsBlockTreeTermsWriter.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsBlockTreeTermsWriter.java
@@ -36,8 +36,8 @@ import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.ByteBuffersDataOutput;
import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
@@ -330,12 +330,12 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
return "BLOCK: " + brToString(prefix);
}
- public void compileIndex(List<PendingBlock> blocks, RAMOutputStream scratchBytes, IntsRefBuilder scratchIntsRef) throws IOException {
+ public void compileIndex(List<PendingBlock> blocks, ByteBuffersDataOutput scratchBytes, IntsRefBuilder scratchIntsRef) throws IOException {
assert (isFloor && blocks.size() > 1) || (isFloor == false && blocks.size() == 1): "isFloor=" + isFloor + " blocks=" + blocks;
assert this == blocks.get(0);
- assert scratchBytes.getFilePointer() == 0;
+ assert scratchBytes.size() == 0;
// TODO: try writing the leading vLong in MSB order
// (opposite of what Lucene does today), for better
@@ -368,10 +368,8 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
// System.out.println(" compile index for prefix=" + prefix);
//}
//indexBuilder.DEBUG = false;
- final byte[] bytes = new byte[(int) scratchBytes.getFilePointer()];
+ final byte[] bytes = scratchBytes.toArrayCopy();
assert bytes.length > 0;
- // System.out.println(" bytes=" + bytes.length);
- scratchBytes.writeTo(bytes, 0);
indexBuilder.add(Util.toIntsRef(prefix, scratchIntsRef),
FST_OUTPUTS.newOutput(new BytesRef(bytes, 0, bytes.length),
0, Long.MAX_VALUE-(sumTotalTermCount-1)));
@@ -423,7 +421,7 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
}
}
- private final RAMOutputStream scratchBytes = new RAMOutputStream();
+ private final ByteBuffersDataOutput scratchBytes = ByteBuffersDataOutput.newResettableInstance();
private final IntsRefBuilder scratchIntsRef = new IntsRefBuilder();
class TermsWriter {
@@ -642,7 +640,7 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
assert longs[pos] >= 0;
metaWriter.writeVLong(longs[pos]);
}
- bytesWriter.writeTo(metaWriter);
+ bytesWriter.copyTo(metaWriter);
bytesWriter.reset();
absolute = false;
}
@@ -693,7 +691,7 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
assert longs[pos] >= 0;
metaWriter.writeVLong(longs[pos]);
}
- bytesWriter.writeTo(metaWriter);
+ bytesWriter.copyTo(metaWriter);
bytesWriter.reset();
absolute = false;
@@ -738,18 +736,18 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
// search on lookup
// Write suffixes byte[] blob to terms dict output:
- out.writeVInt((int) (suffixWriter.getFilePointer() << 1) | (isLeafBlock ? 1:0));
- suffixWriter.writeTo(out);
+ out.writeVInt((int) (suffixWriter.size() << 1) | (isLeafBlock ? 1:0));
+ suffixWriter.copyTo(out);
suffixWriter.reset();
// Write term stats byte[] blob
- out.writeVInt((int) statsWriter.getFilePointer());
- statsWriter.writeTo(out);
+ out.writeVInt((int) statsWriter.size());
+ statsWriter.copyTo(out);
statsWriter.reset();
// Write term meta data byte[] blob
- out.writeVInt((int) metaWriter.getFilePointer());
- metaWriter.writeTo(out);
+ out.writeVInt((int) metaWriter.size());
+ metaWriter.copyTo(out);
metaWriter.reset();
// if (DEBUG) {
@@ -885,10 +883,10 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
}
}
- private final RAMOutputStream suffixWriter = new RAMOutputStream();
- private final RAMOutputStream statsWriter = new RAMOutputStream();
- private final RAMOutputStream metaWriter = new RAMOutputStream();
- private final RAMOutputStream bytesWriter = new RAMOutputStream();
+ private final ByteBuffersDataOutput suffixWriter = ByteBuffersDataOutput.newResettableInstance();
+ private final ByteBuffersDataOutput statsWriter = ByteBuffersDataOutput.newResettableInstance();
+ private final ByteBuffersDataOutput metaWriter = ByteBuffersDataOutput.newResettableInstance();
+ private final ByteBuffersDataOutput bytesWriter = ByteBuffersDataOutput.newResettableInstance();
}
private boolean closed;
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java
index 6c8853d..15c0025 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java
@@ -39,8 +39,8 @@ import org.apache.lucene.index.SlowImpactsEnum;
import org.apache.lucene.index.TermState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.ByteBuffersDataOutput;
import org.apache.lucene.store.IOContext;
-import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.Accountables;
import org.apache.lucene.util.ArrayUtil;
@@ -333,7 +333,7 @@ public final class DirectPostingsFormat extends PostingsFormat {
final IntArrayWriter scratch = new IntArrayWriter();
// Used for payloads, if any:
- final RAMOutputStream ros = new RAMOutputStream();
+ final ByteBuffersDataOutput ros = ByteBuffersDataOutput.newResettableInstance();
// if (DEBUG) {
// System.out.println("\nLOAD terms seg=" + state.segmentInfo.name + " field=" + field + " hasOffsets=" + hasOffsets + " hasFreq=" + hasFreq + " hasPos=" + hasPos + " hasPayloads=" + hasPayloads);
@@ -374,7 +374,6 @@ public final class DirectPostingsFormat extends PostingsFormat {
int docID;
if (docFreq <= lowFreqCutoff) {
-
ros.reset();
// Pack postings for low-freq terms into a single int[]:
@@ -404,14 +403,7 @@ public final class DirectPostingsFormat extends PostingsFormat {
}
}
- final byte[] payloads;
- if (hasPayloads) {
- payloads = new byte[(int) ros.getFilePointer()];
- ros.writeTo(payloads, 0);
- } else {
- payloads = null;
- }
-
+ final byte[] payloads = hasPayloads ? ros.toArrayCopy() : null;
final int[] postings = scratch.get();
ent = new LowFreqTerm(postings, payloads, docFreq, (int) totalTermFreq);
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsWriter.java b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsWriter.java
index b59d41d..0077045 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsWriter.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsWriter.java
@@ -34,9 +34,9 @@ import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.ByteBuffersDataOutput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.IOUtils;
@@ -236,14 +236,14 @@ public class FSTOrdTermsWriter extends FieldsConsumer {
blockOut.writeVLong(field.sumDocFreq);
blockOut.writeVInt(field.docCount);
blockOut.writeVInt(field.longsSize);
- blockOut.writeVLong(field.statsOut.getFilePointer());
- blockOut.writeVLong(field.metaLongsOut.getFilePointer());
- blockOut.writeVLong(field.metaBytesOut.getFilePointer());
+ blockOut.writeVLong(field.statsOut.size());
+ blockOut.writeVLong(field.metaLongsOut.size());
+ blockOut.writeVLong(field.metaBytesOut.size());
- field.skipOut.writeTo(blockOut);
- field.statsOut.writeTo(blockOut);
- field.metaLongsOut.writeTo(blockOut);
- field.metaBytesOut.writeTo(blockOut);
+ field.skipOut.copyTo(blockOut);
+ field.statsOut.copyTo(blockOut);
+ field.metaLongsOut.copyTo(blockOut);
+ field.metaBytesOut.copyTo(blockOut);
field.dict.save(indexOut);
}
writeTrailer(blockOut, blockDirStart);
@@ -277,13 +277,13 @@ public class FSTOrdTermsWriter extends FieldsConsumer {
// TODO: block encode each part
// vint encode next skip point (fully decoded when reading)
- public RAMOutputStream skipOut;
+ public ByteBuffersDataOutput skipOut;
// vint encode df, (ttf-df)
- public RAMOutputStream statsOut;
+ public ByteBuffersDataOutput statsOut;
// vint encode monotonic long[] and length for corresponding byte[]
- public RAMOutputStream metaLongsOut;
+ public ByteBuffersDataOutput metaLongsOut;
// generic byte[]
- public RAMOutputStream metaBytesOut;
+ public ByteBuffersDataOutput metaBytesOut;
}
final class TermsWriter {
@@ -294,11 +294,10 @@ public class FSTOrdTermsWriter extends FieldsConsumer {
private long numTerms;
private final IntsRefBuilder scratchTerm = new IntsRefBuilder();
- private final RAMOutputStream statsOut = new RAMOutputStream();
- private final RAMOutputStream metaLongsOut = new RAMOutputStream();
- private final RAMOutputStream metaBytesOut = new RAMOutputStream();
-
- private final RAMOutputStream skipOut = new RAMOutputStream();
+ private final ByteBuffersDataOutput statsOut = new ByteBuffersDataOutput();
+ private final ByteBuffersDataOutput metaLongsOut = new ByteBuffersDataOutput();
+ private final ByteBuffersDataOutput metaBytesOut = new ByteBuffersDataOutput();
+ private final ByteBuffersDataOutput skipOut = new ByteBuffersDataOutput();
private long lastBlockStatsFP;
private long lastBlockMetaLongsFP;
private long lastBlockMetaBytesFP;
@@ -345,12 +344,12 @@ public class FSTOrdTermsWriter extends FieldsConsumer {
metaLongsOut.writeVLong(longs[i] - lastLongs[i]);
lastLongs[i] = longs[i];
}
- metaLongsOut.writeVLong(metaBytesOut.getFilePointer() - lastMetaBytesFP);
+ metaLongsOut.writeVLong(metaBytesOut.size() - lastMetaBytesFP);
builder.add(Util.toIntsRef(text, scratchTerm), numTerms);
numTerms++;
- lastMetaBytesFP = metaBytesOut.getFilePointer();
+ lastMetaBytesFP = metaBytesOut.size();
}
public void finish(long sumTotalTermFreq, long sumDocFreq, int docCount) throws IOException {
@@ -372,15 +371,15 @@ public class FSTOrdTermsWriter extends FieldsConsumer {
}
private void bufferSkip() throws IOException {
- skipOut.writeVLong(statsOut.getFilePointer() - lastBlockStatsFP);
- skipOut.writeVLong(metaLongsOut.getFilePointer() - lastBlockMetaLongsFP);
- skipOut.writeVLong(metaBytesOut.getFilePointer() - lastBlockMetaBytesFP);
+ skipOut.writeVLong(statsOut.size() - lastBlockStatsFP);
+ skipOut.writeVLong(metaLongsOut.size() - lastBlockMetaLongsFP);
+ skipOut.writeVLong(metaBytesOut.size() - lastBlockMetaBytesFP);
for (int i = 0; i < longsSize; i++) {
skipOut.writeVLong(lastLongs[i] - lastBlockLongs[i]);
}
- lastBlockStatsFP = statsOut.getFilePointer();
- lastBlockMetaLongsFP = metaLongsOut.getFilePointer();
- lastBlockMetaBytesFP = metaBytesOut.getFilePointer();
+ lastBlockStatsFP = statsOut.size();
+ lastBlockMetaLongsFP = metaLongsOut.size();
+ lastBlockMetaBytesFP = metaBytesOut.size();
System.arraycopy(lastLongs, 0, lastBlockLongs, 0, longsSize);
}
}
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsWriter.java b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsWriter.java
index 8e55d41..22eff0b 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsWriter.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsWriter.java
@@ -26,17 +26,17 @@ import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.codecs.FieldsConsumer;
import org.apache.lucene.codecs.NormsProducer;
import org.apache.lucene.codecs.PostingsWriterBase;
-import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.ByteBuffersDataOutput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.IOUtils;
@@ -254,7 +254,7 @@ public class FSTTermsWriter extends FieldsConsumer {
private long numTerms;
private final IntsRefBuilder scratchTerm = new IntsRefBuilder();
- private final RAMOutputStream metaWriter = new RAMOutputStream();
+ private final ByteBuffersDataOutput metaWriter = ByteBuffersDataOutput.newResettableInstance();
TermsWriter(FieldInfo fieldInfo) {
this.numTerms = 0;
@@ -272,10 +272,8 @@ public class FSTTermsWriter extends FieldsConsumer {
meta.docFreq = state.docFreq;
meta.totalTermFreq = state.totalTermFreq;
postingsWriter.encodeTerm(meta.longs, metaWriter, fieldInfo, state, true);
- final int bytesSize = (int)metaWriter.getFilePointer();
- if (bytesSize > 0) {
- meta.bytes = new byte[bytesSize];
- metaWriter.writeTo(meta.bytes, 0);
+ if (metaWriter.size() > 0) {
+ meta.bytes = metaWriter.toArrayCopy();
metaWriter.reset();
}
builder.add(Util.toIntsRef(text, scratchTerm), meta);
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/MultiLevelSkipListWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/MultiLevelSkipListWriter.java
index 8e090be..7cf04a0 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/MultiLevelSkipListWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/MultiLevelSkipListWriter.java
@@ -19,8 +19,9 @@ package org.apache.lucene.codecs;
import java.io.IOException;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.MathUtil;
/**
@@ -62,7 +63,7 @@ public abstract class MultiLevelSkipListWriter {
private final int skipMultiplier;
/** for every skip level a different buffer is used */
- private RAMOutputStream[] skipBuffer;
+ private ByteBuffersDataOutput[] skipBuffer;
/** Creates a {@code MultiLevelSkipListWriter}. */
protected MultiLevelSkipListWriter(int skipInterval, int skipMultiplier, int maxSkipLevels, int df) {
@@ -93,9 +94,9 @@ public abstract class MultiLevelSkipListWriter {
/** Allocates internal skip buffers. */
protected void init() {
- skipBuffer = new RAMOutputStream[numberOfSkipLevels];
+ skipBuffer = new ByteBuffersDataOutput [numberOfSkipLevels];
for (int i = 0; i < numberOfSkipLevels; i++) {
- skipBuffer[i] = new RAMOutputStream();
+ skipBuffer[i] = ByteBuffersDataOutput.newResettableInstance();
}
}
@@ -116,7 +117,7 @@ public abstract class MultiLevelSkipListWriter {
* @param level the level skip data shall be writing for
* @param skipBuffer the skip buffer to write to
*/
- protected abstract void writeSkipData(int level, IndexOutput skipBuffer) throws IOException;
+ protected abstract void writeSkipData(int level, DataOutput skipBuffer) throws IOException;
/**
* Writes the current skip data to the buffers. The current document frequency determines
@@ -142,7 +143,7 @@ public abstract class MultiLevelSkipListWriter {
for (int level = 0; level < numLevels; level++) {
writeSkipData(level, skipBuffer[level]);
- long newChildPointer = skipBuffer[level].getFilePointer();
+ long newChildPointer = skipBuffer[level].size();
if (level != 0) {
// store child pointers for all levels except the lowest
@@ -166,13 +167,13 @@ public abstract class MultiLevelSkipListWriter {
if (skipBuffer == null || skipBuffer.length == 0) return skipPointer;
for (int level = numberOfSkipLevels - 1; level > 0; level--) {
- long length = skipBuffer[level].getFilePointer();
+ long length = skipBuffer[level].size();
if (length > 0) {
output.writeVLong(length);
- skipBuffer[level].writeTo(output);
+ skipBuffer[level].copyTo(output);
}
}
- skipBuffer[0].writeTo(output);
+ skipBuffer[0].copyTo(output);
return skipPointer;
}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java
index ec3f6e6..3059d5a 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java
@@ -34,9 +34,9 @@ import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.ByteBuffersDataOutput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
@@ -428,12 +428,12 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
return "BLOCK: prefix=" + brToString(prefix);
}
- public void compileIndex(List<PendingBlock> blocks, RAMOutputStream scratchBytes, IntsRefBuilder scratchIntsRef) throws IOException {
+ public void compileIndex(List<PendingBlock> blocks, ByteBuffersDataOutput scratchBytes, IntsRefBuilder scratchIntsRef) throws IOException {
assert (isFloor && blocks.size() > 1) || (isFloor == false && blocks.size() == 1): "isFloor=" + isFloor + " blocks=" + blocks;
assert this == blocks.get(0);
- assert scratchBytes.getFilePointer() == 0;
+ assert scratchBytes.size() == 0;
// TODO: try writing the leading vLong in MSB order
// (opposite of what Lucene does today), for better
@@ -461,9 +461,8 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
// System.out.println(" compile index for prefix=" + prefix);
//}
//indexBuilder.DEBUG = false;
- final byte[] bytes = new byte[(int) scratchBytes.getFilePointer()];
+ final byte[] bytes = scratchBytes.toArrayCopy();
assert bytes.length > 0;
- scratchBytes.writeTo(bytes, 0);
indexBuilder.add(Util.toIntsRef(prefix, scratchIntsRef), new BytesRef(bytes, 0, bytes.length));
scratchBytes.reset();
@@ -504,7 +503,7 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
}
}
- private final RAMOutputStream scratchBytes = new RAMOutputStream();
+ private final ByteBuffersDataOutput scratchBytes = ByteBuffersDataOutput.newResettableInstance();
private final IntsRefBuilder scratchIntsRef = new IntsRefBuilder();
static final BytesRef EMPTY_BYTES_REF = new BytesRef();
@@ -727,7 +726,7 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
assert longs[pos] >= 0;
metaWriter.writeVLong(longs[pos]);
}
- bytesWriter.writeTo(metaWriter);
+ bytesWriter.copyTo(metaWriter);
bytesWriter.reset();
absolute = false;
}
@@ -778,7 +777,7 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
assert longs[pos] >= 0;
metaWriter.writeVLong(longs[pos]);
}
- bytesWriter.writeTo(metaWriter);
+ bytesWriter.copyTo(metaWriter);
bytesWriter.reset();
absolute = false;
} else {
@@ -817,18 +816,18 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
// search on lookup
// Write suffixes byte[] blob to terms dict output:
- termsOut.writeVInt((int) (suffixWriter.getFilePointer() << 1) | (isLeafBlock ? 1:0));
- suffixWriter.writeTo(termsOut);
+ termsOut.writeVInt((int) (suffixWriter.size() << 1) | (isLeafBlock ? 1:0));
+ suffixWriter.copyTo(termsOut);
suffixWriter.reset();
// Write term stats byte[] blob
- termsOut.writeVInt((int) statsWriter.getFilePointer());
- statsWriter.writeTo(termsOut);
+ termsOut.writeVInt((int) statsWriter.size());
+ statsWriter.copyTo(termsOut);
statsWriter.reset();
// Write term meta data byte[] blob
- termsOut.writeVInt((int) metaWriter.getFilePointer());
- metaWriter.writeTo(termsOut);
+ termsOut.writeVInt((int) metaWriter.size());
+ metaWriter.copyTo(termsOut);
metaWriter.reset();
// if (DEBUG) {
@@ -976,10 +975,10 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
}
}
- private final RAMOutputStream suffixWriter = new RAMOutputStream();
- private final RAMOutputStream statsWriter = new RAMOutputStream();
- private final RAMOutputStream metaWriter = new RAMOutputStream();
- private final RAMOutputStream bytesWriter = new RAMOutputStream();
+ private final ByteBuffersDataOutput suffixWriter = ByteBuffersDataOutput.newResettableInstance();
+ private final ByteBuffersDataOutput statsWriter = ByteBuffersDataOutput.newResettableInstance();
+ private final ByteBuffersDataOutput metaWriter = ByteBuffersDataOutput.newResettableInstance();
+ private final ByteBuffersDataOutput bytesWriter = ByteBuffersDataOutput.newResettableInstance();
}
private boolean closed;
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java
index 5b8eb9e..5d779ab4 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java
@@ -17,6 +17,8 @@
package org.apache.lucene.codecs.compressing;
+import static org.apache.lucene.search.DocIdSetIterator.*;
+
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
@@ -33,9 +35,9 @@ import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.store.ByteBuffersDataOutput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.GrowableByteArrayDataOutput;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
@@ -46,8 +48,6 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.packed.PackedInts;
-import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
-
/**
* {@link StoredFieldsWriter} impl for {@link CompressingStoredFieldsFormat}.
* @lucene.experimental
@@ -84,7 +84,7 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
private final int chunkSize;
private final int maxDocsPerChunk;
- private final GrowableByteArrayDataOutput bufferedDocs;
+ private final ByteBuffersDataOutput bufferedDocs;
private int[] numStoredFields; // number of stored fields
private int[] endOffsets; // end offsets in bufferedDocs
private int docBase; // doc ID at the beginning of the chunk
@@ -103,7 +103,7 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
this.chunkSize = chunkSize;
this.maxDocsPerChunk = maxDocsPerChunk;
this.docBase = 0;
- this.bufferedDocs = new GrowableByteArrayDataOutput(chunkSize);
+ this.bufferedDocs = ByteBuffersDataOutput.newResettableInstance();
this.numStoredFields = new int[16];
this.endOffsets = new int[16];
this.numBufferedDocs = 0;
@@ -162,7 +162,7 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
}
this.numStoredFields[numBufferedDocs] = numStoredFieldsInDoc;
numStoredFieldsInDoc = 0;
- endOffsets[numBufferedDocs] = bufferedDocs.getPosition();
+ endOffsets[numBufferedDocs] = Math.toIntExact(bufferedDocs.size());
++numBufferedDocs;
if (triggerFlush()) {
flush();
@@ -215,7 +215,7 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
}
private boolean triggerFlush() {
- return bufferedDocs.getPosition() >= chunkSize || // chunks of at least chunkSize bytes
+ return bufferedDocs.size() >= chunkSize || // chunks of at least chunkSize bytes
numBufferedDocs >= maxDocsPerChunk;
}
@@ -228,17 +228,23 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
lengths[i] = endOffsets[i] - endOffsets[i - 1];
assert lengths[i] >= 0;
}
- final boolean sliced = bufferedDocs.getPosition() >= 2 * chunkSize;
+ final boolean sliced = bufferedDocs.size() >= 2 * chunkSize;
writeHeader(docBase, numBufferedDocs, numStoredFields, lengths, sliced);
- // compress stored fields to fieldsStream
+ // compress stored fields to fieldsStream.
+ //
+ // TODO: do we need to slice it since we already have the slices in the buffer? Perhaps
+ // we should use max-block-bits restriction on the buffer itself, then we won't have to check it here.
+ byte [] content = bufferedDocs.toArrayCopy();
+ bufferedDocs.reset();
+
if (sliced) {
// big chunk, slice it
- for (int compressed = 0; compressed < bufferedDocs.getPosition(); compressed += chunkSize) {
- compressor.compress(bufferedDocs.getBytes(), compressed, Math.min(chunkSize, bufferedDocs.getPosition() - compressed), fieldsStream);
+ for (int compressed = 0; compressed < content.length; compressed += chunkSize) {
+ compressor.compress(content, compressed, Math.min(chunkSize, content.length - compressed), fieldsStream);
}
} else {
- compressor.compress(bufferedDocs.getBytes(), 0, bufferedDocs.getPosition(), fieldsStream);
+ compressor.compress(content, 0, content.length, fieldsStream);
}
// reset
@@ -464,7 +470,7 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
flush();
numDirtyChunks++; // incomplete: we had to force this flush
} else {
- assert bufferedDocs.getPosition() == 0;
+ assert bufferedDocs.size() == 0;
}
if (docBase != numDocs) {
throw new RuntimeException("Wrote " + docBase + " docs, finish called with numDocs=" + numDocs);
@@ -473,9 +479,9 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
fieldsStream.writeVLong(numChunks);
fieldsStream.writeVLong(numDirtyChunks);
CodecUtil.writeFooter(fieldsStream);
- assert bufferedDocs.getPosition() == 0;
+ assert bufferedDocs.size() == 0;
}
-
+
// bulk merge is scary: its caused corruption bugs in the past.
// we try to be extra safe with this impl, but add an escape hatch to
// have a workaround for undiscovered bugs.
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java
index 4f8d004..46ac128 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java
@@ -35,9 +35,9 @@ import org.apache.lucene.index.Fields;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.store.ByteBuffersDataOutput;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.GrowableByteArrayDataOutput;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
@@ -198,8 +198,8 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
private FieldData curField; // current field
private final BytesRef lastTerm;
private int[] positionsBuf, startOffsetsBuf, lengthsBuf, payloadLengthsBuf;
- private final GrowableByteArrayDataOutput termSuffixes; // buffered term suffixes
- private final GrowableByteArrayDataOutput payloadBytes; // buffered term payloads
+ private final ByteBuffersDataOutput termSuffixes; // buffered term suffixes
+ private final ByteBuffersDataOutput payloadBytes; // buffered term payloads
private final BlockPackedWriter writer;
/** Sole constructor. */
@@ -213,8 +213,8 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
numDocs = 0;
pendingDocs = new ArrayDeque<>();
- termSuffixes = new GrowableByteArrayDataOutput(ArrayUtil.oversize(chunkSize, 1));
- payloadBytes = new GrowableByteArrayDataOutput(ArrayUtil.oversize(1, 1));
+ termSuffixes = ByteBuffersDataOutput.newResettableInstance();
+ payloadBytes = ByteBuffersDataOutput.newResettableInstance();
lastTerm = new BytesRef(ArrayUtil.oversize(30, 1));
boolean success = false;
@@ -269,7 +269,7 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
@Override
public void finishDocument() throws IOException {
// append the payload bytes of the doc after its terms
- termSuffixes.writeBytes(payloadBytes.getBytes(), payloadBytes.getPosition());
+ payloadBytes.copyTo(termSuffixes);
payloadBytes.reset();
++numDocs;
if (triggerFlush()) {
@@ -322,7 +322,7 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
}
private boolean triggerFlush() {
- return termSuffixes.getPosition() >= chunkSize
+ return termSuffixes.size() >= chunkSize
|| pendingDocs.size() >= MAX_DOCUMENTS_PER_CHUNK;
}
@@ -361,7 +361,11 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
flushPayloadLengths();
// compress terms and payloads and write them to the output
- compressor.compress(termSuffixes.getBytes(), 0, termSuffixes.getPosition(), vectorsStream);
+ //
+ // TODO: We could compress in the slices we already have in the buffer (min/max slice
+ // can be set on the buffer itself).
+ byte[] content = termSuffixes.toArrayCopy();
+ compressor.compress(content, 0, content.length, vectorsStream);
}
// reset
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SkipWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SkipWriter.java
index 37044b5..9b9b4de 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SkipWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SkipWriter.java
@@ -25,8 +25,9 @@ import java.util.SortedSet;
import org.apache.lucene.codecs.CompetitiveImpactAccumulator;
import org.apache.lucene.codecs.MultiLevelSkipListWriter;
import org.apache.lucene.index.Impact;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMOutputStream;
/**
* Write skip lists with multiple levels, and support skip within block ints.
@@ -162,10 +163,10 @@ final class Lucene50SkipWriter extends MultiLevelSkipListWriter {
bufferSkip(numDocs);
}
- private final RAMOutputStream freqNormOut = new RAMOutputStream();
+ private final ByteBuffersDataOutput freqNormOut = ByteBuffersDataOutput.newResettableInstance();
@Override
- protected void writeSkipData(int level, IndexOutput skipBuffer) throws IOException {
+ protected void writeSkipData(int level, DataOutput skipBuffer) throws IOException {
int delta = curDoc - lastSkipDoc[level];
@@ -197,13 +198,13 @@ final class Lucene50SkipWriter extends MultiLevelSkipListWriter {
curCompetitiveFreqNorms[level + 1].addAll(competitiveFreqNorms);
}
writeImpacts(competitiveFreqNorms, freqNormOut);
- skipBuffer.writeVInt(Math.toIntExact(freqNormOut.getFilePointer()));
- freqNormOut.writeTo(skipBuffer);
+ skipBuffer.writeVInt(Math.toIntExact(freqNormOut.size()));
+ freqNormOut.copyTo(skipBuffer);
freqNormOut.reset();
competitiveFreqNorms.clear();
}
- static void writeImpacts(CompetitiveImpactAccumulator acc, IndexOutput out) throws IOException {
+ static void writeImpacts(CompetitiveImpactAccumulator acc, DataOutput out) throws IOException {
SortedSet<Impact> impacts = acc.getCompetitiveFreqNormPairs();
Impact previous = new Impact(0, 0);
for (Impact impact : impacts) {
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesConsumer.java
index e25e8e7..6db3cca 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesConsumer.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesConsumer.java
@@ -44,9 +44,9 @@ import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.SortedSetSelector;
-import org.apache.lucene.store.GrowableByteArrayDataOutput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.ByteBuffersIndexOutput;
import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.IOUtils;
@@ -285,7 +285,7 @@ final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Close
private void writeValuesMultipleBlocks(SortedNumericDocValues values, long gcd) throws IOException {
final long[] buffer = new long[NUMERIC_BLOCK_SIZE];
- final GrowableByteArrayDataOutput encodeBuffer = new GrowableByteArrayDataOutput(NUMERIC_BLOCK_SIZE);
+ final ByteBuffersDataOutput encodeBuffer = ByteBuffersDataOutput.newResettableInstance();
int upTo = 0;
for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
for (int i = 0, count = values.docValueCount(); i < count; ++i) {
@@ -301,7 +301,7 @@ final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Close
}
}
- private void writeBlock(long[] values, int length, long gcd, GrowableByteArrayDataOutput buffer) throws IOException {
+ private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutput buffer) throws IOException {
assert length > 0;
long min = values[0];
long max = values[0];
@@ -317,7 +317,7 @@ final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Close
} else {
final int bitsPerValue = DirectWriter.unsignedBitsRequired(max - min);
buffer.reset();
- assert buffer.getPosition() == 0;
+ assert buffer.size() == 0;
final DirectWriter w = DirectWriter.getInstance(buffer, length, bitsPerValue);
for (int i = 0; i < length; ++i) {
w.add((values[i] - min) / gcd);
@@ -325,8 +325,8 @@ final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Close
w.finish();
data.writeByte((byte) bitsPerValue);
data.writeLong(min);
- data.writeInt(buffer.getPosition());
- data.writeBytes(buffer.getBytes(), buffer.getPosition());
+ data.writeInt(Math.toIntExact(buffer.size()));
+ buffer.copyTo(data);
}
}
@@ -442,10 +442,11 @@ final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Close
meta.writeVLong(size);
meta.writeInt(Lucene70DocValuesFormat.TERMS_DICT_BLOCK_SHIFT);
- RAMOutputStream addressBuffer = new RAMOutputStream();
+ ByteBuffersDataOutput addressBuffer = new ByteBuffersDataOutput();
+ ByteBuffersIndexOutput addressOutput = new ByteBuffersIndexOutput(addressBuffer, "temp", "temp");
meta.writeInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
long numBlocks = (size + Lucene70DocValuesFormat.TERMS_DICT_BLOCK_MASK) >>> Lucene70DocValuesFormat.TERMS_DICT_BLOCK_SHIFT;
- DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance(meta, addressBuffer, numBlocks, DIRECT_MONOTONIC_BLOCK_SHIFT);
+ DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance(meta, addressOutput, numBlocks, DIRECT_MONOTONIC_BLOCK_SHIFT);
BytesRefBuilder previous = new BytesRefBuilder();
long ord = 0;
@@ -480,7 +481,7 @@ final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Close
meta.writeLong(start);
meta.writeLong(data.getFilePointer() - start);
start = data.getFilePointer();
- addressBuffer.writeTo(data);
+ addressBuffer.copyTo(data);
meta.writeLong(start);
meta.writeLong(data.getFilePointer() - start);
@@ -494,9 +495,10 @@ final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Close
long start = data.getFilePointer();
long numBlocks = 1L + ((size + Lucene70DocValuesFormat.TERMS_DICT_REVERSE_INDEX_MASK) >>> Lucene70DocValuesFormat.TERMS_DICT_REVERSE_INDEX_SHIFT);
- RAMOutputStream addressBuffer = new RAMOutputStream();
- DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance(meta, addressBuffer, numBlocks, DIRECT_MONOTONIC_BLOCK_SHIFT);
-
+ ByteBuffersDataOutput addressBuffer = new ByteBuffersDataOutput();
+ DirectMonotonicWriter writer;
+ try (ByteBuffersIndexOutput addressOutput = new ByteBuffersIndexOutput(addressBuffer, "temp", "temp")) {
+ writer = DirectMonotonicWriter.getInstance(meta, addressOutput, numBlocks, DIRECT_MONOTONIC_BLOCK_SHIFT);
TermsEnum iterator = values.termsEnum();
BytesRefBuilder previous = new BytesRefBuilder();
long offset = 0;
@@ -523,10 +525,11 @@ final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Close
meta.writeLong(start);
meta.writeLong(data.getFilePointer() - start);
start = data.getFilePointer();
- addressBuffer.writeTo(data);
+ addressBuffer.copyTo(data);
meta.writeLong(start);
meta.writeLong(data.getFilePointer() - start);
}
+ }
@Override
public void addSortedNumericField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
diff --git a/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java b/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java
index 7a65d71..6820561 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java
@@ -17,12 +17,12 @@
package org.apache.lucene.index;
import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
import java.util.Objects;
-import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.RAMFile;
-import org.apache.lucene.store.RAMInputStream;
-import org.apache.lucene.store.RAMOutputStream;
+import org.apache.lucene.store.ByteBuffersDataInput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
@@ -35,18 +35,19 @@ import org.apache.lucene.util.StringHelper;
* @lucene.internal
*/
public class PrefixCodedTerms implements Accountable {
- final RAMFile buffer;
+ private final List<ByteBuffer> content;
private final long size;
private long delGen;
+ private int lazyHash;
- private PrefixCodedTerms(RAMFile buffer, long size) {
- this.buffer = Objects.requireNonNull(buffer);
+ private PrefixCodedTerms(List<ByteBuffer> content, long size) {
+ this.content = Objects.requireNonNull(content);
this.size = size;
}
@Override
public long ramBytesUsed() {
- return buffer.ramBytesUsed() + 2 * Long.BYTES;
+ return content.stream().mapToLong(buf -> buf.capacity()).sum() + 2 * Long.BYTES;
}
/** Records del gen for this packet. */
@@ -56,8 +57,7 @@ public class PrefixCodedTerms implements Accountable {
/** Builds a PrefixCodedTerms: call add repeatedly, then finish. */
public static class Builder {
- private RAMFile buffer = new RAMFile();
- private RAMOutputStream output = new RAMOutputStream(buffer, false);
+ private ByteBuffersDataOutput output = new ByteBuffersDataOutput();
private Term lastTerm = new Term("");
private BytesRefBuilder lastTermBytes = new BytesRefBuilder();
private long size;
@@ -101,37 +101,28 @@ public class PrefixCodedTerms implements Accountable {
/** return finalized form */
public PrefixCodedTerms finish() {
- try {
- output.close();
- return new PrefixCodedTerms(buffer, size);
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
+ return new PrefixCodedTerms(output.toBufferList(), size);
}
}
/** An iterator over the list of terms stored in a {@link PrefixCodedTerms}. */
public static class TermIterator extends FieldTermIterator {
- final IndexInput input;
+ final ByteBuffersDataInput input;
final BytesRefBuilder builder = new BytesRefBuilder();
final BytesRef bytes = builder.get();
final long end;
final long delGen;
String field = "";
- private TermIterator(long delGen, RAMFile buffer) {
- try {
- input = new RAMInputStream("PrefixCodedTermsIterator", buffer);
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- end = input.length();
+ private TermIterator(long delGen, ByteBuffersDataInput input) {
+ this.input = input;
+ end = input.size();
this.delGen = delGen;
}
@Override
public BytesRef next() {
- if (input.getFilePointer() < end) {
+ if (input.position() < end) {
try {
int code = input.readVInt();
boolean newField = (code & 1) != 0;
@@ -171,7 +162,7 @@ public class PrefixCodedTerms implements Accountable {
/** Return an iterator over the terms stored in this {@link PrefixCodedTerms}. */
public TermIterator iterator() {
- return new TermIterator(delGen, buffer);
+ return new TermIterator(delGen, new ByteBuffersDataInput(content));
}
/** Return the number of terms stored in this {@link PrefixCodedTerms}. */
@@ -181,17 +172,29 @@ public class PrefixCodedTerms implements Accountable {
@Override
public int hashCode() {
- int h = buffer.hashCode();
- h = 31 * h + (int) (delGen ^ (delGen >>> 32));
- return h;
+ if (lazyHash == 0) {
+ int h = 1;
+ for (ByteBuffer bb : content) {
+ h = h + 31 * bb.hashCode();
+ }
+ h = 31 * h + (int) (delGen ^ (delGen >>> 32));
+ lazyHash = h;
+ }
+ return lazyHash;
}
@Override
public boolean equals(Object obj) {
- if (this == obj) return true;
- if (obj == null) return false;
- if (getClass() != obj.getClass()) return false;
+ if (this == obj) {
+ return true;
+ }
+
+ if (obj == null || getClass() != obj.getClass()) {
+ return false;
+ }
+
PrefixCodedTerms other = (PrefixCodedTerms) obj;
- return buffer.equals(other.buffer) && delGen == other.delGen;
+ return delGen == other.delGen &&
+ this.content.equals(other.content);
}
}
diff --git a/lucene/core/src/java/org/apache/lucene/index/SortingLeafReader.java b/lucene/core/src/java/org/apache/lucene/index/SortingLeafReader.java
index d9ed94b..104ef06 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SortingLeafReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SortingLeafReader.java
@@ -24,11 +24,9 @@ import java.util.Map;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.Sort;
-import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMFile;
-import org.apache.lucene.store.RAMInputStream;
-import org.apache.lucene.store.RAMOutputStream;
+import org.apache.lucene.store.ByteBuffersDataInput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BitSet;
import org.apache.lucene.util.Bits;
@@ -856,7 +854,7 @@ class SortingLeafReader extends FilterLeafReader {
private long[] offsets;
private final int upto;
- private final IndexInput postingInput;
+ private final ByteBuffersDataInput postingInput;
private final boolean storeOffsets;
private int docIt = -1;
@@ -866,7 +864,7 @@ class SortingLeafReader extends FilterLeafReader {
private final BytesRef payload;
private int currFreq;
- private final RAMFile file;
+ private final ByteBuffersDataOutput buffer;
SortingPostingsEnum(int maxDoc, SortingPostingsEnum reuse, final PostingsEnum in, Sorter.DocMap docMap, boolean storeOffsets) throws IOException {
super(in);
@@ -876,7 +874,8 @@ class SortingLeafReader extends FilterLeafReader {
docs = reuse.docs;
offsets = reuse.offsets;
payload = reuse.payload;
- file = reuse.file;
+ buffer = reuse.buffer;
+ buffer.reset();
if (reuse.maxDoc == maxDoc) {
sorter = reuse.sorter;
} else {
@@ -886,10 +885,10 @@ class SortingLeafReader extends FilterLeafReader {
docs = new int[32];
offsets = new long[32];
payload = new BytesRef(32);
- file = new RAMFile();
+ buffer = ByteBuffersDataOutput.newResettableInstance();
sorter = new DocOffsetSorter(maxDoc);
}
- final IndexOutput out = new RAMOutputStream(file, false);
+
int doc;
int i = 0;
while ((doc = in.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
@@ -899,15 +898,15 @@ class SortingLeafReader extends FilterLeafReader {
offsets = ArrayUtil.growExact(offsets, newLength);
}
docs[i] = docMap.oldToNew(doc);
- offsets[i] = out.getFilePointer();
- addPositions(in, out);
+ offsets[i] = buffer.size();
+ addPositions(in, buffer);
i++;
}
upto = i;
sorter.reset(docs, offsets);
sorter.sort(0, upto);
- out.close();
- this.postingInput = new RAMInputStream("", file);
+
+ this.postingInput = buffer.toDataInput();
}
// for testing
@@ -918,7 +917,7 @@ class SortingLeafReader extends FilterLeafReader {
return docs == ((SortingPostingsEnum) other).docs;
}
- private void addPositions(final PostingsEnum in, final IndexOutput out) throws IOException {
+ private void addPositions(final PostingsEnum in, final DataOutput out) throws IOException {
int freq = in.freq();
out.writeVInt(freq);
int previousPosition = 0;
diff --git a/lucene/core/src/java/org/apache/lucene/store/GrowableByteArrayDataOutput.java b/lucene/core/src/java/org/apache/lucene/store/GrowableByteArrayDataOutput.java
deleted file mode 100644
index c74d4ef..0000000
--- a/lucene/core/src/java/org/apache/lucene/store/GrowableByteArrayDataOutput.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * 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.lucene.store;
-
-import java.io.IOException;
-
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.UnicodeUtil;
-
-/**
- * A {@link DataOutput} that can be used to build a byte[].
- *
- * @lucene.internal
- */
-public final class GrowableByteArrayDataOutput extends DataOutput {
-
- /** Minimum utf8 byte size of a string over which double pass over string is to save memory during encode */
- static final int MIN_UTF8_SIZE_TO_ENABLE_DOUBLE_PASS_ENCODING = 65536;
-
- /** The bytes */
- private byte[] bytes;
-
- /** The length */
- private int length;
-
- // scratch for utf8 encoding of small strings
- private byte[] scratchBytes;
-
- /** Create a {@link GrowableByteArrayDataOutput} with the given initial capacity. */
- public GrowableByteArrayDataOutput(int cp) {
- this.bytes = new byte[ArrayUtil.oversize(cp, 1)];
- this.length = 0;
- }
-
- @Override
- public void writeByte(byte b) {
- if (length >= bytes.length) {
- bytes = ArrayUtil.grow(bytes);
- }
- bytes[length++] = b;
- }
-
- @Override
- public void writeBytes(byte[] b, int off, int len) {
- final int newLength = length + len;
- if (newLength > bytes.length) {
- bytes = ArrayUtil.grow(bytes, newLength);
- }
- System.arraycopy(b, off, bytes, length, len);
- length = newLength;
- }
-
- @Override
- public void writeString(String string) throws IOException {
- int maxLen = UnicodeUtil.maxUTF8Length(string.length());
- if (maxLen <= MIN_UTF8_SIZE_TO_ENABLE_DOUBLE_PASS_ENCODING) {
- // string is small enough that we don't need to save memory by falling back to double-pass approach
- // this is just an optimized writeString() that re-uses scratchBytes.
- if (scratchBytes == null) {
- scratchBytes = new byte[ArrayUtil.oversize(maxLen, Character.BYTES)];
- } else {
- scratchBytes = ArrayUtil.grow(scratchBytes, maxLen);
- }
- int len = UnicodeUtil.UTF16toUTF8(string, 0, string.length(), scratchBytes);
- writeVInt(len);
- writeBytes(scratchBytes, len);
- } else {
- // use a double pass approach to avoid allocating a large intermediate buffer for string encoding
- int numBytes = UnicodeUtil.calcUTF16toUTF8Length(string, 0, string.length());
- writeVInt(numBytes);
- bytes = ArrayUtil.grow(bytes, length + numBytes);
- length = UnicodeUtil.UTF16toUTF8(string, 0, string.length(), bytes, length);
- }
- }
-
- public byte[] getBytes() {
- return bytes;
- }
-
- public int getPosition() {
- return length;
- }
-
- public void reset() {
- length = 0;
- }
-}
diff --git a/lucene/core/src/java/org/apache/lucene/util/IOUtils.java b/lucene/core/src/java/org/apache/lucene/util/IOUtils.java
index a556789..ba0be42 100644
--- a/lucene/core/src/java/org/apache/lucene/util/IOUtils.java
+++ b/lucene/core/src/java/org/apache/lucene/util/IOUtils.java
@@ -41,6 +41,7 @@ import java.util.LinkedHashMap;
import java.util.Map;
import java.util.Objects;
+import org.apache.lucene.store.ByteBuffersDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FSDirectory;
import org.apache.lucene.store.FileSwitchDirectory;
@@ -486,7 +487,7 @@ public final class IOUtils {
FileSwitchDirectory fsd = (FileSwitchDirectory) dir;
// Spinning is contagious:
return spins(fsd.getPrimaryDir()) || spins(fsd.getSecondaryDir());
- } else if (dir instanceof RAMDirectory) {
+ } else if (dir instanceof RAMDirectory || dir instanceof ByteBuffersDirectory) {
return false;
} else if (dir instanceof FSDirectory) {
return spins(((FSDirectory) dir).getDirectory());
diff --git a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
index 676896f..82c490a 100644
--- a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
@@ -29,13 +29,12 @@ import org.apache.lucene.codecs.MutablePointValues;
import org.apache.lucene.index.MergeState;
import org.apache.lucene.index.PointValues.IntersectVisitor;
import org.apache.lucene.index.PointValues.Relation;
+import org.apache.lucene.store.ByteBuffersDataOutput;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.GrowableByteArrayDataOutput;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.store.TrackingDirectoryWrapper;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
@@ -572,8 +571,8 @@ public class BKDWriter implements Closeable {
return oneDimWriter.finish();
}
- // reused when writing leaf blocks
- private final GrowableByteArrayDataOutput scratchOut = new GrowableByteArrayDataOutput(32*1024);
+ // Reused when writing leaf blocks
+ private final ByteBuffersDataOutput scratchOut = ByteBuffersDataOutput.newResettableInstance();
private class OneDimensionBKDWriter {
@@ -686,7 +685,7 @@ public class BKDWriter implements Closeable {
commonPrefixLengths[0] = prefix;
- assert scratchOut.getPosition() == 0;
+ assert scratchOut.size() == 0;
writeLeafBlockDocs(scratchOut, leafDocs, 0, leafCount);
writeCommonPrefixes(scratchOut, commonPrefixLengths, leafValues);
@@ -704,7 +703,7 @@ public class BKDWriter implements Closeable {
ArrayUtil.copyOfSubArray(leafValues, (leafCount - 1) * packedBytesLength, leafCount * packedBytesLength),
packedValues, leafDocs, 0);
writeLeafBlockPackedValues(scratchOut, commonPrefixLengths, leafCount, 0, packedValues);
- out.writeBytes(scratchOut.getBytes(), 0, scratchOut.getPosition());
+ scratchOut.copyTo(out);
scratchOut.reset();
}
}
@@ -1094,7 +1093,7 @@ public class BKDWriter implements Closeable {
}
/** Reused while packing the index */
- RAMOutputStream writeBuffer = new RAMOutputStream();
+ ByteBuffersDataOutput writeBuffer = ByteBuffersDataOutput.newResettableInstance();
// This is the "file" we append the byte[] to:
List<byte[]> blocks = new ArrayList<>();
@@ -1115,19 +1114,17 @@ public class BKDWriter implements Closeable {
}
/** Appends the current contents of writeBuffer as another block on the growing in-memory file */
- private int appendBlock(RAMOutputStream writeBuffer, List<byte[]> blocks) throws IOException {
- int pos = Math.toIntExact(writeBuffer.getFilePointer());
- byte[] bytes = new byte[pos];
- writeBuffer.writeTo(bytes, 0);
+ private int appendBlock(ByteBuffersDataOutput writeBuffer, List<byte[]> blocks) throws IOException {
+ byte[] block = writeBuffer.toArrayCopy();
+ blocks.add(block);
writeBuffer.reset();
- blocks.add(bytes);
- return pos;
+ return block.length;
}
/**
* lastSplitValues is per-dimension split value previously seen; we use this to prefix-code the split byte[] on each inner node
*/
- private int recursePackIndex(RAMOutputStream writeBuffer, long[] leafBlockFPs, byte[] splitPackedValues, long minBlockFP, List<byte[]> blocks,
+ private int recursePackIndex(ByteBuffersDataOutput writeBuffer, long[] leafBlockFPs, byte[] splitPackedValues, long minBlockFP, List<byte[]> blocks,
int nodeID, byte[] lastSplitValues, boolean[] negativeDeltas, boolean isLeft) throws IOException {
if (nodeID >= leafBlockFPs.length) {
int leafID = nodeID - leafBlockFPs.length;
@@ -1225,9 +1222,8 @@ public class BKDWriter implements Closeable {
} else {
assert leftNumBytes == 0: "leftNumBytes=" + leftNumBytes;
}
- int numBytes2 = Math.toIntExact(writeBuffer.getFilePointer());
- byte[] bytes2 = new byte[numBytes2];
- writeBuffer.writeTo(bytes2, 0);
+
+ byte[] bytes2 = writeBuffer.toArrayCopy();
writeBuffer.reset();
// replace our placeholder:
blocks.set(idxSav, bytes2);
@@ -1242,7 +1238,7 @@ public class BKDWriter implements Closeable {
assert Arrays.equals(lastSplitValues, cmp);
- return numBytes + numBytes2 + leftNumBytes + rightNumBytes;
+ return numBytes + bytes2.length + leftNumBytes + rightNumBytes;
}
}
@@ -1605,7 +1601,7 @@ public class BKDWriter implements Closeable {
// Save the block file pointer:
leafBlockFPs[nodeID - leafNodeOffset] = out.getFilePointer();
- assert scratchOut.getPosition() == 0;
+ assert scratchOut.size() == 0;
// Write doc IDs
int[] docIDs = spareDocIds;
@@ -1631,10 +1627,8 @@ public class BKDWriter implements Closeable {
assert valuesInOrderAndBounds(count, sortedDim, minPackedValue, maxPackedValue, packedValues,
docIDs, 0);
writeLeafBlockPackedValues(scratchOut, commonPrefixLengths, count, sortedDim, packedValues);
-
- out.writeBytes(scratchOut.getBytes(), 0, scratchOut.getPosition());
+ scratchOut.copyTo(out);
scratchOut.reset();
-
} else {
// inner node
diff --git a/lucene/core/src/java/org/apache/lucene/util/fst/FST.java b/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
index 92973f6..8e84e3c 100644
--- a/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
+++ b/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
@@ -27,11 +27,11 @@ import java.nio.file.Path;
import org.apache.lucene.codecs.CodecUtil;
import org.apache.lucene.store.ByteArrayDataOutput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
import org.apache.lucene.store.DataInput;
import org.apache.lucene.store.DataOutput;
import org.apache.lucene.store.InputStreamDataInput;
import org.apache.lucene.store.OutputStreamDataOutput;
-import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.Accountable;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.Constants;
@@ -432,16 +432,14 @@ public final class FST<T> implements Accountable {
out.writeByte((byte) 1);
// Serialize empty-string output:
- RAMOutputStream ros = new RAMOutputStream();
+ ByteBuffersDataOutput ros = new ByteBuffersDataOutput();
outputs.writeFinalOutput(emptyOutput, ros);
-
- byte[] emptyOutputBytes = new byte[(int) ros.getFilePointer()];
- ros.writeTo(emptyOutputBytes, 0);
+ byte[] emptyOutputBytes = ros.toArrayCopy();
// reverse
final int stopAt = emptyOutputBytes.length/2;
int upto = 0;
- while(upto < stopAt) {
+ while (upto < stopAt) {
final byte b = emptyOutputBytes[upto];
emptyOutputBytes[upto] = emptyOutputBytes[emptyOutputBytes.length-upto-1];
emptyOutputBytes[emptyOutputBytes.length-upto-1] = b;
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/TestCodecUtil.java b/lucene/core/src/test/org/apache/lucene/codecs/TestCodecUtil.java
index a15475b..0a11a9b 100644
--- a/lucene/core/src/test/org/apache/lucene/codecs/TestCodecUtil.java
+++ b/lucene/core/src/test/org/apache/lucene/codecs/TestCodecUtil.java
@@ -22,12 +22,12 @@ import java.util.concurrent.atomic.AtomicLong;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.store.BufferedChecksumIndexInput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.ByteBuffersIndexInput;
+import org.apache.lucene.store.ByteBuffersIndexOutput;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMFile;
-import org.apache.lucene.store.RAMInputStream;
-import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.StringHelper;
@@ -35,13 +35,13 @@ import org.apache.lucene.util.StringHelper;
public class TestCodecUtil extends LuceneTestCase {
public void testHeaderLength() throws Exception {
- RAMFile file = new RAMFile();
- IndexOutput output = new RAMOutputStream(file, true);
+ ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+ IndexOutput output = new ByteBuffersIndexOutput(out, "temp", "temp");
CodecUtil.writeHeader(output, "FooBar", 5);
output.writeString("this is the data");
output.close();
- IndexInput input = new RAMInputStream("file", file);
+ IndexInput input = new ByteBuffersIndexInput(out.toDataInput(), "temp");
input.seek(CodecUtil.headerLength("FooBar"));
assertEquals("this is the data", input.readString());
input.close();
@@ -52,55 +52,55 @@ public class TestCodecUtil extends LuceneTestCase {
for (int i = 0; i < 128; i++) {
tooLong.append('a');
}
- RAMFile file = new RAMFile();
- IndexOutput output = new RAMOutputStream(file, true);
+ ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+ IndexOutput output = new ByteBuffersIndexOutput(out, "temp", "temp");
expectThrows(IllegalArgumentException.class, () -> {
CodecUtil.writeHeader(output, tooLong.toString(), 5);
});
}
public void testWriteNonAsciiHeader() throws Exception {
- RAMFile file = new RAMFile();
- IndexOutput output = new RAMOutputStream(file, true);
+ ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+ IndexOutput output = new ByteBuffersIndexOutput(out, "temp", "temp");
expectThrows(IllegalArgumentException.class, () -> {
CodecUtil.writeHeader(output, "\u1234", 5);
});
}
public void testReadHeaderWrongMagic() throws Exception {
- RAMFile file = new RAMFile();
- IndexOutput output = new RAMOutputStream(file, true);
+ ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+ IndexOutput output = new ByteBuffersIndexOutput(out, "temp", "temp");
output.writeInt(1234);
output.close();
- IndexInput input = new RAMInputStream("file", file);
+ IndexInput input = new ByteBuffersIndexInput(out.toDataInput(), "temp");
expectThrows(CorruptIndexException.class, () -> {
CodecUtil.checkHeader(input, "bogus", 1, 1);
});
}
public void testChecksumEntireFile() throws Exception {
- RAMFile file = new RAMFile();
- IndexOutput output = new RAMOutputStream(file, true);
+ ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+ IndexOutput output = new ByteBuffersIndexOutput(out, "temp", "temp");
CodecUtil.writeHeader(output, "FooBar", 5);
output.writeString("this is the data");
CodecUtil.writeFooter(output);
output.close();
- IndexInput input = new RAMInputStream("file", file);
+ IndexInput input = new ByteBuffersIndexInput(out.toDataInput(), "temp");
CodecUtil.checksumEntireFile(input);
input.close();
}
public void testCheckFooterValid() throws Exception {
- RAMFile file = new RAMFile();
- IndexOutput output = new RAMOutputStream(file, true);
+ ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+ IndexOutput output = new ByteBuffersIndexOutput(out, "temp", "temp");
CodecUtil.writeHeader(output, "FooBar", 5);
output.writeString("this is the data");
CodecUtil.writeFooter(output);
output.close();
- ChecksumIndexInput input = new BufferedChecksumIndexInput(new RAMInputStream("file", file));
+ ChecksumIndexInput input = new BufferedChecksumIndexInput(new ByteBuffersIndexInput(out.toDataInput(), "temp"));
Exception mine = new RuntimeException("fake exception");
RuntimeException expected = expectThrows(RuntimeException.class, () -> {
CodecUtil.checkFooter(input, mine);
@@ -113,14 +113,14 @@ public class TestCodecUtil extends LuceneTestCase {
}
public void testCheckFooterValidAtFooter() throws Exception {
- RAMFile file = new RAMFile();
- IndexOutput output = new RAMOutputStream(file, true);
+ ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+ IndexOutput output = new ByteBuffersIndexOutput(out, "temp", "temp");
CodecUtil.writeHeader(output, "FooBar", 5);
output.writeString("this is the data");
CodecUtil.writeFooter(output);
output.close();
- ChecksumIndexInput input = new BufferedChecksumIndexInput(new RAMInputStream("file", file));
+ ChecksumIndexInput input = new BufferedChecksumIndexInput(new ByteBuffersIndexInput(out.toDataInput(), "temp"));
CodecUtil.checkHeader(input, "FooBar", 5, 5);
assertEquals("this is the data", input.readString());
Exception mine = new RuntimeException("fake exception");
@@ -135,14 +135,14 @@ public class TestCodecUtil extends LuceneTestCase {
}
public void testCheckFooterValidPastFooter() throws Exception {
- RAMFile file = new RAMFile();
- IndexOutput output = new RAMOutputStream(file, true);
+ ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+ IndexOutput output = new ByteBuffersIndexOutput(out, "temp", "temp");
CodecUtil.writeHeader(output, "FooBar", 5);
output.writeString("this is the data");
CodecUtil.writeFooter(output);
output.close();
- ChecksumIndexInput input = new BufferedChecksumIndexInput(new RAMInputStream("file", file));
+ ChecksumIndexInput input = new BufferedChecksumIndexInput(new ByteBuffersIndexInput(out.toDataInput(), "temp"));
CodecUtil.checkHeader(input, "FooBar", 5, 5);
assertEquals("this is the data", input.readString());
// bogusly read a byte too far (can happen)
@@ -159,16 +159,16 @@ public class TestCodecUtil extends LuceneTestCase {
}
public void testCheckFooterInvalid() throws Exception {
- RAMFile file = new RAMFile();
- IndexOutput output = new RAMOutputStream(file, true);
+ ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+ IndexOutput output = new ByteBuffersIndexOutput(out, "temp", "temp");
CodecUtil.writeHeader(output, "FooBar", 5);
output.writeString("this is the data");
output.writeInt(CodecUtil.FOOTER_MAGIC);
output.writeInt(0);
output.writeLong(1234567); // write a bogus checksum
output.close();
-
- ChecksumIndexInput input = new BufferedChecksumIndexInput(new RAMInputStream("file", file));
+
+ ChecksumIndexInput input = new BufferedChecksumIndexInput(new ByteBuffersIndexInput(out.toDataInput(), "temp"));
CodecUtil.checkHeader(input, "FooBar", 5, 5);
assertEquals("this is the data", input.readString());
Exception mine = new RuntimeException("fake exception");
@@ -183,13 +183,13 @@ public class TestCodecUtil extends LuceneTestCase {
}
public void testSegmentHeaderLength() throws Exception {
- RAMFile file = new RAMFile();
- IndexOutput output = new RAMOutputStream(file, true);
+ ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+ IndexOutput output = new ByteBuffersIndexOutput(out, "temp", "temp");
CodecUtil.writeIndexHeader(output, "FooBar", 5, StringHelper.randomId(), "xyz");
output.writeString("this is the data");
output.close();
- IndexInput input = new RAMInputStream("file", file);
+ IndexInput input = new ByteBuffersIndexInput(out.toDataInput(), "temp");
input.seek(CodecUtil.indexHeaderLength("FooBar", "xyz"));
assertEquals("this is the data", input.readString());
input.close();
@@ -200,8 +200,8 @@ public class TestCodecUtil extends LuceneTestCase {
for (int i = 0; i < 256; i++) {
tooLong.append('a');
}
- RAMFile file = new RAMFile();
- IndexOutput output = new RAMOutputStream(file, true);
+ ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+ IndexOutput output = new ByteBuffersIndexOutput(out, "temp", "temp");
expectThrows(IllegalArgumentException.class, () -> {
CodecUtil.writeIndexHeader(output, "foobar", 5, StringHelper.randomId(), tooLong.toString());
});
@@ -212,13 +212,13 @@ public class TestCodecUtil extends LuceneTestCase {
for (int i = 0; i < 255; i++) {
justLongEnough.append('a');
}
- RAMFile file = new RAMFile();
- IndexOutput output = new RAMOutputStream(file, true);
+ ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+ IndexOutput output = new ByteBuffersIndexOutput(out, "temp", "temp");
byte[] id = StringHelper.randomId();
CodecUtil.writeIndexHeader(output, "foobar", 5, id, justLongEnough.toString());
output.close();
- IndexInput input = new RAMInputStream("file", file);
+ IndexInput input = new ByteBuffersIndexInput(out.toDataInput(), "temp");
CodecUtil.checkIndexHeader(input, "foobar", 5, 5, id, justLongEnough.toString());
assertEquals(input.getFilePointer(), input.length());
assertEquals(input.getFilePointer(), CodecUtil.indexHeaderLength("foobar", justLongEnough.toString()));
@@ -226,22 +226,22 @@ public class TestCodecUtil extends LuceneTestCase {
}
public void testWriteNonAsciiSuffix() throws Exception {
- RAMFile file = new RAMFile();
- IndexOutput output = new RAMOutputStream(file, true);
+ ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+ IndexOutput output = new ByteBuffersIndexOutput(out, "temp", "temp");
expectThrows(IllegalArgumentException.class, () -> {
CodecUtil.writeIndexHeader(output, "foobar", 5, StringHelper.randomId(), "\u1234");
});
}
public void testReadBogusCRC() throws Exception {
- RAMFile file = new RAMFile();
- IndexOutput output = new RAMOutputStream(file, false);
+ ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+ IndexOutput output = new ByteBuffersIndexOutput(out, "temp", "temp");
output.writeLong(-1L); // bad
output.writeLong(1L << 32); // bad
output.writeLong(-(1L << 32)); // bad
output.writeLong((1L << 32) - 1); // ok
output.close();
- IndexInput input = new RAMInputStream("file", file);
+ IndexInput input = new BufferedChecksumIndexInput(new ByteBuffersIndexInput(out.toDataInput(), "temp"));
// read 3 bogus values
for (int i = 0; i < 3; i++) {
expectThrows(CorruptIndexException.class, () -> {
@@ -253,9 +253,10 @@ public class TestCodecUtil extends LuceneTestCase {
}
public void testWriteBogusCRC() throws Exception {
- RAMFile file = new RAMFile();
- final IndexOutput output = new RAMOutputStream(file, false);
+ ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+ IndexOutput output = new ByteBuffersIndexOutput(out, "temp", "temp");
AtomicLong fakeChecksum = new AtomicLong();
+
// wrap the index input where we control the checksum for mocking
IndexOutput fakeOutput = new IndexOutput("fake", "fake") {
@Override
@@ -304,15 +305,18 @@ public class TestCodecUtil extends LuceneTestCase {
}
public void testTruncatedFileThrowsCorruptIndexException() throws IOException {
- RAMFile file = new RAMFile();
- IndexOutput output = new RAMOutputStream(file, false);
+ ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+ IndexOutput output = new ByteBuffersIndexOutput(out, "temp", "temp");
output.close();
- IndexInput input = new RAMInputStream("file", file);
+
+ IndexInput input = new ByteBuffersIndexInput(out.toDataInput(), "temp");
+
CorruptIndexException e = expectThrows(CorruptIndexException.class,
() -> CodecUtil.checksumEntireFile(input));
- assertEquals("misplaced codec footer (file truncated?): length=0 but footerLength==16 (resource=RAMInputStream(name=file))", e.getMessage());
+ assertTrue(e.getMessage(), e.getMessage().contains("misplaced codec footer (file truncated?): length=0 but footerLength==16 (resource"));
+
e = expectThrows(CorruptIndexException.class,
() -> CodecUtil.retrieveChecksum(input));
- assertEquals("misplaced codec footer (file truncated?): length=0 but footerLength==16 (resource=RAMInputStream(name=file))", e.getMessage());
+ assertTrue(e.getMessage(), e.getMessage().contains("misplaced codec footer (file truncated?): length=0 but footerLength==16 (resource"));
}
}
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70DocValuesFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70DocValuesFormat.java
index 1355a9d..9ed3173 100644
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70DocValuesFormat.java
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70DocValuesFormat.java
@@ -60,13 +60,12 @@ import org.apache.lucene.index.SortedNumericDocValues;
import org.apache.lucene.index.SortedSetDocValues;
import org.apache.lucene.index.Term;
import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.index.TermsEnum.SeekStatus;
import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.ByteBuffersDataInput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.RAMFile;
-import org.apache.lucene.store.RAMInputStream;
-import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import org.apache.lucene.util.TestUtil;
@@ -446,8 +445,7 @@ public class TestLucene70DocValuesFormat extends BaseCompressingDocValuesFormatT
for (int maxDoc = frontier - 1; maxDoc <= frontier + 1; ++maxDoc) {
final Directory dir = newDirectory();
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig().setMergePolicy(newLogMergePolicy()));
- RAMFile buffer = new RAMFile();
- RAMOutputStream out = new RAMOutputStream(buffer, false);
+ ByteBuffersDataOutput buffer = new ByteBuffersDataOutput();
Document doc = new Document();
SortedSetDocValuesField field1 = new SortedSetDocValuesField("sset", new BytesRef());
doc.add(field1);
@@ -460,13 +458,12 @@ public class TestLucene70DocValuesFormat extends BaseCompressingDocValuesFormatT
field2.setBytesValue(s2);
w.addDocument(doc);
Set<BytesRef> set = new TreeSet<>(Arrays.asList(s1, s2));
- out.writeVInt(set.size());
+ buffer.writeVInt(set.size());
for (BytesRef ref : set) {
- out.writeVInt(ref.length);
- out.writeBytes(ref.bytes, ref.offset, ref.length);
+ buffer.writeVInt(ref.length);
+ buffer.writeBytes(ref.bytes, ref.offset, ref.length);
}
}
- out.close();
w.forceMerge(1);
DirectoryReader r = DirectoryReader.open(w);
w.close();
@@ -474,7 +471,7 @@ public class TestLucene70DocValuesFormat extends BaseCompressingDocValuesFormatT
assertEquals(maxDoc, sr.maxDoc());
SortedSetDocValues values = sr.getSortedSetDocValues("sset");
assertNotNull(values);
- RAMInputStream in = new RAMInputStream("", buffer);
+ ByteBuffersDataInput in = buffer.toDataInput();
BytesRefBuilder b = new BytesRefBuilder();
for (int i = 0; i < maxDoc; ++i) {
assertEquals(i, values.nextDoc());
@@ -500,8 +497,8 @@ public class TestLucene70DocValuesFormat extends BaseCompressingDocValuesFormatT
for (int maxDoc = frontier - 1; maxDoc <= frontier + 1; ++maxDoc) {
final Directory dir = newDirectory();
IndexWriter w = new IndexWriter(dir, newIndexWriterConfig().setMergePolicy(newLogMergePolicy()));
- RAMFile buffer = new RAMFile();
- RAMOutputStream out = new RAMOutputStream(buffer, false);
+ ByteBuffersDataOutput buffer = new ByteBuffersDataOutput();
+
Document doc = new Document();
SortedNumericDocValuesField field1 = new SortedNumericDocValuesField("snum", 0L);
doc.add(field1);
@@ -513,10 +510,10 @@ public class TestLucene70DocValuesFormat extends BaseCompressingDocValuesFormatT
field1.setLongValue(s1);
field2.setLongValue(s2);
w.addDocument(doc);
- out.writeVLong(Math.min(s1, s2));
- out.writeVLong(Math.max(s1, s2));
+ buffer.writeVLong(Math.min(s1, s2));
+ buffer.writeVLong(Math.max(s1, s2));
}
- out.close();
+
w.forceMerge(1);
DirectoryReader r = DirectoryReader.open(w);
w.close();
@@ -524,12 +521,12 @@ public class TestLucene70DocValuesFormat extends BaseCompressingDocValuesFormatT
assertEquals(maxDoc, sr.maxDoc());
SortedNumericDocValues values = sr.getSortedNumericDocValues("snum");
assertNotNull(values);
- RAMInputStream in = new RAMInputStream("", buffer);
+ ByteBuffersDataInput dataInput = buffer.toDataInput();
for (int i = 0; i < maxDoc; ++i) {
assertEquals(i, values.nextDoc());
assertEquals(2, values.docValueCount());
- assertEquals(in.readVLong(), values.nextValue());
- assertEquals(in.readVLong(), values.nextValue());
+ assertEquals(dataInput.readVLong(), values.nextValue());
+ assertEquals(dataInput.readVLong(), values.nextValue());
}
r.close();
dir.close();
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java
index 3629721..a00e01b 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java
@@ -547,7 +547,7 @@ public class TestIndexWriterOnDiskFull extends LuceneTestCase {
.setMergeScheduler(new ConcurrentMergeScheduler())
.setCommitOnClose(false));
writer.commit(); // empty commit, to not create confusing situation with first commit
- dir.setMaxSizeInBytes(Math.max(1, dir.getRecomputedActualSizeInBytes()));
+ dir.setMaxSizeInBytes(Math.max(1, dir.sizeInBytes()));
final Document doc = new Document();
FieldType customType = new FieldType(TextField.TYPE_STORED);
doc.add(newField("field", "aaa bbb ccc ddd eee fff ggg hhh iii jjj", customType));
diff --git a/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java b/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java
index 9633ea4..caabfef 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java
@@ -175,7 +175,7 @@ public class TermInSetQueryTest extends LuceneTestCase {
public void testRamBytesUsed() {
List<BytesRef> terms = new ArrayList<>();
- final int numTerms = 1000 + random().nextInt(1000);
+ final int numTerms = 10000 + random().nextInt(1000);
for (int i = 0; i < numTerms; ++i) {
terms.add(new BytesRef(RandomStrings.randomUnicodeOfLength(random(), 10)));
}
@@ -183,7 +183,7 @@ public class TermInSetQueryTest extends LuceneTestCase {
final long actualRamBytesUsed = RamUsageTester.sizeOf(query);
final long expectedRamBytesUsed = query.ramBytesUsed();
// error margin within 5%
- assertEquals(actualRamBytesUsed, expectedRamBytesUsed, actualRamBytesUsed / 20);
+ assertEquals(expectedRamBytesUsed, actualRamBytesUsed, actualRamBytesUsed / 20);
}
private static class TermsCountingDirectoryReaderWrapper extends FilterDirectoryReader {
diff --git a/lucene/core/src/test/org/apache/lucene/store/BaseDataOutputTestCase.java b/lucene/core/src/test/org/apache/lucene/store/BaseDataOutputTestCase.java
index 4578a4f..d77e133 100644
--- a/lucene/core/src/test/org/apache/lucene/store/BaseDataOutputTestCase.java
+++ b/lucene/core/src/test/org/apache/lucene/store/BaseDataOutputTestCase.java
@@ -24,7 +24,6 @@ import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.List;
import java.util.Random;
-
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.IOUtils.IOConsumer;
import org.junit.Test;
diff --git a/lucene/core/src/test/org/apache/lucene/store/TestByteBuffersDataInput.java b/lucene/core/src/test/org/apache/lucene/store/TestByteBuffersDataInput.java
index 20e3bd2..3cd76fe 100644
--- a/lucene/core/src/test/org/apache/lucene/store/TestByteBuffersDataInput.java
+++ b/lucene/core/src/test/org/apache/lucene/store/TestByteBuffersDataInput.java
@@ -21,8 +21,8 @@ import static org.junit.Assert.*;
import java.io.EOFException;
import java.io.IOException;
import java.nio.ByteBuffer;
-import java.util.ArrayList;
import java.util.List;
+import java.util.ArrayList;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.IOUtils.IOConsumer;
@@ -157,6 +157,7 @@ public final class TestByteBuffersDataInput extends RandomizedTest {
byte [] array = dst.toArrayCopy();
array = ArrayUtil.copyOfSubArray(array, prefix.length, array.length);
+
for (int i = 0; i < 1000; i++) {
int offs = randomIntBetween(0, array.length - 1);
in.seek(offs);
diff --git a/lucene/core/src/test/org/apache/lucene/store/TestByteBuffersDataOutput.java b/lucene/core/src/test/org/apache/lucene/store/TestByteBuffersDataOutput.java
index 893aa37..42e10b0 100644
--- a/lucene/core/src/test/org/apache/lucene/store/TestByteBuffersDataOutput.java
+++ b/lucene/core/src/test/org/apache/lucene/store/TestByteBuffersDataOutput.java
@@ -22,7 +22,6 @@ import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.Random;
import java.util.concurrent.atomic.AtomicInteger;
-
import org.apache.lucene.util.ArrayUtil;
import org.junit.Assert;
import org.junit.Test;
diff --git a/lucene/core/src/test/org/apache/lucene/store/TestGrowableByteArrayDataOutput.java b/lucene/core/src/test/org/apache/lucene/store/TestGrowableByteArrayDataOutput.java
deleted file mode 100644
index 10992b7..0000000
--- a/lucene/core/src/test/org/apache/lucene/store/TestGrowableByteArrayDataOutput.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * 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.lucene.store;
-
-import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.TestUtil;
-import org.apache.lucene.util.UnicodeUtil;
-import org.junit.Test;
-
-/**
- * Test for {@link GrowableByteArrayDataOutput}
- */
-public class TestGrowableByteArrayDataOutput extends LuceneTestCase {
-
- @Test
- public void testWriteSmallStrings() throws Exception {
- int minSizeForDoublePass = GrowableByteArrayDataOutput.MIN_UTF8_SIZE_TO_ENABLE_DOUBLE_PASS_ENCODING;
-
- // a simple string encoding test
- int num = atLeast(1000);
- for (int i = 0; i < num; i++) {
- // create a small string such that the single pass approach is used
- int length = TestUtil.nextInt(random(), 1, minSizeForDoublePass - 1);
- String unicode = TestUtil.randomFixedByteLengthUnicodeString(random(), length);
- byte[] utf8 = new byte[UnicodeUtil.maxUTF8Length(unicode.length())];
- int len = UnicodeUtil.UTF16toUTF8(unicode, 0, unicode.length(), utf8);
-
- GrowableByteArrayDataOutput dataOutput = new GrowableByteArrayDataOutput(1 << 8);
- //explicitly write utf8 len so that we know how many bytes it occupies
- dataOutput.writeVInt(len);
- int vintLen = dataOutput.getPosition();
- // now write the string which will internally write number of bytes as a vint and then utf8 bytes
- dataOutput.writeString(unicode);
-
- assertEquals("GrowableByteArrayDataOutput wrote the wrong length after encode", len + vintLen * 2, dataOutput.getPosition());
- for (int j = 0, k = vintLen * 2; j < len; j++, k++) {
- assertEquals(utf8[j], dataOutput.getBytes()[k]);
- }
- }
- }
-
- @Test
- public void testWriteLargeStrings() throws Exception {
- int minSizeForDoublePass = GrowableByteArrayDataOutput.MIN_UTF8_SIZE_TO_ENABLE_DOUBLE_PASS_ENCODING;
-
- int num = atLeast(100);
- for (int i = 0; i < num; i++) {
- String unicode = TestUtil.randomRealisticUnicodeString(random(), minSizeForDoublePass, 10 * minSizeForDoublePass);
- byte[] utf8 = new byte[UnicodeUtil.maxUTF8Length(unicode.length())];
- int len = UnicodeUtil.UTF16toUTF8(unicode, 0, unicode.length(), utf8);
-
- GrowableByteArrayDataOutput dataOutput = new GrowableByteArrayDataOutput(1 << 8);
- //explicitly write utf8 len so that we know how many bytes it occupies
- dataOutput.writeVInt(len);
- int vintLen = dataOutput.getPosition();
- // now write the string which will internally write number of bytes as a vint and then utf8 bytes
- dataOutput.writeString(unicode);
-
- assertEquals("GrowableByteArrayDataOutput wrote the wrong length after encode", len + vintLen * 2, dataOutput.getPosition());
- for (int j = 0, k = vintLen * 2; j < len; j++, k++) {
- assertEquals(utf8[j], dataOutput.getBytes()[k]);
- }
- }
- }
-}
diff --git a/lucene/core/src/test/org/apache/lucene/store/TestHugeRamFile.java b/lucene/core/src/test/org/apache/lucene/store/TestHugeRamFile.java
deleted file mode 100644
index 04c5900..0000000
--- a/lucene/core/src/test/org/apache/lucene/store/TestHugeRamFile.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * 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.lucene.store;
-
-
-import java.io.IOException;
-import java.util.HashMap;
-
-import org.apache.lucene.util.LuceneTestCase;
-
-/** Test huge RAMFile with more than Integer.MAX_VALUE bytes. */
-public class TestHugeRamFile extends LuceneTestCase {
-
- private static final long MAX_VALUE = (long) 2 * (long) Integer.MAX_VALUE;
-
- /** Fake a huge ram file by using the same byte buffer for all
- * buffers under maxint. */
- private static class DenseRAMFile extends RAMFile {
- private long capacity = 0;
- private HashMap<Integer,byte[]> singleBuffers = new HashMap<>();
- @Override
- protected byte[] newBuffer(int size) {
- capacity += size;
- if (capacity <= MAX_VALUE) {
- // below maxint we reuse buffers
- byte buf[] = singleBuffers.get(Integer.valueOf(size));
- if (buf==null) {
- buf = new byte[size];
- //System.out.println("allocate: "+size);
- singleBuffers.put(Integer.valueOf(size),buf);
- }
- return buf;
- }
- //System.out.println("allocate: "+size); System.out.flush();
- return new byte[size];
- }
- }
-
- /** Test huge RAMFile with more than Integer.MAX_VALUE bytes. (LUCENE-957) */
- public void testHugeFile() throws IOException {
- DenseRAMFile f = new DenseRAMFile();
- // output part
- RAMOutputStream out = new RAMOutputStream(f, true);
- byte b1[] = new byte[RAMOutputStream.BUFFER_SIZE];
- byte b2[] = new byte[RAMOutputStream.BUFFER_SIZE / 3];
- for (int i = 0; i < b1.length; i++) {
- b1[i] = (byte) (i & 0x0007F);
- }
- for (int i = 0; i < b2.length; i++) {
- b2[i] = (byte) (i & 0x0003F);
- }
- long n = 0;
- assertEquals("output length must match",n,out.getFilePointer());
- while (n <= MAX_VALUE - b1.length) {
- out.writeBytes(b1,0,b1.length);
- out.flush();
- n += b1.length;
- assertEquals("output length must match",n,out.getFilePointer());
- }
- //System.out.println("after writing b1's, length = "+out.length()+" (MAX_VALUE="+MAX_VALUE+")");
- int m = b2.length;
- long L = 12;
- for (int j=0; j<L; j++) {
- for (int i = 0; i < b2.length; i++) {
- b2[i]++;
- }
- out.writeBytes(b2,0,m);
- out.flush();
- n += m;
- assertEquals("output length must match",n,out.getFilePointer());
- }
- out.close();
- // input part
- RAMInputStream in = new RAMInputStream("testcase", f);
- assertEquals("input length must match",n,in.length());
- //System.out.println("input length = "+in.length()+" % 1024 = "+in.length()%1024);
- for (int j=0; j<L; j++) {
- long loc = n - (L-j)*m;
- in.seek(loc/3);
- in.seek(loc);
- for (int i=0; i<m; i++) {
- byte bt = in.readByte();
- byte expected = (byte) (1 + j + (i & 0x0003F));
- assertEquals("must read same value that was written! j="+j+" i="+i,expected,bt);
- }
- }
- }
-}
diff --git a/lucene/core/src/test/org/apache/lucene/store/TestRAMDirectory.java b/lucene/core/src/test/org/apache/lucene/store/TestRAMDirectory.java
deleted file mode 100644
index 02ea9d7..0000000
--- a/lucene/core/src/test/org/apache/lucene/store/TestRAMDirectory.java
+++ /dev/null
@@ -1,189 +0,0 @@
-/*
- * 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.lucene.store;
-
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Random;
-
-import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.IndexWriterConfig.OpenMode;
-import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.util.English;
-import org.apache.lucene.util.IOUtils;
-
-/**
- * JUnit testcase to test RAMDirectory. RAMDirectory itself is used in many testcases,
- * but not one of them uses an different constructor other than the default constructor.
- */
-public class TestRAMDirectory extends BaseDirectoryTestCase {
-
- @Override
- protected Directory getDirectory(Path path) {
- return new RAMDirectory();
- }
-
- // add enough document so that the index will be larger than RAMDirectory.READ_BUFFER_SIZE
- private static final int DOCS_TO_ADD = 500;
-
- private Path buildIndex() throws IOException {
- Path path = createTempDir("buildIndex");
-
- Directory dir = newFSDirectory(path);
- IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig(
- new MockAnalyzer(random())).setOpenMode(OpenMode.CREATE));
- // add some documents
- Document doc = null;
- for (int i = 0; i < DOCS_TO_ADD; i++) {
- doc = new Document();
- doc.add(newStringField("content", English.intToEnglish(i).trim(), Field.Store.YES));
- writer.addDocument(doc);
- }
- assertEquals(DOCS_TO_ADD, writer.getDocStats().maxDoc);
- writer.close();
- dir.close();
-
- return path;
- }
-
- // LUCENE-1468
- public void testCopySubdir() throws Throwable {
- Path path = createTempDir("testsubdir");
- FSDirectory fsDir = null;
- try {
- Files.createDirectory(path.resolve("subdir"));
- fsDir = new SimpleFSDirectory(path);
- RAMDirectory ramDir = new RAMDirectory(fsDir, newIOContext(random()));
- List<String> files = Arrays.asList(ramDir.listAll());
- assertFalse(files.contains("subdir"));
- } finally {
- IOUtils.close(fsDir);
- }
- }
-
- public void testRAMDirectory () throws IOException {
- Path indexDir = buildIndex();
-
- FSDirectory dir = new SimpleFSDirectory(indexDir);
- MockDirectoryWrapper ramDir = new MockDirectoryWrapper(random(), new RAMDirectory(dir, newIOContext(random())));
-
- // close the underlaying directory
- dir.close();
-
- // Check size
- assertEquals(ramDir.sizeInBytes(), ramDir.getRecomputedSizeInBytes());
-
- // open reader to test document count
- IndexReader reader = DirectoryReader.open(ramDir);
- assertEquals(DOCS_TO_ADD, reader.numDocs());
-
- // open search zo check if all doc's are there
- IndexSearcher searcher = newSearcher(reader);
-
- // search for all documents
- for (int i = 0; i < DOCS_TO_ADD; i++) {
- Document doc = searcher.doc(i);
- assertTrue(doc.getField("content") != null);
- }
-
- // cleanup
- reader.close();
- }
-
- private static final int NUM_THREADS = 10;
- private static final int DOCS_PER_THREAD = 40;
-
- public void testRAMDirectorySize() throws IOException, InterruptedException {
-
- Path indexDir = buildIndex();
-
- FSDirectory dir = new SimpleFSDirectory(indexDir);
- final MockDirectoryWrapper ramDir = new MockDirectoryWrapper(random(), new RAMDirectory(dir, newIOContext(random())));
- dir.close();
-
- final IndexWriter writer = new IndexWriter(ramDir, new IndexWriterConfig(
- new MockAnalyzer(random())).setOpenMode(OpenMode.APPEND));
- writer.forceMerge(1);
-
- assertEquals(ramDir.sizeInBytes(), ramDir.getRecomputedSizeInBytes());
-
- Thread[] threads = new Thread[NUM_THREADS];
- for (int i = 0; i< NUM_THREADS; i++) {
- final int num = i;
- threads[i] = new Thread(){
- @Override
- public void run() {
- for (int j = 1; j< DOCS_PER_THREAD; j++) {
- Document doc = new Document();
- doc.add(newStringField("sizeContent", English.intToEnglish(num* DOCS_PER_THREAD +j).trim(), Field.Store.YES));
- try {
- writer.addDocument(doc);
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- }
- }
- };
- }
- for (int i = 0; i< NUM_THREADS; i++) {
- threads[i].start();
- }
- for (int i = 0; i< NUM_THREADS; i++) {
- threads[i].join();
- }
-
- writer.forceMerge(1);
- assertEquals(ramDir.sizeInBytes(), ramDir.getRecomputedSizeInBytes());
-
- writer.close();
- }
-
- public void testShouldThrowEOFException() throws Exception {
- final Random random = random();
-
- try (Directory dir = newDirectory()) {
- final int len = 16 + random().nextInt(2048) / 16 * 16;
- final byte[] bytes = new byte[len];
-
- try (IndexOutput os = dir.createOutput("foo", newIOContext(random))) {
- os.writeBytes(bytes, bytes.length);
- }
-
- try (IndexInput is = dir.openInput("foo", newIOContext(random))) {
- expectThrows(EOFException.class, () -> {
- is.seek(0);
- // Here, I go past EOF.
- is.seek(len + random().nextInt(2048));
- // since EOF is not enforced by the previous call in RAMInputStream
- // this call to readBytes should throw the exception.
- is.readBytes(bytes, 0, 16);
- });
- }
- }
- }
-}
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java
index 2d3e4d6..6b90c78 100644
--- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java
@@ -34,21 +34,22 @@ import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.SearcherFactory;
import org.apache.lucene.search.SearcherManager;
import org.apache.lucene.store.AlreadyClosedException;
-import org.apache.lucene.store.RAMFile;
-import org.apache.lucene.store.RAMOutputStream;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.ByteBuffersIndexOutput;
import org.apache.lucene.util.ThreadInterruptedException;
/*
* This just asks IndexWriter to open new NRT reader, in order to publish a new NRT point. This could be improved, if we separated out 1)
* nrt flush (and incRef the SIS) from 2) opening a new reader, but this is tricky with IW's concurrency, and it would also be hard-ish to share
- * IW's reader pool with our searcher manager. So we do the simpler solution now, but that adds some unecessary latency to NRT refresh on
+ * IW's reader pool with our searcher manager. So we do the simpler solution now, but that adds some unnecessary latency to NRT refresh on
* replicas since step 2) could otherwise be done concurrently with replicas copying files over.
*/
-/** Node that holds an IndexWriter, indexing documents into its local index.
+/**
+ * Node that holds an IndexWriter, indexing documents into its local index.
*
- * @lucene.experimental */
-
+ * @lucene.experimental
+ */
public abstract class PrimaryNode extends Node {
// Current NRT segment infos, incRef'd with IndexWriter.deleter:
@@ -241,11 +242,12 @@ public abstract class PrimaryNode extends Node {
message("top: switch to infos=" + infos.toString() + " version=" + infos.getVersion());
- // Serialize the SegmentInfos:
- RAMOutputStream out = new RAMOutputStream(new RAMFile(), true);
- infos.write(dir, out);
- byte[] infosBytes = new byte[(int) out.getFilePointer()];
- out.writeTo(infosBytes, 0);
+ // Serialize the SegmentInfos.
+ ByteBuffersDataOutput buffer = new ByteBuffersDataOutput();
+ try (ByteBuffersIndexOutput tmpIndexOutput = new ByteBuffersIndexOutput(buffer, "temporary", "temporary")) {
+ infos.write(dir, tmpIndexOutput);
+ }
+ byte[] infosBytes = buffer.toArrayCopy();
Map<String,FileMetaData> filesMetaData = new HashMap<String,FileMetaData>();
for(SegmentCommitInfo info : infos) {
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleTransLog.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleTransLog.java
index ce75a29..9968bd8 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleTransLog.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleTransLog.java
@@ -31,8 +31,8 @@ import org.apache.lucene.document.StringField;
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
import org.apache.lucene.store.DataInput;
-import org.apache.lucene.store.RAMOutputStream;
/** This is a stupid yet functional transaction log: it never fsync's, never prunes, it's over-synchronized, it hard-wires id field name to "docid", can
* only handle specific docs/fields used by this test, etc. It's just barely enough to show how a translog could work on top of NRT
@@ -41,7 +41,7 @@ import org.apache.lucene.store.RAMOutputStream;
class SimpleTransLog implements Closeable {
final FileChannel channel;
- final RAMOutputStream buffer = new RAMOutputStream();
+ final ByteBuffersDataOutput buffer = ByteBuffersDataOutput.newResettableInstance();
final byte[] intBuffer = new byte[4];
final ByteBuffer intByteBuffer = ByteBuffer.wrap(intBuffer);
@@ -59,7 +59,7 @@ class SimpleTransLog implements Closeable {
/** Appends an addDocument op */
public synchronized long addDocument(String id, Document doc) throws IOException {
- assert buffer.getFilePointer() == 0;
+ assert buffer.size() == 0;
buffer.writeByte(OP_ADD_DOCUMENT);
encode(id, doc);
return flushBuffer();
@@ -67,7 +67,7 @@ class SimpleTransLog implements Closeable {
/** Appends an updateDocument op */
public synchronized long updateDocument(String id, Document doc) throws IOException {
- assert buffer.getFilePointer() == 0;
+ assert buffer.size() == 0;
buffer.writeByte(OP_UPDATE_DOCUMENT);
encode(id, doc);
return flushBuffer();
@@ -75,7 +75,7 @@ class SimpleTransLog implements Closeable {
/** Appends a deleteDocuments op */
public synchronized long deleteDocuments(String id) throws IOException {
- assert buffer.getFilePointer() == 0;
+ assert buffer.size() == 0;
buffer.writeByte(OP_DELETE_DOCUMENTS);
buffer.writeString(id);
return flushBuffer();
@@ -84,9 +84,8 @@ class SimpleTransLog implements Closeable {
/** Writes buffer to the file and returns the start position. */
private synchronized long flushBuffer() throws IOException {
long pos = channel.position();
- int len = (int) buffer.getFilePointer();
- byte[] bytes = new byte[len];
- buffer.writeTo(bytes, 0);
+ int len = Math.toIntExact(buffer.size());
+ byte[] bytes = buffer.toArrayCopy();
buffer.reset();
intBuffer[0] = (byte) (len >> 24);
diff --git a/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java b/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java
index 078ca8b..3f0c2bd 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java
@@ -34,8 +34,8 @@ import org.apache.lucene.index.IndexOptions;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.ByteBuffersDataOutput;
import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMOutputStream;
import org.apache.lucene.util.ArrayUtil;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
@@ -47,8 +47,8 @@ import org.apache.lucene.util.fst.Builder;
import org.apache.lucene.util.fst.ByteSequenceOutputs;
import org.apache.lucene.util.fst.BytesRefFSTEnum;
import org.apache.lucene.util.fst.FST;
-import org.apache.lucene.util.fst.PairOutputs.Pair;
import org.apache.lucene.util.fst.PairOutputs;
+import org.apache.lucene.util.fst.PairOutputs.Pair;
import org.apache.lucene.util.fst.PositiveIntOutputs;
import org.apache.lucene.util.fst.Util;
@@ -162,7 +162,6 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
}
private final List<FieldMetaData> fields = new ArrayList<>();
- private final String segment;
/** Create a new writer. The number of items (terms or
* sub-blocks) per block will aim to be between
@@ -176,8 +175,6 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
throws IOException
{
BlockTreeTermsWriter.validateSettings(minItemsInBlock, maxItemsInBlock);
- segment = state.segmentInfo.name;
-
maxDoc = state.segmentInfo.maxDoc();
final String termsFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_EXTENSION);
@@ -326,12 +323,12 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
return "BLOCK: " + brToString(prefix);
}
- public void compileIndex(List<PendingBlock> blocks, RAMOutputStream scratchBytes, IntsRefBuilder scratchIntsRef) throws IOException {
+ public void compileIndex(List<PendingBlock> blocks, ByteBuffersDataOutput scratchBytes, IntsRefBuilder scratchIntsRef) throws IOException {
assert (isFloor && blocks.size() > 1) || (isFloor == false && blocks.size() == 1): "isFloor=" + isFloor + " blocks=" + blocks;
assert this == blocks.get(0);
- assert scratchBytes.getFilePointer() == 0;
+ assert scratchBytes.size() == 0;
long maxVersionIndex = maxVersion;
@@ -360,9 +357,8 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
// System.out.println(" compile index for prefix=" + prefix);
//}
//indexBuilder.DEBUG = false;
- final byte[] bytes = new byte[(int) scratchBytes.getFilePointer()];
+ final byte[] bytes = scratchBytes.toArrayCopy();
assert bytes.length > 0;
- scratchBytes.writeTo(bytes, 0);
indexBuilder.add(Util.toIntsRef(prefix, scratchIntsRef), FST_OUTPUTS.newPair(new BytesRef(bytes, 0, bytes.length), Long.MAX_VALUE - maxVersionIndex));
scratchBytes.reset();
@@ -403,7 +399,7 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
}
}
- private final RAMOutputStream scratchBytes = new RAMOutputStream();
+ private final ByteBuffersDataOutput scratchBytes = ByteBuffersDataOutput.newResettableInstance();
private final IntsRefBuilder scratchIntsRef = new IntsRefBuilder();
class TermsWriter {
@@ -615,7 +611,7 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
assert longs[pos] >= 0;
metaWriter.writeVLong(longs[pos]);
}
- bytesWriter.writeTo(metaWriter);
+ bytesWriter.copyTo(metaWriter);
bytesWriter.reset();
absolute = false;
}
@@ -658,7 +654,7 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
assert longs[pos] >= 0;
metaWriter.writeVLong(longs[pos]);
}
- bytesWriter.writeTo(metaWriter);
+ bytesWriter.copyTo(metaWriter);
bytesWriter.reset();
absolute = false;
} else {
@@ -700,13 +696,13 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
// search on lookup
// Write suffixes byte[] blob to terms dict output:
- out.writeVInt((int) (suffixWriter.getFilePointer() << 1) | (isLeafBlock ? 1:0));
- suffixWriter.writeTo(out);
+ out.writeVInt((int) (suffixWriter.size() << 1) | (isLeafBlock ? 1:0));
+ suffixWriter.copyTo(out);
suffixWriter.reset();
// Write term meta data byte[] blob
- out.writeVInt((int) metaWriter.getFilePointer());
- metaWriter.writeTo(out);
+ out.writeVInt((int) metaWriter.size());
+ metaWriter.copyTo(out);
metaWriter.reset();
// if (DEBUG) {
@@ -831,9 +827,9 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
}
}
- private final RAMOutputStream suffixWriter = new RAMOutputStream();
- private final RAMOutputStream metaWriter = new RAMOutputStream();
- private final RAMOutputStream bytesWriter = new RAMOutputStream();
+ private final ByteBuffersDataOutput suffixWriter = ByteBuffersDataOutput.newResettableInstance();
+ private final ByteBuffersDataOutput metaWriter = ByteBuffersDataOutput.newResettableInstance();
+ private final ByteBuffersDataOutput bytesWriter = ByteBuffersDataOutput.newResettableInstance();
}
private boolean closed;
diff --git a/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java
index d9eeb88..4d12f0a 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java
@@ -40,9 +40,6 @@ import java.util.stream.Collectors;
import java.util.stream.IntStream;
import java.util.zip.CRC32;
-import com.carrotsearch.randomizedtesting.RandomizedTest;
-import com.carrotsearch.randomizedtesting.generators.RandomBytes;
-import com.carrotsearch.randomizedtesting.generators.RandomPicks;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexNotFoundException;
@@ -52,8 +49,12 @@ import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
import org.junit.Assert;
+import com.carrotsearch.randomizedtesting.RandomizedTest;
+import com.carrotsearch.randomizedtesting.generators.RandomBytes;
+import com.carrotsearch.randomizedtesting.generators.RandomPicks;
+
/**
- * Base class for per-Directory tests.
+ * Base class for {@link Directory} implementations.
*/
public abstract class BaseDirectoryTestCase extends LuceneTestCase {
@@ -410,7 +411,6 @@ public abstract class BaseDirectoryTestCase extends LuceneTestCase {
AtomicBoolean stop = new AtomicBoolean();
Thread writer = new Thread(() -> {
try {
- Random rnd = new Random(RandomizedTest.randomLong() + 1);
for (int i = 0, max = RandomizedTest.randomIntBetween(500, 1000); i < max; i++) {
String fileName = "file-" + i;
try (IndexOutput output = dir.createOutput(fileName, newIOContext(random()))) {
@@ -640,7 +640,8 @@ public abstract class BaseDirectoryTestCase extends LuceneTestCase {
// LUCENE-3541
public void testCopyBytesWithThreads() throws Exception {
try (Directory d = getDirectory(createTempDir("testCopyBytesWithThreads"))) {
- byte data[] = RandomBytes.randomBytesOfLengthBetween(random(), 101, 10000);
+ int headerLen = 100;
+ byte data[] = RandomBytes.randomBytesOfLengthBetween(random(), headerLen + 1, 10000);
IndexOutput output = d.createOutput("data", IOContext.DEFAULT);
output.writeBytes(data, 0, data.length);
@@ -648,8 +649,8 @@ public abstract class BaseDirectoryTestCase extends LuceneTestCase {
IndexInput input = d.openInput("data", IOContext.DEFAULT);
IndexOutput outputHeader = d.createOutput("header", IOContext.DEFAULT);
- // copy our 100-byte header
- outputHeader.copyBytes(input, 100);
+ // copy our header
+ outputHeader.copyBytes(input, headerLen);
outputHeader.close();
// now make N copies of the remaining bytes
@@ -662,7 +663,7 @@ public abstract class BaseDirectoryTestCase extends LuceneTestCase {
try {
start.await();
IndexOutput dst = d.createOutput("copy" + i, IOContext.DEFAULT);
- dst.copyBytes(src, src.length() - 100);
+ dst.copyBytes(src, src.length() - headerLen);
dst.close();
} catch (Exception e) {
throw new RuntimeException(e);
@@ -680,8 +681,8 @@ public abstract class BaseDirectoryTestCase extends LuceneTestCase {
for (int i = 0; i < threads; i++) {
try (IndexInput copiedData = d.openInput("copy" + i, IOContext.DEFAULT)) {
byte[] dataCopy = new byte[data.length];
- System.arraycopy(data, 0, dataCopy, 0, 100);
- copiedData.readBytes(dataCopy, 100, data.length - 100);
+ System.arraycopy(data, 0, dataCopy, 0, headerLen);
+ copiedData.readBytes(dataCopy, headerLen, data.length - headerLen);
assertArrayEquals(data, dataCopy);
}
}
@@ -723,7 +724,6 @@ public abstract class BaseDirectoryTestCase extends LuceneTestCase {
}
// random access APIs
-
public void testRandomLong() throws Exception {
try (Directory dir = getDirectory(createTempDir("testLongs"))) {
IndexOutput output = dir.createOutput("longs", newIOContext(random()));
@@ -935,7 +935,7 @@ public abstract class BaseDirectoryTestCase extends LuceneTestCase {
output.close();
IndexInput input = dir.openInput("bytes", newIOContext(random()));
- // seek to a random spot shouldnt impact slicing.
+ // seek to a random spot should not impact slicing.
input.seek(TestUtil.nextLong(random(), 0, input.length()));
for (int i = 0; i < num; i += 16) {
IndexInput slice1 = input.slice("slice1", i, num - i);
diff --git a/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java b/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
index 15f6454..316796a 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
@@ -255,20 +255,15 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
}
public synchronized final long sizeInBytes() throws IOException {
- if (in instanceof RAMDirectory)
- return ((RAMDirectory) in).ramBytesUsed();
- else {
- // hack
- long size = 0;
- for (String file : in.listAll()) {
- // hack 2: see TODO in ExtrasFS (ideally it would always return 0 byte
- // size for extras it creates, even though the size of non-regular files is not defined)
- if (!file.startsWith("extra")) {
- size += in.fileLength(file);
- }
+ long size = 0;
+ for (String file : in.listAll()) {
+ // hack 2: see TODO in ExtrasFS (ideally it would always return 0 byte
+ // size for extras it creates, even though the size of non-regular files is not defined)
+ if (!file.startsWith("extra")) {
+ size += in.fileLength(file);
}
- return size;
}
+ return size;
}
public synchronized void corruptUnknownFiles() throws IOException {
@@ -510,7 +505,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
return this.maxUsedSize;
}
public void resetMaxUsedSizeInBytes() throws IOException {
- this.maxUsedSize = getRecomputedActualSizeInBytes();
+ this.maxUsedSize = sizeInBytes();
}
/**
@@ -773,32 +768,6 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
addFileHandle(ii, name, Handle.Input);
return ii;
}
-
- /** Provided for testing purposes. Use sizeInBytes() instead. */
- public synchronized final long getRecomputedSizeInBytes() throws IOException {
- if (!(in instanceof RAMDirectory))
- return sizeInBytes();
- long size = 0;
- for(final RAMFile file: ((RAMDirectory)in).fileMap.values()) {
- size += file.ramBytesUsed();
- }
- return size;
- }
-
- /** Like getRecomputedSizeInBytes(), but, uses actual file
- * lengths rather than buffer allocations (which are
- * quantized up to nearest
- * RAMOutputStream.BUFFER_SIZE (now 1024) bytes.
- */
-
- public final synchronized long getRecomputedActualSizeInBytes() throws IOException {
- if (!(in instanceof RAMDirectory))
- return sizeInBytes();
- long size = 0;
- for (final RAMFile file : ((RAMDirectory)in).fileMap.values())
- size += file.length;
- return size;
- }
// NOTE: This is off by default; see LUCENE-5574
private volatile boolean assertNoUnreferencedFilesOnClose;
diff --git a/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java b/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java
index 510e930..17fb07e 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java
@@ -59,7 +59,7 @@ public class MockIndexOutputWrapper extends IndexOutput {
if (dir.maxSize != 0 && freeSpace <= len) {
// Compute the real disk free. This will greatly slow
// down our test but makes it more accurate:
- realUsage = dir.getRecomputedActualSizeInBytes();
+ realUsage = dir.sizeInBytes();
freeSpace = dir.maxSize - realUsage;
}
@@ -75,7 +75,7 @@ public class MockIndexOutputWrapper extends IndexOutput {
if (realUsage > dir.maxUsedSize) {
dir.maxUsedSize = realUsage;
}
- String message = "fake disk full at " + dir.getRecomputedActualSizeInBytes() + " bytes when writing " + name + " (file length=" + delegate.getFilePointer();
+ String message = "fake disk full at " + dir.sizeInBytes() + " bytes when writing " + name + " (file length=" + delegate.getFilePointer();
if (freeSpace > 0) {
message += "; wrote " + freeSpace + " of " + len + " bytes";
}
@@ -106,7 +106,7 @@ public class MockIndexOutputWrapper extends IndexOutput {
if (dir.trackDiskUsage) {
// Now compute actual disk usage & track the maxUsedSize
// in the MockDirectoryWrapper:
- long size = dir.getRecomputedActualSizeInBytes();
+ long size = dir.sizeInBytes();
if (size > dir.maxUsedSize) {
dir.maxUsedSize = size;
}
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java b/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java
index 6437d8e..9b7b666 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java
@@ -21,6 +21,7 @@ import java.io.File;
import java.lang.reflect.Array;
import java.lang.reflect.Field;
import java.lang.reflect.Modifier;
+import java.nio.ByteBuffer;
import java.nio.file.Path;
import java.security.AccessController;
import java.security.PrivilegedAction;
@@ -211,6 +212,8 @@ public final class RamUsageTester {
a(StringBuffer.class, v -> charArraySize(v.capacity()));
// Types with large buffers:
a(ByteArrayOutputStream.class, v -> byteArraySize(v.size()));
+ // Approximate ByteBuffers with their underling storage (ignores field overhead).
+ a(ByteBuffer.class, v -> byteArraySize(v.capacity()));
// For File and Path, we just take the length of String representation as approximation:
a(File.class, v -> charArraySize(v.toString().length()));
a(Path.class, v -> charArraySize(v.toString().length()));