mirror of https://github.com/apache/lucene.git
LUCENE-8560: TestByteBuffersDirectory.testSeekPastEOF() failures with ByteArrayIndexInput. ByteArrayIndexInput removed entirely, without a replacement.
This commit is contained in:
parent
397b88aefa
commit
4e2481b04b
|
@ -1,208 +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.io.UncheckedIOException;
|
||||
import java.util.Locale;
|
||||
|
||||
/**
|
||||
* A {@link IndexInput} backed by a byte array.
|
||||
*
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public final class ByteArrayIndexInput extends IndexInput implements RandomAccessInput {
|
||||
private byte[] bytes;
|
||||
|
||||
private final int offset;
|
||||
private final int length;
|
||||
|
||||
private int pos;
|
||||
|
||||
public ByteArrayIndexInput(String description, byte[] bytes) {
|
||||
this(description, bytes, 0, bytes.length);
|
||||
}
|
||||
|
||||
public ByteArrayIndexInput(String description, byte[] bytes, int offs, int length) {
|
||||
super(description);
|
||||
this.offset = offs;
|
||||
this.bytes = bytes;
|
||||
this.length = length;
|
||||
this.pos = offs;
|
||||
}
|
||||
|
||||
public long getFilePointer() {
|
||||
return pos - offset;
|
||||
}
|
||||
|
||||
public void seek(long pos) throws EOFException {
|
||||
int newPos = Math.toIntExact(pos + offset);
|
||||
try {
|
||||
if (pos < 0 || pos > length) {
|
||||
throw new EOFException();
|
||||
}
|
||||
} finally {
|
||||
this.pos = newPos;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long length() {
|
||||
return length;
|
||||
}
|
||||
|
||||
@Override
|
||||
public short readShort() {
|
||||
return (short) (((bytes[pos++] & 0xFF) << 8) | (bytes[pos++] & 0xFF));
|
||||
}
|
||||
|
||||
@Override
|
||||
public int readInt() {
|
||||
return ((bytes[pos++] & 0xFF) << 24) | ((bytes[pos++] & 0xFF) << 16)
|
||||
| ((bytes[pos++] & 0xFF) << 8) | (bytes[pos++] & 0xFF);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long readLong() {
|
||||
final int i1 = ((bytes[pos++] & 0xff) << 24) | ((bytes[pos++] & 0xff) << 16) |
|
||||
((bytes[pos++] & 0xff) << 8) | (bytes[pos++] & 0xff);
|
||||
final int i2 = ((bytes[pos++] & 0xff) << 24) | ((bytes[pos++] & 0xff) << 16) |
|
||||
((bytes[pos++] & 0xff) << 8) | (bytes[pos++] & 0xff);
|
||||
return (((long)i1) << 32) | (i2 & 0xFFFFFFFFL);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int readVInt() {
|
||||
byte b = bytes[pos++];
|
||||
if (b >= 0) return b;
|
||||
int i = b & 0x7F;
|
||||
b = bytes[pos++];
|
||||
i |= (b & 0x7F) << 7;
|
||||
if (b >= 0) return i;
|
||||
b = bytes[pos++];
|
||||
i |= (b & 0x7F) << 14;
|
||||
if (b >= 0) return i;
|
||||
b = bytes[pos++];
|
||||
i |= (b & 0x7F) << 21;
|
||||
if (b >= 0) return i;
|
||||
b = bytes[pos++];
|
||||
// Warning: the next ands use 0x0F / 0xF0 - beware copy/paste errors:
|
||||
i |= (b & 0x0F) << 28;
|
||||
if ((b & 0xF0) == 0) return i;
|
||||
throw new RuntimeException("Invalid vInt detected (too many bits)");
|
||||
}
|
||||
|
||||
@Override
|
||||
public long readVLong() {
|
||||
byte b = bytes[pos++];
|
||||
if (b >= 0) return b;
|
||||
long i = b & 0x7FL;
|
||||
b = bytes[pos++];
|
||||
i |= (b & 0x7FL) << 7;
|
||||
if (b >= 0) return i;
|
||||
b = bytes[pos++];
|
||||
i |= (b & 0x7FL) << 14;
|
||||
if (b >= 0) return i;
|
||||
b = bytes[pos++];
|
||||
i |= (b & 0x7FL) << 21;
|
||||
if (b >= 0) return i;
|
||||
b = bytes[pos++];
|
||||
i |= (b & 0x7FL) << 28;
|
||||
if (b >= 0) return i;
|
||||
b = bytes[pos++];
|
||||
i |= (b & 0x7FL) << 35;
|
||||
if (b >= 0) return i;
|
||||
b = bytes[pos++];
|
||||
i |= (b & 0x7FL) << 42;
|
||||
if (b >= 0) return i;
|
||||
b = bytes[pos++];
|
||||
i |= (b & 0x7FL) << 49;
|
||||
if (b >= 0) return i;
|
||||
b = bytes[pos++];
|
||||
i |= (b & 0x7FL) << 56;
|
||||
if (b >= 0) return i;
|
||||
throw new RuntimeException("Invalid vLong detected (negative values disallowed)");
|
||||
}
|
||||
|
||||
// NOTE: AIOOBE not EOF if you read too much
|
||||
@Override
|
||||
public byte readByte() {
|
||||
return bytes[pos++];
|
||||
}
|
||||
|
||||
// NOTE: AIOOBE not EOF if you read too much
|
||||
@Override
|
||||
public void readBytes(byte[] b, int offset, int len) {
|
||||
System.arraycopy(bytes, pos, b, offset, len);
|
||||
pos += len;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
bytes = null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public IndexInput clone() {
|
||||
ByteArrayIndexInput slice = slice("(cloned)" + toString(), 0, length());
|
||||
try {
|
||||
slice.seek(getFilePointer());
|
||||
} catch (EOFException e) {
|
||||
throw new UncheckedIOException(e);
|
||||
}
|
||||
return slice;
|
||||
}
|
||||
|
||||
public ByteArrayIndexInput slice(String sliceDescription, long offset, long length) {
|
||||
if (offset < 0 || length < 0 || offset + length > this.length) {
|
||||
throw new IllegalArgumentException(String.format(Locale.ROOT,
|
||||
"slice(offset=%s, length=%s) is out of bounds: %s",
|
||||
offset, length, this));
|
||||
}
|
||||
|
||||
return new ByteArrayIndexInput(sliceDescription, this.bytes, Math.toIntExact(this.offset + offset),
|
||||
Math.toIntExact(length));
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte readByte(long pos) throws IOException {
|
||||
return bytes[Math.toIntExact(offset + pos)];
|
||||
}
|
||||
|
||||
@Override
|
||||
public short readShort(long pos) throws IOException {
|
||||
int i = Math.toIntExact(offset + pos);
|
||||
return (short) (((bytes[i] & 0xFF) << 8) |
|
||||
(bytes[i + 1] & 0xFF));
|
||||
}
|
||||
|
||||
@Override
|
||||
public int readInt(long pos) throws IOException {
|
||||
int i = Math.toIntExact(offset + pos);
|
||||
return ((bytes[i] & 0xFF) << 24) |
|
||||
((bytes[i + 1] & 0xFF) << 16) |
|
||||
((bytes[i + 2] & 0xFF) << 8) |
|
||||
(bytes[i + 3] & 0xFF);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long readLong(long pos) throws IOException {
|
||||
return (((long) readInt(pos)) << 32) |
|
||||
(readInt(pos + 4) & 0xFFFFFFFFL);
|
||||
}
|
||||
}
|
|
@ -68,21 +68,14 @@ public final class ByteBuffersDirectory extends BaseDirectory {
|
|||
return new ByteBuffersIndexInput(dataInput, inputName);
|
||||
};
|
||||
|
||||
public static final BiFunction<String, ByteBuffersDataOutput, IndexInput> OUTPUT_AS_BYTE_ARRAY =
|
||||
(fileName, output) -> {
|
||||
byte[] array = output.toArrayCopy();
|
||||
String inputName = String.format(Locale.ROOT, "%s (file=%s, length=%s)",
|
||||
ByteArrayIndexInput.class.getSimpleName(),
|
||||
fileName,
|
||||
array.length);
|
||||
return new ByteArrayIndexInput(inputName, array, 0, array.length);
|
||||
};
|
||||
public static final BiFunction<String, ByteBuffersDataOutput, IndexInput> OUTPUT_AS_BYTE_ARRAY = OUTPUT_AS_ONE_BUFFER;
|
||||
|
||||
public static final BiFunction<String, ByteBuffersDataOutput, IndexInput> OUTPUT_AS_MANY_BUFFERS_LUCENE =
|
||||
(fileName, output) -> {
|
||||
List<ByteBuffer> bufferList = output.toBufferList();
|
||||
int chunkSizePower;
|
||||
bufferList.add(ByteBuffer.allocate(0));
|
||||
|
||||
int chunkSizePower;
|
||||
int blockSize = ByteBuffersDataInput.determineBlockPage(bufferList);
|
||||
if (blockSize == 0) {
|
||||
chunkSizePower = 30;
|
||||
|
@ -95,8 +88,8 @@ public final class ByteBuffersDirectory extends BaseDirectory {
|
|||
fileName);
|
||||
|
||||
ByteBufferGuard guard = new ByteBufferGuard("none", (String resourceDescription, ByteBuffer b) -> {});
|
||||
return ByteBufferIndexInput.newInstance(inputName,
|
||||
bufferList.toArray(new ByteBuffer [bufferList.size()]),
|
||||
return ByteBufferIndexInput.newInstance(inputName,
|
||||
bufferList.toArray(new ByteBuffer [bufferList.size()]),
|
||||
output.size(), chunkSizePower, guard);
|
||||
};
|
||||
|
||||
|
|
|
@ -19,6 +19,7 @@ package org.apache.lucene.replicator.nrt;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
|
@ -42,7 +43,9 @@ import org.apache.lucene.search.SearcherFactory;
|
|||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.store.AlreadyClosedException;
|
||||
import org.apache.lucene.store.BufferedChecksumIndexInput;
|
||||
import org.apache.lucene.store.ByteArrayIndexInput;
|
||||
import org.apache.lucene.store.ByteBuffersDataInput;
|
||||
import org.apache.lucene.store.ByteBuffersIndexInput;
|
||||
import org.apache.lucene.store.ChecksumIndexInput;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexOutput;
|
||||
|
@ -242,7 +245,7 @@ public abstract class ReplicaNode extends Node {
|
|||
byte[] infosBytes = job.getCopyState().infosBytes;
|
||||
|
||||
SegmentInfos syncInfos = SegmentInfos.readCommit(dir,
|
||||
new BufferedChecksumIndexInput(new ByteArrayIndexInput("SegmentInfos", job.getCopyState().infosBytes)),
|
||||
toIndexInput(job.getCopyState().infosBytes),
|
||||
job.getCopyState().gen);
|
||||
|
||||
// Must always commit to a larger generation than what's currently in the index:
|
||||
|
@ -382,7 +385,7 @@ public abstract class ReplicaNode extends Node {
|
|||
// Turn byte[] back to SegmentInfos:
|
||||
byte[] infosBytes = copyState.infosBytes;
|
||||
SegmentInfos infos = SegmentInfos.readCommit(dir,
|
||||
new BufferedChecksumIndexInput(new ByteArrayIndexInput("SegmentInfos", copyState.infosBytes)),
|
||||
toIndexInput(copyState.infosBytes),
|
||||
copyState.gen);
|
||||
assert infos.getVersion() == copyState.version;
|
||||
|
||||
|
@ -439,6 +442,13 @@ public abstract class ReplicaNode extends Node {
|
|||
markerCount));
|
||||
}
|
||||
|
||||
private ChecksumIndexInput toIndexInput(byte[] input) {
|
||||
return new BufferedChecksumIndexInput(
|
||||
new ByteBuffersIndexInput(
|
||||
new ByteBuffersDataInput(
|
||||
Arrays.asList(ByteBuffer.wrap(input))), "SegmentInfos"));
|
||||
}
|
||||
|
||||
/** Start a background copying job, to copy the specified files from the current primary node. If files is null then the latest copy
|
||||
* state should be copied. If prevJob is not null, then the new copy job is replacing it and should 1) cancel the previous one, and
|
||||
* 2) optionally salvage e.g. partially copied and, shared with the new copy job, files. */
|
||||
|
|
|
@ -513,10 +513,15 @@ public abstract class BaseDirectoryTestCase extends LuceneTestCase {
|
|||
o.writeBytes(b, 0, len);
|
||||
o.close();
|
||||
IndexInput i = dir.openInput("out", newIOContext(random()));
|
||||
expectThrows(EOFException.class, () -> {
|
||||
i.seek(len + random().nextInt(2048));
|
||||
i.readByte();
|
||||
});
|
||||
|
||||
// Seeking past EOF should always throw EOFException
|
||||
expectThrows(EOFException.class, () -> i.seek(len + RandomizedTest.randomIntBetween(1, 2048)));
|
||||
|
||||
// Seeking exactly to EOF should never throw any exception.
|
||||
i.seek(len);
|
||||
|
||||
// But any read following the seek(len) should throw an EOFException.
|
||||
expectThrows(EOFException.class, i::readByte);
|
||||
|
||||
i.close();
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue