sync with latest patch of byte buffer directory submitted ot Lucene, seems to pass all Luceen tests

This commit is contained in:
Shay Banon 2011-12-22 20:31:32 +02:00
parent 415ee6425a
commit a0fb6f3d92
7 changed files with 196 additions and 166 deletions

View File

@ -1,24 +1,22 @@
/*
* Licensed to Elastic Search and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Elastic Search 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.bytebuffer;
/**
* 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.
*/
import java.io.IOException;
import java.lang.reflect.Method;
import java.nio.ByteBuffer;

View File

@ -1,24 +1,22 @@
/*
* Licensed to Elastic Search and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Elastic Search 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.bytebuffer;
/**
* 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.
*/
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
@ -41,8 +39,6 @@ import java.util.concurrent.ConcurrentHashMap;
* <p/>
* <p>If constructed with {@link ByteBufferAllocator}, it allows to control the allocation and release of
* byte buffer. For example, custom implementations can include caching of byte buffers.
*
*
*/
public class ByteBufferDirectory extends Directory {
@ -110,7 +106,7 @@ public class ByteBufferDirectory extends Directory {
do {
try {
Thread.sleep(0, 1);
} catch (InterruptedException ie) {
} catch (java.lang.InterruptedException ie) {
// In 3.0 we will change this to throw
// InterruptedException instead
Thread.currentThread().interrupt();
@ -127,7 +123,7 @@ public class ByteBufferDirectory extends Directory {
ByteBufferFile file = files.remove(name);
if (file == null)
throw new FileNotFoundException(name);
file.clean();
file.delete();
}
@Override
@ -147,9 +143,9 @@ public class ByteBufferDirectory extends Directory {
ByteBufferFile file = new ByteBufferFile(this, allocator.sizeInBytes(allocatorType));
ByteBufferFile existing = files.put(name, file);
if (existing != null) {
existing.clean();
existing.delete();
}
return new ByteBufferIndexOutput(allocator, allocatorType, file);
return new ByteBufferIndexOutput(name, allocator, allocatorType, file);
}
@Override
@ -157,7 +153,7 @@ public class ByteBufferDirectory extends Directory {
ByteBufferFile file = files.get(name);
if (file == null)
throw new FileNotFoundException(name);
return new ByteBufferIndexInput(file);
return new ByteBufferIndexInput(name, file);
}
@Override

View File

@ -1,29 +1,27 @@
/*
* Licensed to Elastic Search and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Elastic Search 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.bytebuffer;
/**
* 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.
*/
import java.nio.ByteBuffer;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.atomic.AtomicInteger;
/**
*
*/
public class ByteBufferFile {
@ -35,6 +33,8 @@ public class ByteBufferFile {
// This is publicly modifiable via Directory.touchFile(), so direct access not supported
private volatile long lastModified = System.currentTimeMillis();
private final AtomicInteger refCount = new AtomicInteger(1);
public ByteBufferFile(ByteBufferDirectory dir, int bufferSize) {
this.dir = dir;
this.bufferSize = bufferSize;
@ -70,8 +70,17 @@ public class ByteBufferFile {
return buffers.size();
}
void clean() {
if (buffers != null) {
void delete() {
decRef();
}
void incRef() {
refCount.incrementAndGet();
}
void decRef() {
if (refCount.decrementAndGet() == 0) {
length = 0;
for (ByteBuffer buffer : buffers) {
dir.releaseBuffer(buffer);
}

View File

@ -19,11 +19,12 @@ package org.apache.lucene.store.bytebuffer;
import org.apache.lucene.store.IndexInput;
import java.io.EOFException;
import java.io.IOException;
import java.nio.BufferUnderflowException;
import java.nio.ByteBuffer;
/**
*
*/
public class ByteBufferIndexInput extends IndexInput {
@ -38,8 +39,12 @@ public class ByteBufferIndexInput extends IndexInput {
private long bufferStart;
private final int BUFFER_SIZE;
public ByteBufferIndexInput(ByteBufferFile file) throws IOException {
private volatile boolean closed = false;
public ByteBufferIndexInput(String name, ByteBufferFile file) throws IOException {
super("BBIndexInput(name=" + name + ")");
this.file = file;
this.file.incRef();
this.length = file.getLength();
this.BUFFER_SIZE = file.bufferSize;
@ -51,7 +56,13 @@ public class ByteBufferIndexInput extends IndexInput {
@Override
public void close() {
// nothing to do here
// we protected from double closing the index input since
// some tests do that...
if (closed) {
return;
}
closed = true;
file.decRef();
}
@Override
@ -59,32 +70,38 @@ public class ByteBufferIndexInput extends IndexInput {
return length;
}
// @Override
// public short readShort() throws IOException {
// try {
// return currentBuffer.getShort();
// } catch (BufferUnderflowException e) {
// return super.readShort();
// }
// }
//
// @Override
// public int readInt() throws IOException {
// try {
// return currentBuffer.getInt();
// } catch (BufferUnderflowException e) {
// return super.readInt();
// }
// }
//
// @Override
// public long readLong() throws IOException {
// try {
// return currentBuffer.getLong();
// } catch (BufferUnderflowException e) {
// return super.readLong();
// }
// }
@Override
public short readShort() throws IOException {
try {
currentBuffer.mark();
return currentBuffer.getShort();
} catch (BufferUnderflowException e) {
currentBuffer.reset();
return super.readShort();
}
}
@Override
public int readInt() throws IOException {
try {
currentBuffer.mark();
return currentBuffer.getInt();
} catch (BufferUnderflowException e) {
currentBuffer.reset();
return super.readInt();
}
}
@Override
public long readLong() throws IOException {
try {
currentBuffer.mark();
return currentBuffer.getLong();
} catch (BufferUnderflowException e) {
currentBuffer.reset();
return super.readLong();
}
}
@Override
public byte readByte() throws IOException {
@ -120,27 +137,28 @@ public class ByteBufferIndexInput extends IndexInput {
public void seek(long pos) throws IOException {
if (currentBuffer == EMPTY_BUFFER || pos < bufferStart || pos >= bufferStart + BUFFER_SIZE) {
currentBufferIndex = (int) (pos / BUFFER_SIZE);
if (currentBufferIndex >= file.numBuffers()) {
// if we are past EOF, don't throw one here, instead, move it to the last position in the last buffer
currentBufferIndex = file.numBuffers() - 1;
currentBuffer = currentBufferIndex == -1 ? EMPTY_BUFFER : file.getBuffer(currentBufferIndex);
currentBuffer.position(currentBuffer.limit());
return;
} else {
switchCurrentBuffer(false);
}
switchCurrentBuffer(false);
}
try {
currentBuffer.position((int) (pos % BUFFER_SIZE));
// Grrr, need to wrap in IllegalArgumentException since tests (if not other places)
// expect an IOException...
} catch (IllegalArgumentException e) {
currentBuffer.position(currentBuffer.limit());
IOException ioException = new IOException("seeking past position");
ioException.initCause(e);
throw ioException;
}
}
private void switchCurrentBuffer(boolean enforceEOF) throws IOException {
if (currentBufferIndex >= file.numBuffers()) {
// end of file reached, no more buffers left
if (enforceEOF) {
throw new IOException("Read past EOF");
throw new EOFException("Read past EOF (resource: " + this + ")");
} else {
// Force EOF if a read takes place at this position
currentBufferIndex--;
currentBuffer.position(currentBuffer.limit());
}
} else {
ByteBuffer buffer = file.getBuffer(currentBufferIndex);
@ -152,8 +170,16 @@ public class ByteBufferIndexInput extends IndexInput {
long buflen = length - bufferStart;
if (buflen < BUFFER_SIZE) {
currentBuffer.limit((int) buflen);
if (enforceEOF && buflen == 0) {
throw new IOException("Read past EOF");
}
// we need to enforce EOF here as well...
if (!currentBuffer.hasRemaining()) {
if (enforceEOF) {
throw new EOFException("Read past EOF (resource: " + this + ")");
} else {
// Force EOF if a read takes place at this position
currentBufferIndex--;
currentBuffer.position(currentBuffer.limit());
}
}
}
@ -162,6 +188,7 @@ public class ByteBufferIndexInput extends IndexInput {
@Override
public Object clone() {
ByteBufferIndexInput cloned = (ByteBufferIndexInput) super.clone();
cloned.file.incRef(); // inc ref on cloned one
if (currentBuffer != EMPTY_BUFFER) {
cloned.currentBuffer = currentBuffer.asReadOnlyBuffer();
cloned.currentBuffer.position(currentBuffer.position());

View File

@ -1,23 +1,22 @@
/*
* Licensed to Elastic Search and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Elastic Search 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
package org.apache.lucene.store.bytebuffer;
/**
* 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
* 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.
* 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.bytebuffer;
import org.apache.lucene.store.IndexOutput;
@ -28,6 +27,9 @@ import java.nio.ByteBuffer;
*/
public class ByteBufferIndexOutput extends IndexOutput {
private final static ByteBuffer EMPTY_BUFFER = ByteBuffer.allocate(0).asReadOnlyBuffer();
private final String name;
private final ByteBufferAllocator allocator;
private final ByteBufferAllocator.Type allocatorType;
private final int BUFFER_SIZE;
@ -38,13 +40,15 @@ public class ByteBufferIndexOutput extends IndexOutput {
private long bufferStart;
public ByteBufferIndexOutput(ByteBufferAllocator allocator, ByteBufferAllocator.Type allocatorType, ByteBufferFile file) throws IOException {
public ByteBufferIndexOutput(String name, ByteBufferAllocator allocator, ByteBufferAllocator.Type allocatorType, ByteBufferFile file) throws IOException {
this.name = name;
this.allocator = allocator;
this.allocatorType = allocatorType;
this.BUFFER_SIZE = file.bufferSize;
this.file = file;
// create the first buffer we write to
switchCurrentBuffer();
currentBufferIndex = -1;
currentBuffer = EMPTY_BUFFER;
}
@Override

View File

@ -1,24 +1,22 @@
/*
* Licensed to Elastic Search and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Elastic Search 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.bytebuffer;
/**
* 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.
*/
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.concurrent.ArrayBlockingQueue;

View File

@ -1,24 +1,22 @@
/*
* Licensed to Elastic Search and Shay Banon under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Elastic Search 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.bytebuffer;
/**
* 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.
*/
import java.io.IOException;
import java.nio.ByteBuffer;