|
| 1 | +/* |
| 2 | + * Licensed to Elasticsearch under one or more contributor |
| 3 | + * license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright |
| 5 | + * ownership. Elasticsearch licenses this file to you under |
| 6 | + * the Apache License, Version 2.0 (the "License"); you may |
| 7 | + * not use this file except in compliance with the License. |
| 8 | + * You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, |
| 13 | + * software distributed under the License is distributed on an |
| 14 | + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY |
| 15 | + * KIND, either express or implied. See the License for the |
| 16 | + * specific language governing permissions and limitations |
| 17 | + * under the License. |
| 18 | + */ |
| 19 | + |
| 20 | +package org.elasticsearch.index.store; |
| 21 | + |
| 22 | +import org.apache.lucene.store.Directory; |
| 23 | +import org.apache.lucene.store.FilterDirectory; |
| 24 | +import org.apache.lucene.store.IOContext; |
| 25 | +import org.apache.lucene.store.IndexOutput; |
| 26 | +import org.elasticsearch.common.lucene.store.FilterIndexOutput; |
| 27 | +import org.elasticsearch.common.unit.TimeValue; |
| 28 | +import org.elasticsearch.common.util.SingleObjectCache; |
| 29 | + |
| 30 | +import java.io.FileNotFoundException; |
| 31 | +import java.io.IOException; |
| 32 | +import java.io.UncheckedIOException; |
| 33 | +import java.nio.file.AccessDeniedException; |
| 34 | +import java.nio.file.NoSuchFileException; |
| 35 | + |
| 36 | +final class ByteSizeCachingDirectory extends FilterDirectory { |
| 37 | + |
| 38 | + private static class SizeAndModCount { |
| 39 | + final long size; |
| 40 | + final long modCount; |
| 41 | + final boolean pendingWrite; |
| 42 | + |
| 43 | + SizeAndModCount(long length, long modCount, boolean pendingWrite) { |
| 44 | + this.size = length; |
| 45 | + this.modCount = modCount; |
| 46 | + this.pendingWrite = pendingWrite; |
| 47 | + } |
| 48 | + } |
| 49 | + |
| 50 | + private static long estimateSizeInBytes(Directory directory) throws IOException { |
| 51 | + long estimatedSize = 0; |
| 52 | + String[] files = directory.listAll(); |
| 53 | + for (String file : files) { |
| 54 | + try { |
| 55 | + estimatedSize += directory.fileLength(file); |
| 56 | + } catch (NoSuchFileException | FileNotFoundException | AccessDeniedException e) { |
| 57 | + // ignore, the file is not there no more; on Windows, if one thread concurrently deletes a file while |
| 58 | + // calling Files.size, you can also sometimes hit AccessDeniedException |
| 59 | + } |
| 60 | + } |
| 61 | + return estimatedSize; |
| 62 | + } |
| 63 | + |
| 64 | + private final SingleObjectCache<SizeAndModCount> size; |
| 65 | + // Both these variables need to be accessed under `this` lock. |
| 66 | + private long modCount = 0; |
| 67 | + private long numOpenOutputs = 0; |
| 68 | + |
| 69 | + ByteSizeCachingDirectory(Directory in, TimeValue refreshInterval) { |
| 70 | + super(in); |
| 71 | + size = new SingleObjectCache<SizeAndModCount>(refreshInterval, new SizeAndModCount(0L, -1L, true)) { |
| 72 | + @Override |
| 73 | + protected SizeAndModCount refresh() { |
| 74 | + // It is ok for the size of the directory to be more recent than |
| 75 | + // the mod count, we would just recompute the size of the |
| 76 | + // directory on the next call as well. However the opposite |
| 77 | + // would be bad as we would potentially have a stale cache |
| 78 | + // entry for a long time. So we fetch the values of modCount and |
| 79 | + // numOpenOutputs BEFORE computing the size of the directory. |
| 80 | + final long modCount; |
| 81 | + final boolean pendingWrite; |
| 82 | + synchronized(ByteSizeCachingDirectory.this) { |
| 83 | + modCount = ByteSizeCachingDirectory.this.modCount; |
| 84 | + pendingWrite = ByteSizeCachingDirectory.this.numOpenOutputs != 0; |
| 85 | + } |
| 86 | + final long size; |
| 87 | + try { |
| 88 | + // Compute this OUTSIDE of the lock |
| 89 | + size = estimateSizeInBytes(getDelegate()); |
| 90 | + } catch (IOException e) { |
| 91 | + throw new UncheckedIOException(e); |
| 92 | + } |
| 93 | + return new SizeAndModCount(size, modCount, pendingWrite); |
| 94 | + } |
| 95 | + |
| 96 | + @Override |
| 97 | + protected boolean needsRefresh() { |
| 98 | + if (super.needsRefresh() == false) { |
| 99 | + // The size was computed recently, don't recompute |
| 100 | + return false; |
| 101 | + } |
| 102 | + SizeAndModCount cached = getNoRefresh(); |
| 103 | + if (cached.pendingWrite) { |
| 104 | + // The cached entry was generated while there were pending |
| 105 | + // writes, so the size might be stale: recompute. |
| 106 | + return true; |
| 107 | + } |
| 108 | + synchronized(ByteSizeCachingDirectory.this) { |
| 109 | + // If there are pending writes or if new files have been |
| 110 | + // written/deleted since last time: recompute |
| 111 | + return numOpenOutputs != 0 || cached.modCount != modCount; |
| 112 | + } |
| 113 | + } |
| 114 | + }; |
| 115 | + } |
| 116 | + |
| 117 | + /** Return the cumulative size of all files in this directory. */ |
| 118 | + long estimateSizeInBytes() throws IOException { |
| 119 | + try { |
| 120 | + return size.getOrRefresh().size; |
| 121 | + } catch (UncheckedIOException e) { |
| 122 | + // we wrapped in the cache and unwrap here |
| 123 | + throw e.getCause(); |
| 124 | + } |
| 125 | + } |
| 126 | + |
| 127 | + @Override |
| 128 | + public IndexOutput createOutput(String name, IOContext context) throws IOException { |
| 129 | + return wrapIndexOutput(super.createOutput(name, context)); |
| 130 | + } |
| 131 | + |
| 132 | + @Override |
| 133 | + public IndexOutput createTempOutput(String prefix, String suffix, IOContext context) throws IOException { |
| 134 | + return wrapIndexOutput(super.createTempOutput(prefix, suffix, context)); |
| 135 | + } |
| 136 | + |
| 137 | + private IndexOutput wrapIndexOutput(IndexOutput out) { |
| 138 | + synchronized (this) { |
| 139 | + numOpenOutputs++; |
| 140 | + } |
| 141 | + return new FilterIndexOutput(out.toString(), out) { |
| 142 | + @Override |
| 143 | + public void writeBytes(byte[] b, int length) throws IOException { |
| 144 | + // Don't write to atomicXXX here since it might be called in |
| 145 | + // tight loops and memory barriers are costly |
| 146 | + super.writeBytes(b, length); |
| 147 | + } |
| 148 | + |
| 149 | + @Override |
| 150 | + public void writeByte(byte b) throws IOException { |
| 151 | + // Don't write to atomicXXX here since it might be called in |
| 152 | + // tight loops and memory barriers are costly |
| 153 | + super.writeByte(b); |
| 154 | + } |
| 155 | + |
| 156 | + @Override |
| 157 | + public void close() throws IOException { |
| 158 | + // Close might cause some data to be flushed from in-memory buffers, so |
| 159 | + // increment the modification counter too. |
| 160 | + try { |
| 161 | + super.close(); |
| 162 | + } finally { |
| 163 | + synchronized (this) { |
| 164 | + numOpenOutputs--; |
| 165 | + modCount++; |
| 166 | + } |
| 167 | + } |
| 168 | + } |
| 169 | + }; |
| 170 | + } |
| 171 | + |
| 172 | + @Override |
| 173 | + public void deleteFile(String name) throws IOException { |
| 174 | + try { |
| 175 | + super.deleteFile(name); |
| 176 | + } finally { |
| 177 | + synchronized (this) { |
| 178 | + modCount++; |
| 179 | + } |
| 180 | + } |
| 181 | + } |
| 182 | + |
| 183 | +} |
0 commit comments