pylucene 3.5.0-3
[pylucene.git] / lucene-java-3.5.0 / lucene / src / java / org / apache / lucene / store / NIOFSDirectory.java
1 package org.apache.lucene.store;
2
3 /**
4  * Licensed to the Apache Software Foundation (ASF) under one or more
5  * contributor license agreements. See the NOTICE file distributed with this
6  * work for additional information regarding copyright ownership. The ASF
7  * licenses this file to You under the Apache License, Version 2.0 (the
8  * "License"); you may not use this file except in compliance with the License.
9  * You may obtain a copy of the License at
10  * 
11  * http://www.apache.org/licenses/LICENSE-2.0
12  * 
13  * Unless required by applicable law or agreed to in writing, software
14  * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
15  * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
16  * License for the specific language governing permissions and limitations under
17  * the License.
18  */
19
20 import java.io.EOFException;
21 import java.io.File;
22 import java.io.IOException;
23 import java.nio.ByteBuffer;
24 import java.nio.channels.ClosedChannelException; // javadoc
25 import java.nio.channels.FileChannel;
26 import java.util.concurrent.Future; // javadoc
27
28 /**
29  * An {@link FSDirectory} implementation that uses java.nio's FileChannel's
30  * positional read, which allows multiple threads to read from the same file
31  * without synchronizing.
32  * <p>
33  * This class only uses FileChannel when reading; writing is achieved with
34  * {@link FSDirectory.FSIndexOutput}.
35  * <p>
36  * <b>NOTE</b>: NIOFSDirectory is not recommended on Windows because of a bug in
37  * how FileChannel.read is implemented in Sun's JRE. Inside of the
38  * implementation the position is apparently synchronized. See <a
39  * href="http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6265734">here</a>
40  * for details.
41  * </p>
42  * <p>
43  * <font color="red"><b>NOTE:</b> Accessing this class either directly or
44  * indirectly from a thread while it's interrupted can close the
45  * underlying file descriptor immediately if at the same time the thread is
46  * blocked on IO. The file descriptor will remain closed and subsequent access
47  * to {@link NIOFSDirectory} will throw a {@link ClosedChannelException}. If
48  * your application uses either {@link Thread#interrupt()} or
49  * {@link Future#cancel(boolean)} you should use {@link SimpleFSDirectory} in
50  * favor of {@link NIOFSDirectory}.</font>
51  * </p>
52  */
53 public class NIOFSDirectory extends FSDirectory {
54
55   /** Create a new NIOFSDirectory for the named location.
56    * 
57    * @param path the path of the directory
58    * @param lockFactory the lock factory to use, or null for the default
59    * ({@link NativeFSLockFactory});
60    * @throws IOException
61    */
62   public NIOFSDirectory(File path, LockFactory lockFactory) throws IOException {
63     super(path, lockFactory);
64   }
65
66   /** Create a new NIOFSDirectory for the named location and {@link NativeFSLockFactory}.
67    *
68    * @param path the path of the directory
69    * @throws IOException
70    */
71   public NIOFSDirectory(File path) throws IOException {
72     super(path, null);
73   }
74
75   /** Creates an IndexInput for the file with the given name. */
76   @Override
77   public IndexInput openInput(String name, int bufferSize) throws IOException {
78     ensureOpen();
79     return new NIOFSIndexInput(new File(getDirectory(), name), bufferSize, getReadChunkSize());
80   }
81
82   protected static class NIOFSIndexInput extends SimpleFSDirectory.SimpleFSIndexInput {
83
84     private ByteBuffer byteBuf; // wraps the buffer for NIO
85
86     private byte[] otherBuffer;
87     private ByteBuffer otherByteBuf;
88
89     final FileChannel channel;
90
91     public NIOFSIndexInput(File path, int bufferSize, int chunkSize) throws IOException {
92       super("NIOFSIndexInput(path=\"" + path + "\")", path, bufferSize, chunkSize);
93       channel = file.getChannel();
94     }
95
96     @Override
97     protected void newBuffer(byte[] newBuffer) {
98       super.newBuffer(newBuffer);
99       byteBuf = ByteBuffer.wrap(newBuffer);
100     }
101
102     @Override
103     public void close() throws IOException {
104       if (!isClone && file.isOpen) {
105         // Close the channel & file
106         try {
107           channel.close();
108         } finally {
109           file.close();
110         }
111       }
112     }
113
114     @Override
115     protected void readInternal(byte[] b, int offset, int len) throws IOException {
116
117       final ByteBuffer bb;
118
119       // Determine the ByteBuffer we should use
120       if (b == buffer && 0 == offset) {
121         // Use our own pre-wrapped byteBuf:
122         assert byteBuf != null;
123         byteBuf.clear();
124         byteBuf.limit(len);
125         bb = byteBuf;
126       } else {
127         if (offset == 0) {
128           if (otherBuffer != b) {
129             // Now wrap this other buffer; with compound
130             // file, we are repeatedly called with its
131             // buffer, so we wrap it once and then re-use it
132             // on subsequent calls
133             otherBuffer = b;
134             otherByteBuf = ByteBuffer.wrap(b);
135           } else
136             otherByteBuf.clear();
137           otherByteBuf.limit(len);
138           bb = otherByteBuf;
139         } else {
140           // Always wrap when offset != 0
141           bb = ByteBuffer.wrap(b, offset, len);
142         }
143       }
144
145       int readOffset = bb.position();
146       int readLength = bb.limit() - readOffset;
147       assert readLength == len;
148
149       long pos = getFilePointer();
150
151       try {
152         while (readLength > 0) {
153           final int limit;
154           if (readLength > chunkSize) {
155             // LUCENE-1566 - work around JVM Bug by breaking
156             // very large reads into chunks
157             limit = readOffset + chunkSize;
158           } else {
159             limit = readOffset + readLength;
160           }
161           bb.limit(limit);
162           int i = channel.read(bb, pos);
163           if (i == -1) {
164             throw new EOFException("read past EOF (resource: " + this + ")");
165           }
166           pos += i;
167           readOffset += i;
168           readLength -= i;
169         }
170       } catch (OutOfMemoryError e) {
171         // propagate OOM up and add a hint for 32bit VM Users hitting the bug
172         // with a large chunk size in the fast path.
173         final OutOfMemoryError outOfMemoryError = new OutOfMemoryError(
174               "OutOfMemoryError likely caused by the Sun VM Bug described in "
175               + "https://issues.apache.org/jira/browse/LUCENE-1566; try calling FSDirectory.setReadChunkSize "
176               + "with a value smaller than the current chunk size (" + chunkSize + ")");
177         outOfMemoryError.initCause(e);
178         throw outOfMemoryError;
179       } catch (IOException ioe) {
180         IOException newIOE = new IOException(ioe.getMessage() + ": " + this);
181         newIOE.initCause(ioe);
182         throw newIOE;
183       }
184     }
185   }
186 }