[STORE] Move to on data.path per shard

This commit moves away from using stripe RAID-0 simumlation across multiple
data paths towards using a single path per shard. Multiple data paths are still
supported but shards and it's data is not striped across multiple paths / disks.
This will for instance prevent to loose all shards if a single disk is corrupted.

Indices that are using this features already will automatically upgraded to a single
datapath based on a simple diskspace based heuristic. In general there must be enough
diskspace to move a single shard at any time otherwise the upgrade will fail.

Closes #9498
This commit is contained in:
Simon Willnauer 2015-04-20 16:12:38 +02:00
parent 91afe64df7
commit 5730c06af9
56 changed files with 1236 additions and 1823 deletions

View File

@ -0,0 +1,375 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch 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.elasticsearch.common.util;
import com.google.common.base.Charsets;
import com.google.common.collect.Sets;
import com.google.common.primitives.Ints;
import org.apache.lucene.index.CheckIndex;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.Lock;
import org.apache.lucene.store.SimpleFSDirectory;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.ElasticsearchIllegalStateException;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.io.FileSystemUtils;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.env.ShardLock;
import org.elasticsearch.gateway.MetaDataStateFormat;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.shard.*;
import java.io.IOException;
import java.io.PrintStream;
import java.nio.file.*;
import java.nio.file.attribute.BasicFileAttributes;
import java.util.*;
/**
*/
public class MultiDataPathUpgrader {
private final NodeEnvironment nodeEnvironment;
private final ESLogger logger = Loggers.getLogger(getClass());
/**
* Creates a new upgrader instance
* @param nodeEnvironment the node env to operate on.
*
*/
public MultiDataPathUpgrader(NodeEnvironment nodeEnvironment) {
this.nodeEnvironment = nodeEnvironment;
}
/**
* Upgrades the given shard Id from multiple shard paths into the given target path.
*
* @see #pickShardPath(org.elasticsearch.index.shard.ShardId)
*/
public void upgrade(ShardId shard, ShardPath targetPath) throws IOException {
final Path[] paths = nodeEnvironment.availableShardPaths(shard); // custom data path doesn't need upgrading
if (isTargetPathConfigured(paths, targetPath) == false) {
throw new IllegalArgumentException("shard path must be one of the shards data paths");
}
assert needsUpgrading(shard) : "Should not upgrade a path that needs no upgrading";
logger.info("{} upgrading multi data dir to {}", shard, targetPath.getDataPath());
final ShardStateMetaData loaded = ShardStateMetaData.FORMAT.loadLatestState(logger, paths);
if (loaded == null) {
throw new IllegalStateException(shard + " no shard state found in any of: " + Arrays.toString(paths) + " please check and remove them if possible");
}
logger.info("{} loaded shard state {}", shard, loaded);
ShardStateMetaData.FORMAT.write(loaded, loaded.version, targetPath.getShardStatePath());
Files.createDirectories(targetPath.resolveIndex());
try (SimpleFSDirectory directory = new SimpleFSDirectory(targetPath.resolveIndex())) {
try (final Lock lock = directory.makeLock(IndexWriter.WRITE_LOCK_NAME)) {
if (lock.obtain(5000)) {
upgradeFiles(shard, targetPath, targetPath.resolveIndex(), ShardPath.INDEX_FOLDER_NAME, paths);
} else {
throw new IllegalStateException("Can't obtain lock on " + targetPath.resolveIndex());
}
}
}
upgradeFiles(shard, targetPath, targetPath.resolveTranslog(), ShardPath.TRANSLOG_FOLDER_NAME, paths);
logger.info("{} wipe upgraded directories", shard);
for (Path path : paths) {
if (path.equals(targetPath.getShardStatePath()) == false) {
logger.info("{} wipe shard directories: [{}]", shard, path);
IOUtils.rm(path);
}
}
if (FileSystemUtils.files(targetPath.resolveIndex()).length == 0) {
throw new IllegalStateException("index folder [" + targetPath.resolveIndex() + "] is empty");
}
if (FileSystemUtils.files(targetPath.resolveTranslog()).length == 0) {
throw new IllegalStateException("translog folder [" + targetPath.resolveTranslog() + "] is empty");
}
}
/**
* Runs check-index on the target shard and throws an exception if it failed
*/
public void checkIndex(ShardPath targetPath) throws IOException {
BytesStreamOutput os = new BytesStreamOutput();
PrintStream out = new PrintStream(os, false, Charsets.UTF_8.name());
try (Directory directory = new SimpleFSDirectory(targetPath.resolveIndex());
final CheckIndex checkIndex = new CheckIndex(directory)) {
checkIndex.setInfoStream(out);
CheckIndex.Status status = checkIndex.checkIndex();
out.flush();
if (!status.clean) {
logger.warn("check index [failure]\n{}", new String(os.bytes().toBytes(), Charsets.UTF_8));
throw new ElasticsearchIllegalStateException("index check failure");
}
}
}
/**
* Returns true iff the given shard needs upgrading.
*/
public boolean needsUpgrading(ShardId shard) {
final Path[] paths = nodeEnvironment.availableShardPaths(shard);
// custom data path doesn't need upgrading neither single path envs
if (paths.length > 1) {
int numPathsExist = 0;
for (Path path : paths) {
if (Files.exists(path.resolve(MetaDataStateFormat.STATE_DIR_NAME))) {
numPathsExist++;
if (numPathsExist > 1) {
return true;
}
}
}
}
return false;
}
/**
* Picks a target ShardPath to allocate and upgrade the given shard to. It picks the target based on a simple
* heuristic:
* <ul>
* <li>if the smallest datapath has 2x more space available that the shards total size the datapath with the most bytes for that shard is picked to minimize the amount of bytes to copy</li>
* <li>otherwise the largest available datapath is used as the target no matter how big of a slice of the shard it already holds.</li>
* </ul>
*/
public ShardPath pickShardPath(ShardId shard) throws IOException {
if (needsUpgrading(shard) == false) {
throw new IllegalStateException("Shard doesn't need upgrading");
}
final NodeEnvironment.NodePath[] paths = nodeEnvironment.nodePaths();
// if we need upgrading make sure we have all paths.
for (NodeEnvironment.NodePath path : paths) {
Files.createDirectories(path.resolve(shard));
}
final ShardFileInfo[] shardFileInfo = getShardFileInfo(shard, paths);
long totalBytesUsedByShard = 0;
long leastUsableSpace = Long.MAX_VALUE;
long mostUsableSpace = Long.MIN_VALUE;
assert shardFileInfo.length == nodeEnvironment.availableShardPaths(shard).length;
for (ShardFileInfo info : shardFileInfo) {
totalBytesUsedByShard += info.spaceUsedByShard;
leastUsableSpace = Math.min(leastUsableSpace, info.usableSpace + info.spaceUsedByShard);
mostUsableSpace = Math.max(mostUsableSpace, info.usableSpace + info.spaceUsedByShard);
}
if (mostUsableSpace < totalBytesUsedByShard) {
throw new IllegalStateException("Can't upgrade path available space: " + new ByteSizeValue(mostUsableSpace) + " required space: " + new ByteSizeValue(totalBytesUsedByShard));
}
ShardFileInfo target = shardFileInfo[0];
if (leastUsableSpace >= (2 * totalBytesUsedByShard)) {
for (ShardFileInfo info : shardFileInfo) {
if (info.spaceUsedByShard > target.spaceUsedByShard) {
target = info;
}
}
} else {
for (ShardFileInfo info : shardFileInfo) {
if (info.usableSpace > target.usableSpace) {
target = info;
}
}
}
return new ShardPath(target.path, target.path, IndexMetaData.INDEX_UUID_NA_VALUE /* we don't know */, shard);
}
private ShardFileInfo[] getShardFileInfo(ShardId shard, NodeEnvironment.NodePath[] paths) throws IOException {
final ShardFileInfo[] info = new ShardFileInfo[paths.length];
for (int i = 0; i < info.length; i++) {
Path path = paths[i].resolve(shard);
final long usabelSpace = getUsabelSpace(paths[i]);
info[i] = new ShardFileInfo(path, usabelSpace, getSpaceUsedByShard(path));
}
return info;
}
protected long getSpaceUsedByShard(Path path) throws IOException {
final long[] spaceUsedByShard = new long[] {0};
if (Files.exists(path)) {
Files.walkFileTree(path, new FileVisitor<Path>() {
@Override
public FileVisitResult preVisitDirectory(Path dir, BasicFileAttributes attrs) throws IOException {
return FileVisitResult.CONTINUE;
}
@Override
public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException {
if (attrs.isRegularFile()) {
spaceUsedByShard[0] += attrs.size();
}
return FileVisitResult.CONTINUE;
}
@Override
public FileVisitResult visitFileFailed(Path file, IOException exc) throws IOException {
return FileVisitResult.CONTINUE;
}
@Override
public FileVisitResult postVisitDirectory(Path dir, IOException exc) throws IOException {
return FileVisitResult.CONTINUE;
}
});
}
return spaceUsedByShard[0];
}
protected long getUsabelSpace(NodeEnvironment.NodePath path) throws IOException {
FileStore fileStore = path.fileStore;
return fileStore.getUsableSpace();
}
static class ShardFileInfo {
final Path path;
final long usableSpace;
final long spaceUsedByShard;
ShardFileInfo(Path path, long usableSpace, long spaceUsedByShard) {
this.path = path;
this.usableSpace = usableSpace;
this.spaceUsedByShard = spaceUsedByShard;
}
}
private void upgradeFiles(ShardId shard, ShardPath targetPath, final Path targetDir, String folderName, Path[] paths) throws IOException {
List<Path> movedFiles = new ArrayList<>();
for (Path path : paths) {
if (path.equals(targetPath.getDataPath()) == false) {
final Path sourceDir = path.resolve(folderName);
if (Files.exists(sourceDir)) {
logger.info("{} upgrading [{}] from [{}] to [{}]", shard, folderName, sourceDir, targetDir);
try (DirectoryStream<Path> stream = Files.newDirectoryStream(sourceDir)) {
Files.createDirectories(targetDir);
for (Path file : stream) {
if (IndexWriter.WRITE_LOCK_NAME.equals(file.getFileName().toString()) || Files.isDirectory(file)) {
continue; // skip write.lock
}
logger.info("{} move file [{}] size: [{}]", shard, file.getFileName(), Files.size(file));
final Path targetFile = targetDir.resolve(file.getFileName());
/* We are pessimistic and do a copy first to the other path and then and atomic move to rename it such that
in the worst case the file exists twice but is never lost or half written.*/
final Path targetTempFile = Files.createTempFile(targetDir, "upgrade_", "_" + file.getFileName().toString());
Files.copy(file, targetTempFile, StandardCopyOption.COPY_ATTRIBUTES, StandardCopyOption.REPLACE_EXISTING);
Files.move(targetTempFile, targetFile, StandardCopyOption.ATOMIC_MOVE); // we are on the same FS - this must work otherwise all bets are off
Files.delete(file);
movedFiles.add(targetFile);
}
}
}
}
}
if (movedFiles.isEmpty() == false) {
// fsync later it might be on disk already
logger.info("{} fsync files", shard);
for (Path moved : movedFiles) {
logger.info("{} syncing [{}]", shard, moved.getFileName());
IOUtils.fsync(moved, false);
}
logger.info("{} syncing directory [{}]", shard, targetDir);
IOUtils.fsync(targetDir, true);
}
}
/**
* Returns <code>true</code> iff the target path is one of the given paths.
*/
private boolean isTargetPathConfigured(final Path[] paths, ShardPath targetPath) {
for (Path path : paths) {
if (path.equals(targetPath.getDataPath())) {
return true;
}
}
return false;
}
/**
* Runs an upgrade on all shards located under the given node environment if there is more than 1 data.path configured
* otherwise this method will return immediately.
*/
public static void upgradeMultiDataPath(NodeEnvironment nodeEnv, ESLogger logger) throws IOException {
if (nodeEnv.nodeDataPaths().length > 1) {
final MultiDataPathUpgrader upgrader = new MultiDataPathUpgrader(nodeEnv);
final Set<String> allIndices = nodeEnv.findAllIndices();
for (String index : allIndices) {
for (ShardId shardId : findAllShardIds(nodeEnv.indexPaths(new Index(index)))) {
try (ShardLock lock = nodeEnv.shardLock(shardId, 0)) {
if (upgrader.needsUpgrading(shardId)) {
final ShardPath shardPath = upgrader.pickShardPath(shardId);
upgrader.upgrade(shardId, shardPath);
// we have to check if the index path exists since we might
// have only upgraded the shard state that is written under /indexname/shardid/_state
// in the case we upgraded a dedicated index directory index
if (Files.exists(shardPath.resolveIndex())) {
upgrader.checkIndex(shardPath);
}
} else {
logger.debug("{} no upgrade needed - already upgraded");
}
}
}
}
}
}
private static Set<ShardId> findAllShardIds(Path... locations) throws IOException {
final Set<ShardId> shardIds = Sets.newHashSet();
for (final Path location : locations) {
if (Files.isDirectory(location)) {
shardIds.addAll(findAllShardsForIndex(location));
}
}
return shardIds;
}
private static Set<ShardId> findAllShardsForIndex(Path indexPath) throws IOException {
Set<ShardId> shardIds = new HashSet<>();
if (Files.isDirectory(indexPath)) {
try (DirectoryStream<Path> stream = Files.newDirectoryStream(indexPath)) {
String currentIndex = indexPath.getFileName().toString();
for (Path shardPath : stream) {
if (Files.isDirectory(shardPath)) {
Integer shardId = Ints.tryParse(shardPath.getFileName().toString());
if (shardId != null) {
ShardId id = new ShardId(currentIndex, shardId);
shardIds.add(id);
}
}
}
}
}
return shardIds;
}
}

View File

@ -21,8 +21,6 @@ package org.elasticsearch.env;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Sets;
import com.google.common.primitives.Ints;
import org.apache.lucene.store.*;
import org.apache.lucene.util.Constants;
import org.apache.lucene.util.IOUtils;
@ -30,14 +28,12 @@ import org.elasticsearch.ElasticsearchIllegalArgumentException;
import org.elasticsearch.ElasticsearchIllegalStateException;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.FileSystemUtils;
import org.elasticsearch.common.io.PathUtils;
import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.shard.ShardId;
@ -86,6 +82,28 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
}
this.spins = spins;
}
/**
* Resolves the given shards directory against this NodePath
*/
public Path resolve(ShardId shardId) {
return resolve(shardId.index()).resolve(Integer.toString(shardId.id()));
}
/**
* Resolves the given indexes directory against this NodePath
*/
public Path resolve(Index index) {
return indicesPath.resolve(index.name());
}
@Override
public String toString() {
return "NodePath{" +
"path=" + path +
", spins=" + spins +
'}';
}
}
private final NodePath[] nodePaths;
@ -313,7 +331,7 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
public void deleteShardDirectorySafe(ShardId shardId, @IndexSettings Settings indexSettings) throws IOException {
// This is to ensure someone doesn't use ImmutableSettings.EMPTY
assert indexSettings != ImmutableSettings.EMPTY;
final Path[] paths = shardPaths(shardId);
final Path[] paths = availableShardPaths(shardId);
logger.trace("deleting shard {} directory, paths: [{}]", shardId, paths);
try (ShardLock lock = shardLock(shardId)) {
deleteShardDirectoryUnderLock(lock, indexSettings);
@ -330,7 +348,7 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
assert indexSettings != ImmutableSettings.EMPTY;
final ShardId shardId = lock.getShardId();
assert isShardLocked(shardId) : "shard " + shardId + " is not locked";
final Path[] paths = shardPaths(shardId);
final Path[] paths = availableShardPaths(shardId);
IOUtils.rm(paths);
if (hasCustomDataPath(indexSettings)) {
Path customLocation = resolveCustomLocation(indexSettings, shardId);
@ -575,7 +593,7 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
}
/**
* Returns an array of all of the {@link #NodePath}s.
* Returns an array of all of the {@link NodePath}s.
*/
public NodePath[] nodePaths() {
assert assertEnvIsLocked();
@ -598,36 +616,24 @@ public class NodeEnvironment extends AbstractComponent implements Closeable {
}
/**
* Returns all paths where lucene data will be stored, if a index.data_path
* setting is present, will return the custom data path to be used
* Returns all shard paths excluding custom shard path. Note: Shards are only allocated on one of the
* returned paths. The returned array may contain paths to non-existing directories.
*
* @see #hasCustomDataPath(org.elasticsearch.common.settings.Settings)
* @see #resolveCustomLocation(org.elasticsearch.common.settings.Settings, org.elasticsearch.index.shard.ShardId)
*
*/
public Path[] shardDataPaths(ShardId shardId, @IndexSettings Settings indexSettings) {
assert indexSettings != ImmutableSettings.EMPTY;
assert assertEnvIsLocked();
if (hasCustomDataPath(indexSettings)) {
return new Path[] {resolveCustomLocation(indexSettings, shardId)};
} else {
return shardPaths(shardId);
}
}
/**
* Returns all shard paths excluding custom shard path
*/
public Path[] shardPaths(ShardId shardId) {
public Path[] availableShardPaths(ShardId shardId) {
assert assertEnvIsLocked();
final NodePath[] nodePaths = nodePaths();
final Path[] shardLocations = new Path[nodePaths.length];
for (int i = 0; i < nodePaths.length; i++) {
// TODO: wtf with resolve(get())
shardLocations[i] = nodePaths[i].path.resolve(PathUtils.get(INDICES_FOLDER,
shardId.index().name(),
Integer.toString(shardId.id())));
shardLocations[i] = nodePaths[i].resolve(shardId);
}
return shardLocations;
}
public Set<String> findAllIndices() throws Exception {
public Set<String> findAllIndices() throws IOException {
if (nodePaths == null || locks == null) {
throw new ElasticsearchIllegalStateException("node is not configured to store local location");
}

View File

@ -28,7 +28,6 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.env.NodeEnvironment;
import java.util.List;
import java.util.Map;
import java.util.Set;

View File

@ -28,6 +28,7 @@ import com.google.common.collect.Sets;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.FailedNodeException;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.MutableShardRouting;
@ -106,6 +107,7 @@ public class GatewayAllocator extends AbstractComponent {
RoutingNodes routingNodes = allocation.routingNodes();
// First, handle primaries, they must find a place to be allocated on here
MetaData metaData = routingNodes.metaData();
Iterator<MutableShardRouting> unassignedIterator = routingNodes.unassigned().iterator();
while (unassignedIterator.hasNext()) {
MutableShardRouting shard = unassignedIterator.next();
@ -118,8 +120,8 @@ public class GatewayAllocator extends AbstractComponent {
if (!routingNodes.routingTable().index(shard.index()).shard(shard.id()).primaryAllocatedPostApi()) {
continue;
}
final String indexUUID = allocation.metaData().index(shard.index()).getUUID();
ObjectLongOpenHashMap<DiscoveryNode> nodesState = buildShardStates(nodes, shard, indexUUID);
ObjectLongOpenHashMap<DiscoveryNode> nodesState = buildShardStates(nodes, shard, metaData.index(shard.index()));
int numberOfAllocationsFound = 0;
long highestVersion = -1;
@ -370,7 +372,7 @@ public class GatewayAllocator extends AbstractComponent {
return changed;
}
private ObjectLongOpenHashMap<DiscoveryNode> buildShardStates(final DiscoveryNodes nodes, MutableShardRouting shard, String indexUUID) {
private ObjectLongOpenHashMap<DiscoveryNode> buildShardStates(final DiscoveryNodes nodes, MutableShardRouting shard, IndexMetaData indexMetaData) {
ObjectLongOpenHashMap<DiscoveryNode> shardStates = cachedShardsState.get(shard.shardId());
ObjectOpenHashSet<String> nodeIds;
if (shardStates == null) {
@ -399,7 +401,7 @@ public class GatewayAllocator extends AbstractComponent {
}
String[] nodesIdsArray = nodeIds.toArray(String.class);
TransportNodesListGatewayStartedShards.NodesGatewayStartedShards response = listGatewayStartedShards.list(shard.shardId(), indexUUID, nodesIdsArray, listTimeout).actionGet();
TransportNodesListGatewayStartedShards.NodesGatewayStartedShards response = listGatewayStartedShards.list(shard.shardId(), indexMetaData.getUUID(), nodesIdsArray, listTimeout).actionGet();
logListActionFailures(shard, "state", response.failures());
for (TransportNodesListGatewayStartedShards.NodeGatewayStartedShards nodeShardState : response) {

View File

@ -37,7 +37,7 @@ import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.common.util.MultiDataPathUpgrader;
import org.elasticsearch.env.NodeEnvironment;
import java.io.IOException;
@ -71,6 +71,7 @@ public class GatewayMetaState extends AbstractComponent implements ClusterStateL
if (DiscoveryNode.dataNode(settings)) {
ensureNoPre019ShardState(nodeEnv);
MultiDataPathUpgrader.upgradeMultiDataPath(nodeEnv, logger);
}
if (DiscoveryNode.masterNode(settings) || DiscoveryNode.dataNode(settings)) {

View File

@ -117,7 +117,7 @@ public class TransportNodesListGatewayStartedShards extends TransportNodesOperat
final ShardId shardId = request.getShardId();
final String indexUUID = request.getIndexUUID();
logger.trace("{} loading local shard state info", shardId);
final ShardStateMetaData shardStateMetaData = ShardStateMetaData.FORMAT.loadLatestState(logger, nodeEnv.shardPaths(request.shardId));
ShardStateMetaData shardStateMetaData = ShardStateMetaData.FORMAT.loadLatestState(logger, nodeEnv.availableShardPaths(request.shardId));
if (shardStateMetaData != null) {
// old shard metadata doesn't have the actual index UUID so we need to check if the actual uuid in the metadata
// is equal to IndexMetaData.INDEX_UUID_NA_VALUE otherwise this shard doesn't belong to the requested index.
@ -155,6 +155,7 @@ public class TransportNodesListGatewayStartedShards extends TransportNodesOperat
this.indexUUID = indexUUID;
}
public ShardId shardId() {
return this.shardId;
}

View File

@ -58,10 +58,7 @@ import org.elasticsearch.index.query.IndexQueryParserService;
import org.elasticsearch.index.search.stats.ShardSearchModule;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.settings.IndexSettingsService;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.IndexShardCreationException;
import org.elasticsearch.index.shard.IndexShardModule;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.*;
import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.index.snapshots.IndexShardSnapshotModule;
import org.elasticsearch.index.store.IndexStore;
@ -296,6 +293,15 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
boolean success = false;
Injector shardInjector = null;
try {
ShardPath path = ShardPath.loadShardPath(logger, nodeEnv, shardId, indexSettings);
if (path == null) {
path = ShardPath.selectNewPathForShard(nodeEnv, shardId, indexSettings);
logger.debug("{} creating using a new path [{}]", shardId, path);
} else {
logger.debug("{} creating using an existing path [{}]", shardId, path);
}
lock = nodeEnv.shardLock(shardId, TimeUnit.SECONDS.toMillis(5));
if (shards.containsKey(shardId.id())) {
throw new IndexShardAlreadyExistsException(shardId + " already exists");
@ -313,8 +319,8 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
modules.add(new ShardIndexingModule());
modules.add(new ShardSearchModule());
modules.add(new ShardGetModule());
modules.add(new StoreModule(indexSettings, injector.getInstance(IndexStore.class), lock,
new StoreCloseListener(shardId, canDeleteShardContent)));
modules.add(new StoreModule(indexSettings, injector.getInstance(IndexStore.class).shardDirectory(), lock,
new StoreCloseListener(shardId, canDeleteShardContent), path));
modules.add(new DeletionPolicyModule(indexSettings));
modules.add(new MergePolicyModule(indexSettings));
modules.add(new MergeSchedulerModule(indexSettings));

View File

@ -19,7 +19,6 @@
package org.elasticsearch.index.gateway;
import com.google.common.collect.Sets;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.SegmentInfos;
@ -33,7 +32,6 @@ import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.CancellableThreads;
import org.elasticsearch.common.util.concurrent.FutureUtils;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.engine.EngineException;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.shard.AbstractIndexShardComponent;

View File

@ -123,8 +123,6 @@ import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import static org.elasticsearch.index.mapper.SourceToParse.source;
/**
*
*/
@ -195,6 +193,7 @@ public class IndexShard extends AbstractIndexShardComponent {
* This setting is realtime updateable.
*/
public static final String INDEX_FLUSH_ON_CLOSE = "index.flush_on_close";
private final ShardPath path;
@Inject
public IndexShard(ShardId shardId, IndexSettingsService indexSettingsService, IndicesLifecycle indicesLifecycle, Store store, MergeSchedulerProvider mergeScheduler, Translog translog,
@ -202,7 +201,7 @@ public class IndexShard extends AbstractIndexShardComponent {
ShardFilterCache shardFilterCache, ShardFieldData shardFieldData, PercolatorQueriesRegistry percolatorQueriesRegistry, ShardPercolateService shardPercolateService, CodecService codecService,
ShardTermVectorsService termVectorsService, IndexFieldDataService indexFieldDataService, IndexService indexService, ShardSuggestService shardSuggestService, ShardQueryCache shardQueryCache, ShardBitsetFilterCache shardBitsetFilterCache,
@Nullable IndicesWarmer warmer, SnapshotDeletionPolicy deletionPolicy, SimilarityService similarityService, MergePolicyProvider mergePolicyProvider, EngineFactory factory,
ClusterService clusterService, NodeEnvironment nodeEnv) {
ClusterService clusterService, NodeEnvironment nodeEnv, ShardPath path) {
super(shardId, indexSettingsService.getSettings());
this.codecService = codecService;
this.warmer = warmer;
@ -244,8 +243,8 @@ public class IndexShard extends AbstractIndexShardComponent {
this.flushOnClose = indexSettings.getAsBoolean(INDEX_FLUSH_ON_CLOSE, true);
this.nodeEnv = nodeEnv;
indexSettingsService.addListener(applyRefreshSettings);
this.mapperAnalyzer = new MapperAnalyzer(mapperService);
this.path = path;
/* create engine config */
logger.debug("state: [CREATED]");
@ -997,7 +996,10 @@ public class IndexShard extends AbstractIndexShardComponent {
if (this.routingEntry() != null && this.routingEntry().active()) {
throw new ElasticsearchIllegalStateException("Can't delete shard state on an active shard");
}
MetaDataStateFormat.deleteMetaState(nodeEnv.shardPaths(shardId));
MetaDataStateFormat.deleteMetaState(shardPath().getDataPath());
}
public ShardPath shardPath() {
return path;
}
private class ApplyRefreshSettings implements IndexSettingsService.Listener {
@ -1200,7 +1202,7 @@ public class IndexShard extends AbstractIndexShardComponent {
public void onFailedEngine(ShardId shardId, String reason, @Nullable Throwable failure) {
try {
// delete the shard state so this folder will not be reused
MetaDataStateFormat.deleteMetaState(nodeEnv.shardPaths(shardId));
MetaDataStateFormat.deleteMetaState(nodeEnv.availableShardPaths(shardId));
} catch (IOException e) {
logger.warn("failed to delete shard state", e);
} finally {
@ -1258,7 +1260,7 @@ public class IndexShard extends AbstractIndexShardComponent {
}
final ShardStateMetaData newShardStateMetadata = new ShardStateMetaData(newRouting.version(), newRouting.primary(), getIndexUUID());
logger.trace("{} writing shard state, reason [{}]", shardId, writeReason);
ShardStateMetaData.FORMAT.write(newShardStateMetadata, newShardStateMetadata.version, nodeEnv.shardPaths(shardId));
ShardStateMetaData.FORMAT.write(newShardStateMetadata, newShardStateMetadata.version, shardPath().getShardStatePath());
} catch (IOException e) { // this is how we used to handle it.... :(
logger.warn("failed to write shard state", e);
// we failed to write the shard state, we will try and write

View File

@ -20,11 +20,9 @@ package org.elasticsearch.index.shard;
import org.elasticsearch.ElasticsearchIllegalStateException;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.aliases.IndexAliasesService;
@ -48,7 +46,6 @@ import org.elasticsearch.index.percolator.PercolatorQueriesRegistry;
import org.elasticsearch.index.percolator.stats.ShardPercolateService;
import org.elasticsearch.index.query.IndexQueryParserService;
import org.elasticsearch.index.search.stats.ShardSearchService;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.settings.IndexSettingsService;
import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.index.store.Store;
@ -82,14 +79,14 @@ public final class ShadowIndexShard extends IndexShard {
IndexService indexService, ShardSuggestService shardSuggestService, ShardQueryCache shardQueryCache,
ShardBitsetFilterCache shardBitsetFilterCache, @Nullable IndicesWarmer warmer,
SnapshotDeletionPolicy deletionPolicy, SimilarityService similarityService,
MergePolicyProvider mergePolicyProvider, EngineFactory factory, ClusterService clusterService, NodeEnvironment nodeEnv) {
MergePolicyProvider mergePolicyProvider, EngineFactory factory, ClusterService clusterService, NodeEnvironment nodeEnv, ShardPath path) {
super(shardId, indexSettingsService, indicesLifecycle, store, mergeScheduler,
translog, threadPool, mapperService, queryParserService, indexCache, indexAliasesService,
indexingService, getService, searchService, shardWarmerService, shardFilterCache,
shardFieldData, percolatorQueriesRegistry, shardPercolateService, codecService,
termVectorsService, indexFieldDataService, indexService, shardSuggestService,
shardQueryCache, shardBitsetFilterCache, warmer, deletionPolicy, similarityService,
mergePolicyProvider, factory, clusterService, nodeEnv);
mergePolicyProvider, factory, clusterService, nodeEnv, path);
}
/**

View File

@ -0,0 +1,184 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch 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.elasticsearch.index.shard;
import org.elasticsearch.ElasticsearchIllegalStateException;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.index.settings.IndexSettings;
import java.io.IOException;
import java.nio.file.FileStore;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.List;
public final class ShardPath {
public static final String INDEX_FOLDER_NAME = "index";
public static final String TRANSLOG_FOLDER_NAME = "translog";
private final Path path;
private final String indexUUID;
private final ShardId shardId;
private final Path shardStatePath;
public ShardPath(Path path, Path shardStatePath, String indexUUID, ShardId shardId) {
this.path = path;
this.indexUUID = indexUUID;
this.shardId = shardId;
this.shardStatePath = shardStatePath;
}
public Path resolveTranslog() {
return path.resolve(TRANSLOG_FOLDER_NAME);
}
public Path resolveIndex() {
return path.resolve(INDEX_FOLDER_NAME);
}
public Path getDataPath() {
return path;
}
public boolean exists() {
return Files.exists(path);
}
public String getIndexUUID() {
return indexUUID;
}
public ShardId getShardId() {
return shardId;
}
public Path getShardStatePath() {
return shardStatePath;
}
/**
* This method walks through the nodes shard paths to find the data and state path for the given shard. If multiple
* directories with a valid shard state exist the one with the highest version will be used.
* <b>Note:</b> this method resolves custom data locations for the shard.
*/
public static ShardPath loadShardPath(ESLogger logger, NodeEnvironment env, ShardId shardId, @IndexSettings Settings indexSettings) throws IOException {
final String indexUUID = indexSettings.get(IndexMetaData.SETTING_UUID, IndexMetaData.INDEX_UUID_NA_VALUE);
final Path[] paths = env.availableShardPaths(shardId);
Path loadedPath = null;
for (Path path : paths) {
ShardStateMetaData load = ShardStateMetaData.FORMAT.loadLatestState(logger, path);
if (load != null) {
if ((load.indexUUID.equals(indexUUID) || IndexMetaData.INDEX_UUID_NA_VALUE.equals(load.indexUUID)) == false) {
throw new ElasticsearchIllegalStateException(shardId + " index UUID in shard state was: " + load.indexUUID + " excepted: " + indexUUID + " on shard path: " + path);
}
if (loadedPath == null) {
loadedPath = path;
} else{
throw new ElasticsearchIllegalStateException(shardId + " more than one shard state found");
}
}
}
if (loadedPath == null) {
return null;
} else {
final Path dataPath;
final Path statePath = loadedPath;
if (NodeEnvironment.hasCustomDataPath(indexSettings)) {
dataPath = env.resolveCustomLocation(indexSettings, shardId);
} else {
dataPath = statePath;
}
logger.debug("{} loaded data path [{}], state path [{}]", shardId, dataPath, statePath);
return new ShardPath(dataPath, statePath, indexUUID, shardId);
}
}
// TODO - do we need something more extensible? Yet, this does the job for now...
public static ShardPath selectNewPathForShard(NodeEnvironment env, ShardId shardId, @IndexSettings Settings indexSettings) throws IOException {
final String indexUUID = indexSettings.get(IndexMetaData.SETTING_UUID, IndexMetaData.INDEX_UUID_NA_VALUE);
final NodeEnvironment.NodePath[] paths = env.nodePaths();
final List<Tuple<Path, Long>> minUsedPaths = new ArrayList<>();
for (NodeEnvironment.NodePath nodePath : paths) {
final Path shardPath = nodePath.resolve(shardId);
FileStore fileStore = nodePath.fileStore;
long usableSpace = fileStore.getUsableSpace();
if (minUsedPaths.isEmpty() || minUsedPaths.get(0).v2() == usableSpace) {
minUsedPaths.add(new Tuple<>(shardPath, usableSpace));
} else if (minUsedPaths.get(0).v2() < usableSpace) {
minUsedPaths.clear();
minUsedPaths.add(new Tuple<>(shardPath, usableSpace));
}
}
Path minUsed = minUsedPaths.get(shardId.id() % minUsedPaths.size()).v1();
final Path dataPath;
final Path statePath = minUsed;
if (NodeEnvironment.hasCustomDataPath(indexSettings)) {
dataPath = env.resolveCustomLocation(indexSettings, shardId);
} else {
dataPath = statePath;
}
return new ShardPath(dataPath, statePath, indexUUID, shardId);
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
final ShardPath shardPath = (ShardPath) o;
if (shardId != null ? !shardId.equals(shardPath.shardId) : shardPath.shardId != null) {
return false;
}
if (indexUUID != null ? !indexUUID.equals(shardPath.indexUUID) : shardPath.indexUUID != null) {
return false;
}
if (path != null ? !path.equals(shardPath.path) : shardPath.path != null) {
return false;
}
return true;
}
@Override
public int hashCode() {
int result = path != null ? path.hashCode() : 0;
result = 31 * result + (indexUUID != null ? indexUUID.hashCode() : 0);
result = 31 * result + (shardId != null ? shardId.hashCode() : 0);
return result;
}
@Override
public String toString() {
return "ShardPath{" +
"path=" + path +
", indexUUID='" + indexUUID + '\'' +
", shard=" + shardId +
'}';
}
}

View File

@ -20,8 +20,6 @@
package org.elasticsearch.index.shard;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentType;
@ -30,8 +28,6 @@ import org.elasticsearch.gateway.MetaDataStateFormat;
import java.io.IOException;
import java.io.OutputStream;
import java.nio.file.Path;
import java.util.regex.Pattern;
/**
*/
@ -142,5 +138,4 @@ public final class ShardStateMetaData {
return new ShardStateMetaData(version, primary, indexUUID);
}
};
}

View File

@ -20,12 +20,10 @@
package org.elasticsearch.index.store;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FilterDirectory;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.shard.AbstractIndexShardComponent;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.store.distributor.Distributor;
import java.io.IOException;
@ -37,25 +35,7 @@ public abstract class DirectoryService extends AbstractIndexShardComponent {
super(shardId, indexSettings);
}
public abstract Directory[] build() throws IOException;
public abstract long throttleTimeInNanos();
/**
* Creates a new Directory from the given distributor.
* The default implementation returns a new {@link org.elasticsearch.index.store.DistributorDirectory}
* if there is more than one data path in the distributor.
*/
public Directory newFromDistributor(final Distributor distributor) throws IOException {
if (distributor.all().length == 1) {
// use filter dir for consistent toString methods
return new FilterDirectory(distributor.primary()) {
@Override
public String toString() {
return distributor.toString();
}
};
}
return new DistributorDirectory(distributor);
}
public abstract Directory newDirectory() throws IOException;
}

View File

@ -1,259 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch 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.elasticsearch.index.store;
import org.apache.lucene.store.*;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.common.math.MathUtils;
import org.elasticsearch.index.store.distributor.Distributor;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.IdentityHashMap;
import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;
/**
* A directory implementation that uses the Elasticsearch {@link Distributor} abstraction to distribute
* files across multiple data directories.
*/
public final class DistributorDirectory extends Directory {
private final Distributor distributor;
private final HashMap<String, Directory> nameDirMapping = new HashMap<>();
private boolean closed = false;
/**
* Creates a new DistributorDirectory from multiple directories. Note: The first directory in the given array
* is used as the primary directory holding the file locks as well as the SEGMENTS_GEN file. All remaining
* directories are used in a round robin fashion.
*/
public DistributorDirectory(final Directory... dirs) throws IOException {
this(new Distributor() {
final AtomicInteger count = new AtomicInteger();
@Override
public Directory primary() {
return dirs[0];
}
@Override
public Directory[] all() {
return dirs;
}
@Override
public synchronized Directory any() {
return dirs[MathUtils.mod(count.incrementAndGet(), dirs.length)];
}
});
}
/**
* Creates a new DistributorDirectory form the given Distributor.
*/
public DistributorDirectory(Distributor distributor) throws IOException {
this.distributor = distributor;
for (Directory dir : distributor.all()) {
for (String file : dir.listAll()) {
nameDirMapping.put(file, dir);
}
}
}
@Override
public synchronized final String[] listAll() throws IOException {
return nameDirMapping.keySet().toArray(new String[nameDirMapping.size()]);
}
@Override
public synchronized void deleteFile(String name) throws IOException {
getDirectory(name, true).deleteFile(name);
Directory remove = nameDirMapping.remove(name);
assert remove != null : "Tried to delete file " + name + " but couldn't";
}
@Override
public synchronized long fileLength(String name) throws IOException {
return getDirectory(name).fileLength(name);
}
@Override
public synchronized IndexOutput createOutput(String name, IOContext context) throws IOException {
return getDirectory(name, false).createOutput(name, context);
}
@Override
public void sync(Collection<String> names) throws IOException {
// no need to sync this operation it could be long running too
final Map<Directory, Collection<String>> perDirectory = new IdentityHashMap<>();
for (String name : names) {
final Directory dir = getDirectory(name);
Collection<String> dirNames = perDirectory.get(dir);
if (dirNames == null) {
dirNames = new ArrayList<>();
perDirectory.put(dir, dirNames);
}
dirNames.add(name);
}
for (Map.Entry<Directory, Collection<String>> entry : perDirectory.entrySet()) {
final Directory dir = entry.getKey();
final Collection<String> dirNames = entry.getValue();
dir.sync(dirNames);
}
}
@Override
public synchronized void renameFile(String source, String dest) throws IOException {
final Directory directory = getDirectory(source);
final Directory targetDir = nameDirMapping.get(dest);
if (targetDir != null && targetDir != directory) {
throw new IOException("Can't rename file from " + source
+ " to: " + dest + ": target file already exists in a different directory");
}
directory.renameFile(source, dest);
nameDirMapping.remove(source);
nameDirMapping.put(dest, directory);
}
@Override
public synchronized IndexInput openInput(String name, IOContext context) throws IOException {
return getDirectory(name).openInput(name, context);
}
@Override
public synchronized void close() throws IOException {
if (closed) {
return;
}
try {
assert assertConsistency();
} finally {
closed = true;
IOUtils.close(distributor.all());
}
}
/**
* Returns the directory that has previously been associated with this file name.
*
* @throws IOException if the name has not yet been associated with any directory ie. fi the file does not exists
*/
synchronized Directory getDirectory(String name) throws IOException { // pkg private for testing
return getDirectory(name, true);
}
/**
* Returns the directory that has previously been associated with this file name or associates the name with a directory
* if failIfNotAssociated is set to false.
*/
private synchronized Directory getDirectory(String name, boolean failIfNotAssociated) throws IOException {
final Directory directory = nameDirMapping.get(name);
if (directory == null) {
if (failIfNotAssociated) {
throw new FileNotFoundException("No such file [" + name + "]");
}
// Pick a directory and associate this new file with it:
final Directory dir = distributor.any();
assert nameDirMapping.containsKey(name) == false;
nameDirMapping.put(name, dir);
return dir;
}
return directory;
}
@Override
public synchronized String toString() {
return distributor.toString();
}
Distributor getDistributor() {
return distributor;
}
/**
* Basic checks to ensure the internal mapping is consistent - should only be used in assertions
*/
private synchronized boolean assertConsistency() throws IOException {
boolean consistent = true;
StringBuilder builder = new StringBuilder();
Directory[] all = distributor.all();
for (Directory d : all) {
for (String file : d.listAll()) {
final Directory directory = nameDirMapping.get(file);
if (directory == null) {
consistent = false;
builder.append("File ").append(file)
.append(" was not mapped to a directory but exists in one of the distributors directories")
.append(System.lineSeparator());
} else if (directory != d) {
consistent = false;
builder.append("File ").append(file).append(" was mapped to a directory ").append(directory)
.append(" but exists in another distributor directory ").append(d)
.append(System.lineSeparator());
}
}
}
assert consistent : builder.toString();
return consistent; // return boolean so it can be easily be used in asserts
}
@Override
public Lock makeLock(final String lockName) {
final Directory primary = distributor.primary();
final Lock delegateLock = primary.makeLock(lockName);
if (DirectoryUtils.getLeaf(primary, FSDirectory.class) != null) {
// Wrap the delegate's lock just so we can monitor when it actually wrote a lock file. We assume that an FSDirectory writes its
// locks as actual files (we don't support NoLockFactory):
return new Lock() {
@Override
public boolean obtain() throws IOException {
if (delegateLock.obtain()) {
synchronized(DistributorDirectory.this) {
assert nameDirMapping.containsKey(lockName) == false || nameDirMapping.get(lockName) == primary;
if (nameDirMapping.get(lockName) == null) {
nameDirMapping.put(lockName, primary);
}
}
return true;
} else {
return false;
}
}
@Override
public void close() throws IOException {
delegateLock.close();
}
@Override
public boolean isLocked() throws IOException {
return delegateLock.isLocked();
}
};
} else {
return delegateLock;
}
}
}

View File

@ -44,13 +44,4 @@ public interface IndexStore extends Closeable {
*/
Class<? extends DirectoryService> shardDirectory();
/**
* Return an array of all index folder locations for a given shard
*/
Path[] shardIndexLocations(ShardId shardId);
/**
* Return an array of all translog folder locations for a given shard
*/
Path[] shardTranslogLocations(ShardId shardId);
}

View File

@ -50,7 +50,6 @@ import org.elasticsearch.env.ShardLock;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.shard.AbstractIndexShardComponent;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.store.distributor.Distributor;
import java.io.*;
import java.nio.file.NoSuchFileException;
@ -106,18 +105,17 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref
}
};
public Store(ShardId shardId, @IndexSettings Settings indexSettings, DirectoryService directoryService, Distributor distributor, ShardLock shardLock) throws IOException {
this(shardId, indexSettings, directoryService, distributor, shardLock, OnClose.EMPTY);
public Store(ShardId shardId, @IndexSettings Settings indexSettings, DirectoryService directoryService, ShardLock shardLock) throws IOException {
this(shardId, indexSettings, directoryService, shardLock, OnClose.EMPTY);
}
@Inject
public Store(ShardId shardId, @IndexSettings Settings indexSettings, DirectoryService directoryService, Distributor distributor, ShardLock shardLock, OnClose onClose) throws IOException {
public Store(ShardId shardId, @IndexSettings Settings indexSettings, DirectoryService directoryService, ShardLock shardLock, OnClose onClose) throws IOException {
super(shardId, indexSettings);
this.directory = new StoreDirectory(directoryService.newFromDistributor(distributor), Loggers.getLogger("index.store.deletes", indexSettings, shardId));
this.directory = new StoreDirectory(directoryService.newDirectory(), Loggers.getLogger("index.store.deletes", indexSettings, shardId));
this.shardLock = shardLock;
this.onClose = onClose;
final TimeValue refreshInterval = indexSettings.getAsTime(INDEX_STORE_STATS_REFRESH_INTERVAL, TimeValue.timeValueSeconds(10));
this.statsCache = new StoreStatsCache(refreshInterval, directory, directoryService);
logger.debug("store stats are refreshed with refresh_interval [{}]", refreshInterval);
@ -365,21 +363,14 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref
*
* @throws IOException if the index we try to read is corrupted
*/
public static MetadataSnapshot readMetadataSnapshot(Path[] indexLocations, ESLogger logger) throws IOException {
final Directory[] dirs = new Directory[indexLocations.length];
try {
for (int i = 0; i < indexLocations.length; i++) {
dirs[i] = new SimpleFSDirectory(indexLocations[i]);
}
DistributorDirectory dir = new DistributorDirectory(dirs);
public static MetadataSnapshot readMetadataSnapshot(Path indexLocation, ESLogger logger) throws IOException {
try (Directory dir = new SimpleFSDirectory(indexLocation)){
failIfCorrupted(dir, new ShardId("", 1));
return new MetadataSnapshot(null, dir, logger);
} catch (IndexNotFoundException ex) {
// that's fine - happens all the time no need to log
} catch (FileNotFoundException | NoSuchFileException ex) {
logger.info("Failed to open / find files while reading metadata snapshot");
} finally {
IOUtils.close(dirs);
}
return MetadataSnapshot.EMPTY;
}

View File

@ -21,12 +21,8 @@ package org.elasticsearch.index.store;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.Callback;
import org.elasticsearch.env.ShardLock;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.store.distributor.Distributor;
import org.elasticsearch.index.store.distributor.LeastUsedDistributor;
import org.elasticsearch.index.store.distributor.RandomWeightedDistributor;
import org.elasticsearch.index.shard.ShardPath;
/**
*
@ -39,48 +35,26 @@ public class StoreModule extends AbstractModule {
private final Settings settings;
private final IndexStore indexStore;
private final ShardLock lock;
private final Store.OnClose closeCallback;
private final ShardPath path;
private final Class<? extends DirectoryService> shardDirectory;
private Class<? extends Distributor> distributor;
public StoreModule(Settings settings, IndexStore indexStore, ShardLock lock, Store.OnClose closeCallback) {
this.indexStore = indexStore;
public StoreModule(Settings settings, Class<? extends DirectoryService> shardDirectory, ShardLock lock, Store.OnClose closeCallback, ShardPath path) {
this.shardDirectory = shardDirectory;
this.settings = settings;
this.lock = lock;
this.closeCallback = closeCallback;
}
public void setDistributor(Class<? extends Distributor> distributor) {
this.distributor = distributor;
this.path = path;
}
@Override
protected void configure() {
bind(DirectoryService.class).to(indexStore.shardDirectory()).asEagerSingleton();
bind(DirectoryService.class).to(shardDirectory).asEagerSingleton();
bind(Store.class).asEagerSingleton();
bind(ShardLock.class).toInstance(lock);
bind(Store.OnClose.class).toInstance(closeCallback);
if (distributor == null) {
distributor = loadDistributor(settings);
}
bind(Distributor.class).to(distributor).asEagerSingleton();
bind(ShardPath.class).toInstance(path);
}
private Class<? extends Distributor> loadDistributor(Settings settings) {
final Class<? extends Distributor> distributor;
final String type = settings.get(DISTIBUTOR_KEY);
if ("least_used".equals(type)) {
distributor = LeastUsedDistributor.class;
} else if ("random".equals(type)) {
distributor = RandomWeightedDistributor.class;
} else {
distributor = settings.getAsClass(DISTIBUTOR_KEY, LeastUsedDistributor.class,
"org.elasticsearch.index.store.distributor.", "Distributor");
}
return distributor;
}
}

View File

@ -1,80 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch 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.elasticsearch.index.store.distributor;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FSDirectory;
import org.apache.lucene.store.StoreUtils;
import org.elasticsearch.index.store.DirectoryService;
import org.elasticsearch.index.store.DirectoryUtils;
import java.io.IOException;
import java.nio.file.FileStore;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.Arrays;
public abstract class AbstractDistributor implements Distributor {
protected final Directory[] delegates;
protected AbstractDistributor(DirectoryService directoryService) throws IOException {
delegates = directoryService.build();
}
@Override
public Directory[] all() {
return delegates;
}
@Override
public Directory primary() {
return delegates[0];
}
@Override
public Directory any() throws IOException {
if (delegates.length == 1) {
return delegates[0];
} else {
return doAny();
}
}
@SuppressWarnings("unchecked")
protected long getUsableSpace(Directory directory) throws IOException {
final FSDirectory leaf = DirectoryUtils.getLeaf(directory, FSDirectory.class);
if (leaf != null) {
return Files.getFileStore(leaf.getDirectory()).getUsableSpace();
} else {
return 0;
}
}
@Override
public String toString() {
return name() + StoreUtils.toString(delegates);
}
protected abstract Directory doAny() throws IOException;
protected abstract String name();
}

View File

@ -1,46 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch 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.elasticsearch.index.store.distributor;
import org.apache.lucene.store.Directory;
import java.io.IOException;
/**
* Keeps track of available directories and selects a directory
* based on some distribution strategy
*/
public interface Distributor {
/**
* Returns primary directory (typically first directory in the list)
*/
Directory primary();
/**
* Returns all directories
*/
Directory[] all();
/**
* Selects one of the directories based on distribution strategy
*/
Directory any() throws IOException;
}

View File

@ -1,67 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch 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.elasticsearch.index.store.distributor;
import org.apache.lucene.store.Directory;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.index.store.DirectoryService;
import java.io.IOException;
import java.util.concurrent.ThreadLocalRandom;
/**
* Implements directory distributor that always return the directory is the most available space
*/
public class LeastUsedDistributor extends AbstractDistributor {
@Inject
public LeastUsedDistributor(DirectoryService directoryService) throws IOException {
super(directoryService);
}
@Override
public Directory doAny() throws IOException {
Directory directory = null;
long size = Long.MIN_VALUE;
int sameSize = 0;
for (Directory delegate : delegates) {
long currentSize = getUsableSpace(delegate);
if (currentSize > size) {
size = currentSize;
directory = delegate;
sameSize = 1;
} else if (currentSize == size) {
sameSize++;
// Ensure uniform distribution between all directories with the same size
if (ThreadLocalRandom.current().nextDouble() < 1.0 / sameSize) {
directory = delegate;
}
}
}
return directory;
}
@Override
public String name() {
return "least_used";
}
}

View File

@ -1,68 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch 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.elasticsearch.index.store.distributor;
import org.apache.lucene.store.Directory;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.index.store.DirectoryService;
import java.io.IOException;
import java.util.concurrent.ThreadLocalRandom;
/**
* Implements directory distributor that picks a directory at random. The probability of selecting a directory
* is proportional to the amount of usable space in this directory.
*/
public class RandomWeightedDistributor extends AbstractDistributor {
@Inject
public RandomWeightedDistributor(DirectoryService directoryService) throws IOException {
super(directoryService);
}
@Override
public Directory doAny() throws IOException {
long[] usableSpace = new long[delegates.length];
long size = 0;
for (int i = 0; i < delegates.length; i++) {
size += getUsableSpace(delegates[i]);
usableSpace[i] = size;
}
if (size != 0) {
long random = ThreadLocalRandom.current().nextLong(size);
for (int i = 0; i < delegates.length; i++) {
if (usableSpace[i] > random) {
return delegates[i];
}
}
}
// TODO: size is 0 - should we bail out or fall back on random distribution?
return delegates[ThreadLocalRandom.current().nextInt(delegates.length)];
}
@Override
public String name() {
return "random";
}
}

View File

@ -25,6 +25,7 @@ import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardPath;
import org.elasticsearch.index.store.IndexStore;
import java.io.File;
@ -44,8 +45,8 @@ public class DefaultFsDirectoryService extends FsDirectoryService {
private static final Set<String> PRIMARY_EXTENSIONS = Collections.unmodifiableSet(Sets.newHashSet("dvd", "tim"));
@Inject
public DefaultFsDirectoryService(ShardId shardId, @IndexSettings Settings indexSettings, IndexStore indexStore) {
super(shardId, indexSettings, indexStore);
public DefaultFsDirectoryService(ShardId shardId, @IndexSettings Settings indexSettings, IndexStore indexStore, ShardPath shardPath) {
super(shardId, indexSettings, indexStore, shardPath);
}
@Override

View File

@ -28,6 +28,7 @@ import org.elasticsearch.common.metrics.CounterMetric;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardPath;
import org.elasticsearch.index.store.DirectoryService;
import org.elasticsearch.index.store.IndexStore;
import org.elasticsearch.index.store.StoreException;
@ -39,9 +40,11 @@ public abstract class FsDirectoryService extends DirectoryService implements Sto
protected final IndexStore indexStore;
private final CounterMetric rateLimitingTimeInNanos = new CounterMetric();
private final ShardPath path;
public FsDirectoryService(ShardId shardId, @IndexSettings Settings indexSettings, IndexStore indexStore) {
public FsDirectoryService(ShardId shardId, @IndexSettings Settings indexSettings, IndexStore indexStore, ShardPath path) {
super(shardId, indexSettings);
this.path = path;
this.indexStore = indexStore;
}
@ -68,19 +71,14 @@ public abstract class FsDirectoryService extends DirectoryService implements Sto
return lockFactory;
}
@Override
public Directory[] build() throws IOException {
Path[] locations = indexStore.shardIndexLocations(shardId);
Directory[] dirs = new Directory[locations.length];
for (int i = 0; i < dirs.length; i++) {
Files.createDirectories(locations[i]);
Directory wrapped = newFSDirectory(locations[i], buildLockFactory());
dirs[i] = new RateLimitedFSDirectory(wrapped, this, this) ;
}
return dirs;
public Directory newDirectory() throws IOException {
final Path location = path.resolveIndex();
Files.createDirectories(location);
Directory wrapped = newFSDirectory(location, buildLockFactory());
return new RateLimitedFSDirectory(wrapped, this, this) ;
}
protected abstract Directory newFSDirectory(Path location, LockFactory lockFactory) throws IOException;
@Override

View File

@ -26,6 +26,7 @@ import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardPath;
import org.elasticsearch.index.store.IndexStore;
import java.io.File;
@ -37,8 +38,8 @@ import java.nio.file.Path;
public class MmapFsDirectoryService extends FsDirectoryService {
@Inject
public MmapFsDirectoryService(ShardId shardId, @IndexSettings Settings indexSettings, IndexStore indexStore) {
super(shardId, indexSettings, indexStore);
public MmapFsDirectoryService(ShardId shardId, @IndexSettings Settings indexSettings, IndexStore indexStore, ShardPath shardPath) {
super(shardId, indexSettings, indexStore, shardPath);
}
@Override

View File

@ -26,6 +26,7 @@ import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardPath;
import org.elasticsearch.index.store.IndexStore;
import java.io.File;
@ -37,8 +38,8 @@ import java.nio.file.Path;
public class NioFsDirectoryService extends FsDirectoryService {
@Inject
public NioFsDirectoryService(ShardId shardId, @IndexSettings Settings indexSettings, IndexStore indexStore) {
super(shardId, indexSettings, indexStore);
public NioFsDirectoryService(ShardId shardId, @IndexSettings Settings indexSettings, IndexStore indexStore, ShardPath shardPath) {
super(shardId, indexSettings, indexStore, shardPath);
}
@Override

View File

@ -26,6 +26,7 @@ import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardPath;
import org.elasticsearch.index.store.IndexStore;
import java.io.File;
@ -37,8 +38,8 @@ import java.nio.file.Path;
public class SimpleFsDirectoryService extends FsDirectoryService {
@Inject
public SimpleFsDirectoryService(ShardId shardId, @IndexSettings Settings indexSettings, IndexStore indexStore) {
super(shardId, indexSettings, indexStore);
public SimpleFsDirectoryService(ShardId shardId, @IndexSettings Settings indexSettings, IndexStore indexStore, ShardPath shardPath) {
super(shardId, indexSettings, indexStore, shardPath);
}
@Override

View File

@ -46,9 +46,6 @@ public abstract class AbstractIndexStore extends AbstractIndexComponent implemen
public static final String INDEX_STORE_THROTTLE_TYPE = "index.store.throttle.type";
public static final String INDEX_STORE_THROTTLE_MAX_BYTES_PER_SEC = "index.store.throttle.max_bytes_per_sec";
public static final String INDEX_FOLDER_NAME = "index";
public static final String TRANSLOG_FOLDER_NAME = "translog";
class ApplySettings implements IndexSettingsService.Listener {
@Override
public void onRefreshSettings(Settings settings) {
@ -114,6 +111,7 @@ public abstract class AbstractIndexStore extends AbstractIndexComponent implemen
} else {
this.locations = null;
}
}
@Override
@ -125,36 +123,4 @@ public abstract class AbstractIndexStore extends AbstractIndexComponent implemen
public StoreRateLimiting rateLimiting() {
return nodeRateLimiting ? indicesStore.rateLimiting() : this.rateLimiting;
}
/**
* Return an array of all index folder locations for a given shard. Uses
* the index settings to determine if a custom data path is set for the
* index and uses that if applicable.
*/
@Override
public Path[] shardIndexLocations(ShardId shardId) {
Path[] shardLocations = nodeEnv.shardDataPaths(shardId, indexSettings);
Path[] locations = new Path[shardLocations.length];
for (int i = 0; i < shardLocations.length; i++) {
locations[i] = shardLocations[i].resolve(INDEX_FOLDER_NAME);
}
logger.debug("using [{}] as shard's index location", locations);
return locations;
}
/**
* Return an array of all translog folder locations for a given shard. Uses
* the index settings to determine if a custom data path is set for the
* index and uses that if applicable.
*/
@Override
public Path[] shardTranslogLocations(ShardId shardId) {
Path[] shardLocations = nodeEnv.shardDataPaths(shardId, indexSettings);
Path[] locations = new Path[shardLocations.length];
for (int i = 0; i < shardLocations.length; i++) {
locations[i] = shardLocations[i].resolve(TRANSLOG_FOLDER_NAME);
}
logger.debug("using [{}] as shard's translog location", locations);
return locations;
}
}

View File

@ -140,7 +140,7 @@ public interface Translog extends IndexShardComponent, Closeable, Accountable {
* These paths don't contain actual translog files they are
* directories holding the transaction logs.
*/
public Path[] locations();
public Path location();
/**
* Returns the translog filename for the given id.

View File

@ -38,6 +38,7 @@ import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.settings.IndexSettingsService;
import org.elasticsearch.index.shard.AbstractIndexShardComponent;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardPath;
import org.elasticsearch.index.store.IndexStore;
import org.elasticsearch.index.translog.*;
@ -78,7 +79,7 @@ public class FsTranslog extends AbstractIndexShardComponent implements Translog
private final BigArrays bigArrays;
private final ReadWriteLock rwl = new ReentrantReadWriteLock();
private final Path[] locations;
private final Path location;
private volatile FsTranslogFile current;
private volatile FsTranslogFile trans;
@ -96,26 +97,22 @@ public class FsTranslog extends AbstractIndexShardComponent implements Translog
@Inject
public FsTranslog(ShardId shardId, @IndexSettings Settings indexSettings, IndexSettingsService indexSettingsService,
BigArrays bigArrays, IndexStore indexStore) throws IOException {
BigArrays bigArrays, ShardPath shardPath) throws IOException {
super(shardId, indexSettings);
this.indexSettingsService = indexSettingsService;
this.bigArrays = bigArrays;
this.locations = indexStore.shardTranslogLocations(shardId);
for (Path location : locations) {
Files.createDirectories(location);
}
this.location = shardPath.resolveTranslog();
Files.createDirectories(location);
this.type = FsTranslogFile.Type.fromString(indexSettings.get("index.translog.fs.type", FsTranslogFile.Type.BUFFERED.name()));
this.bufferSize = (int) indexSettings.getAsBytesSize("index.translog.fs.buffer_size", ByteSizeValue.parseBytesSizeValue("64k")).bytes(); // Not really interesting, updated by IndexingMemoryController...
this.transientBufferSize = (int) indexSettings.getAsBytesSize("index.translog.fs.transient_buffer_size", ByteSizeValue.parseBytesSizeValue("8k")).bytes();
indexSettingsService.addListener(applySettings);
}
public FsTranslog(ShardId shardId, @IndexSettings Settings indexSettings, Path location) throws IOException {
super(shardId, indexSettings);
this.indexSettingsService = null;
this.locations = new Path[]{location};
this.location = location;
Files.createDirectories(location);
this.bigArrays = BigArrays.NON_RECYCLING_INSTANCE;
@ -155,8 +152,8 @@ public class FsTranslog extends AbstractIndexShardComponent implements Translog
}
@Override
public Path[] locations() {
return locations;
public Path location() {
return location;
}
@Override
@ -200,19 +197,15 @@ public class FsTranslog extends AbstractIndexShardComponent implements Translog
public int clearUnreferenced() {
rwl.writeLock().lock();
int deleted = 0;
try {
for (Path location : locations) {
try (DirectoryStream<Path> stream = Files.newDirectoryStream(location, TRANSLOG_FILE_PREFIX + "[0-9]*")) {
for (Path file : stream) {
if (isReferencedTranslogFile(file) == false) {
try {
logger.trace("delete unreferenced translog file: " + file);
Files.delete(file);
deleted++;
} catch (Exception ex) {
logger.debug("failed to delete " + file, ex);
}
}
try (DirectoryStream<Path> stream = Files.newDirectoryStream(location, TRANSLOG_FILE_PREFIX + "[0-9]*")) {
for (Path file : stream) {
if (isReferencedTranslogFile(file) == false) {
try {
logger.trace("delete unreferenced translog file: " + file);
Files.delete(file);
deleted++;
} catch (Exception ex) {
logger.debug("failed to delete " + file, ex);
}
}
}
@ -229,15 +222,6 @@ public class FsTranslog extends AbstractIndexShardComponent implements Translog
rwl.writeLock().lock();
try {
FsTranslogFile newFile;
long size = Long.MAX_VALUE;
Path location = null;
for (Path file : locations) {
long currentFree = Files.getFileStore(file).getUsableSpace();
if (currentFree < size) {
size = currentFree;
location = file;
}
}
try {
newFile = type.create(shardId, id, new InternalChannelReference(location.resolve(getFilename(id)), StandardOpenOption.READ, StandardOpenOption.WRITE, StandardOpenOption.CREATE_NEW), bufferSize);
} catch (IOException e) {
@ -256,15 +240,6 @@ public class FsTranslog extends AbstractIndexShardComponent implements Translog
rwl.writeLock().lock();
try {
assert this.trans == null;
long size = Long.MAX_VALUE;
Path location = null;
for (Path file : locations) {
long currentFree = Files.getFileStore(file).getUsableSpace();
if (currentFree < size) {
size = currentFree;
location = file;
}
}
this.trans = type.create(shardId, id, new InternalChannelReference(location.resolve(getFilename(id)), StandardOpenOption.READ, StandardOpenOption.WRITE, StandardOpenOption.CREATE_NEW), transientBufferSize);
} catch (IOException e) {
throw new TranslogException(shardId, "failed to create new translog file", e);
@ -450,18 +425,16 @@ public class FsTranslog extends AbstractIndexShardComponent implements Translog
rwl.readLock().lock();
try {
long maxId = this.currentId();
for (Path location : locations()) {
try (DirectoryStream<Path> stream = Files.newDirectoryStream(location, TRANSLOG_FILE_PREFIX + "[0-9]*")) {
for (Path translogFile : stream) {
try {
final String fileName = translogFile.getFileName().toString();
final Matcher matcher = PARSE_ID_PATTERN.matcher(fileName);
if (matcher.matches()) {
maxId = Math.max(maxId, Long.parseLong(matcher.group(1)));
}
} catch (NumberFormatException ex) {
logger.warn("Couldn't parse translog id from file " + translogFile + " skipping");
try (DirectoryStream<Path> stream = Files.newDirectoryStream(location, TRANSLOG_FILE_PREFIX + "[0-9]*")) {
for (Path translogFile : stream) {
try {
final String fileName = translogFile.getFileName().toString();
final Matcher matcher = PARSE_ID_PATTERN.matcher(fileName);
if (matcher.matches()) {
maxId = Math.max(maxId, Long.parseLong(matcher.group(1)));
}
} catch (NumberFormatException ex) {
logger.warn("Couldn't parse translog id from file " + translogFile + " skipping");
}
}
}
@ -475,17 +448,12 @@ public class FsTranslog extends AbstractIndexShardComponent implements Translog
public OperationIterator openIterator(long translogId) throws IOException {
final String translogName = getFilename(translogId);
Path recoveringTranslogFile = null;
logger.trace("try open translog file {} locations: {}", translogName, Arrays.toString(locations()));
OUTER:
for (Path translogLocation : locations()) {
// we have to support .recovering since it's a leftover from previous version but might still be on the filesystem
// we used to rename the foo into foo.recovering since foo was reused / overwritten but we fixed that in 2.0
for (Path recoveryFiles : FileSystemUtils.files(translogLocation, translogName + "{.recovering,}")) {
logger.trace("translog file found in {}", recoveryFiles);
recoveringTranslogFile = recoveryFiles;
break OUTER;
}
logger.trace("translog file NOT found in {} - continue", translogLocation);
logger.trace("try open translog file {} locations {}", translogName, location);
// we have to support .recovering since it's a leftover from previous version but might still be on the filesystem
// we used to rename the foo into foo.recovering since foo was reused / overwritten but we fixed that in 2.0
for (Path recoveryFiles : FileSystemUtils.files(location, translogName + "{.recovering,}")) {
logger.trace("translog file found in {}", recoveryFiles);
recoveringTranslogFile = recoveryFiles;
}
final boolean translogFileExists = recoveringTranslogFile != null && Files.exists(recoveringTranslogFile);
if (translogFileExists) {
@ -495,6 +463,7 @@ public class FsTranslog extends AbstractIndexShardComponent implements Translog
final TranslogStream translogStream = TranslogStreams.translogStreamFor(recoveringTranslogFile);
return new OperationIteratorImpl(logger, translogStream, translogStream.openInput(recoveringTranslogFile));
}
logger.trace("translog file NOT found in {}", location);
throw new FileNotFoundException("no translog file found for id: " + translogId);
}

View File

@ -80,7 +80,7 @@ import org.elasticsearch.plugins.PluginsService;
import java.io.Closeable;
import java.io.IOException;
import java.nio.file.Path;
import java.nio.file.Files;
import java.util.*;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
@ -587,8 +587,11 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
final IndexService indexService = indexServiceInjectorTuple.v1();
return indexService.hasShard(shardId.id()) == false;
} else if (nodeEnv.hasNodeFile()) {
final Path[] shardLocations = nodeEnv.shardDataPaths(shardId, indexSettings);
return FileSystemUtils.exists(shardLocations);
if (NodeEnvironment.hasCustomDataPath(indexSettings)) {
return Files.exists(nodeEnv.resolveCustomLocation(indexSettings, shardId));
} else {
return FileSystemUtils.exists(nodeEnv.availableShardPaths(shardId));
}
}
} else {
logger.trace("{} skipping shard directory deletion due to shadow replicas", shardId);

View File

@ -46,7 +46,6 @@ public class InternalIndicesLifecycle extends AbstractComponent implements Indic
public InternalIndicesLifecycle(Settings settings) {
super(settings);
}
@Override
public void addListener(Listener listener) {
listeners.add(listener);

View File

@ -33,7 +33,6 @@ import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.FileSystemUtils;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Streamable;
@ -43,6 +42,7 @@ import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardPath;
import org.elasticsearch.index.store.Store;
import org.elasticsearch.index.store.StoreFileMetaData;
import org.elasticsearch.indices.IndicesService;
@ -50,7 +50,6 @@ import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import java.io.IOException;
import java.nio.file.Path;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
@ -173,16 +172,11 @@ public class TransportNodesListShardStoreMetaData extends TransportNodesOperatio
if (!storeType.contains("fs")) {
return new StoreFilesMetaData(false, shardId, ImmutableMap.<String, StoreFileMetaData>of());
}
Path[] shardLocations = nodeEnv.shardDataPaths(shardId, metaData.settings());
Path[] shardIndexLocations = new Path[shardLocations.length];
for (int i = 0; i < shardLocations.length; i++) {
shardIndexLocations[i] = shardLocations[i].resolve("index");
}
exists = FileSystemUtils.exists(shardIndexLocations);
if (!exists) {
final ShardPath shardPath = ShardPath.loadShardPath(logger, nodeEnv, shardId, metaData.settings());
if (shardPath == null) {
return new StoreFilesMetaData(false, shardId, ImmutableMap.<String, StoreFileMetaData>of());
}
return new StoreFilesMetaData(false, shardId, Store.readMetadataSnapshot(shardIndexLocations, logger).asMap());
return new StoreFilesMetaData(false, shardId, Store.readMetadataSnapshot(shardPath.resolveIndex(), logger).asMap());
} finally {
TimeValue took = new TimeValue(System.currentTimeMillis() - startTime);
if (exists) {
@ -220,14 +214,6 @@ public class TransportNodesListShardStoreMetaData extends TransportNodesOperatio
return this.shardId;
}
public long totalSizeInBytes() {
long totalSizeInBytes = 0;
for (StoreFileMetaData file : this) {
totalSizeInBytes += file.length();
}
return totalSizeInBytes;
}
@Override
public Iterator<StoreFileMetaData> iterator() {
return files.values().iterator();

View File

@ -30,9 +30,11 @@ import org.elasticsearch.action.get.GetResponse;
import org.elasticsearch.action.search.SearchRequestBuilder;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.common.io.FileSystemUtils;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.MultiDataPathUpgrader;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.index.engine.EngineConfig;
@ -170,12 +172,15 @@ public class OldIndexBackwardsCompatibilityTests extends ElasticsearchIntegratio
if (randomBoolean()) {
logger.info("--> injecting index [{}] into single data path", indexName);
copyIndex(src, indexName, singleDataPath);
copyIndex(logger, src, indexName, singleDataPath);
} else {
logger.info("--> injecting index [{}] into multi data path", indexName);
copyIndex(src, indexName, multiDataPath);
copyIndex(logger, src, indexName, multiDataPath);
}
final Iterable<NodeEnvironment> instances = internalCluster().getInstances(NodeEnvironment.class);
for (NodeEnvironment nodeEnv : instances) { // upgrade multidata path
MultiDataPathUpgrader.upgradeMultiDataPath(nodeEnv, logger);
}
// force reloading dangling indices with a cluster state republish
client().admin().cluster().prepareReroute().get();
ensureGreen(indexName);
@ -183,7 +188,7 @@ public class OldIndexBackwardsCompatibilityTests extends ElasticsearchIntegratio
}
// randomly distribute the files from src over dests paths
void copyIndex(final Path src, final String indexName, final Path... dests) throws IOException {
public static void copyIndex(final ESLogger logger, final Path src, final String indexName, final Path... dests) throws IOException {
for (Path dest : dests) {
Path indexDir = dest.resolve(indexName);
assertFalse(Files.exists(indexDir));
@ -382,4 +387,5 @@ public class OldIndexBackwardsCompatibilityTests extends ElasticsearchIntegratio
UpgradeTest.runUpgrade(httpClient, indexName);
UpgradeTest.assertUpgraded(httpClient, indexName);
}
}

View File

@ -188,7 +188,7 @@ public class ClusterRerouteTests extends ElasticsearchIntegrationTest {
client().prepareIndex("test", "type", "1").setSource("field", "value").setRefresh(true).execute().actionGet();
logger.info("--> closing all nodes");
Path[] shardLocation = internalCluster().getInstance(NodeEnvironment.class, node_1).shardPaths(new ShardId("test", 0));
Path[] shardLocation = internalCluster().getInstance(NodeEnvironment.class, node_1).availableShardPaths(new ShardId("test", 0));
assertThat(FileSystemUtils.exists(shardLocation), equalTo(true)); // make sure the data is there!
internalCluster().closeNonSharedNodes(false); // don't wipe data directories the index needs to be there!

View File

@ -0,0 +1,290 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch 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.elasticsearch.common.util;
import com.carrotsearch.randomizedtesting.annotations.Repeat;
import com.google.common.base.Charsets;
import com.google.common.collect.Sets;
import org.apache.lucene.util.CollectionUtil;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TestUtil;
import org.elasticsearch.bwcompat.OldIndexBackwardsCompatibilityTests;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.io.FileSystemUtils;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.gateway.MetaDataStateFormat;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardPath;
import org.elasticsearch.index.shard.ShardStateMetaData;
import org.elasticsearch.test.ElasticsearchTestCase;
import java.io.BufferedWriter;
import java.io.IOException;
import java.io.InputStream;
import java.net.URISyntaxException;
import java.nio.file.*;
import java.util.*;
/**
*/
@LuceneTestCase.SuppressFileSystems("ExtrasFS")
public class MultiDataPathUpgraderTests extends ElasticsearchTestCase {
public void testUpgradeRandomPaths() throws IOException {
try (NodeEnvironment nodeEnvironment = newNodeEnvironment()) {
final String uuid = Strings.base64UUID();
final ShardId shardId = new ShardId("foo", 0);
final Path[] shardDataPaths = nodeEnvironment.availableShardPaths(shardId);
if (nodeEnvironment.nodeDataPaths().length == 1) {
MultiDataPathUpgrader helper = new MultiDataPathUpgrader(nodeEnvironment);
assertFalse(helper.needsUpgrading(shardId));
return;
}
int numIdxFiles = 0;
int numTranslogFiles = 0;
int metaStateVersion = 0;
for (Path shardPath : shardDataPaths) {
final Path translog = shardPath.resolve(ShardPath.TRANSLOG_FOLDER_NAME);
final Path idx = shardPath.resolve(ShardPath.INDEX_FOLDER_NAME);
Files.createDirectories(translog);
Files.createDirectories(idx);
int numFiles = randomIntBetween(1, 10);
for (int i = 0; i < numFiles; i++, numIdxFiles++) {
String filename = Integer.toString(numIdxFiles);
try (BufferedWriter w = Files.newBufferedWriter(idx.resolve(filename + ".tst"), Charsets.UTF_8)) {
w.write(filename);
}
}
numFiles = randomIntBetween(1, 10);
for (int i = 0; i < numFiles; i++, numTranslogFiles++) {
String filename = Integer.toString(numTranslogFiles);
try (BufferedWriter w = Files.newBufferedWriter(translog.resolve(filename + ".translog"), Charsets.UTF_8)) {
w.write(filename);
}
}
++metaStateVersion;
ShardStateMetaData.FORMAT.write(new ShardStateMetaData(metaStateVersion, true, uuid), metaStateVersion, shardDataPaths);
}
final Path path = randomFrom(shardDataPaths);
ShardPath targetPath = new ShardPath(path, path, uuid, new ShardId("foo", 0));
MultiDataPathUpgrader helper = new MultiDataPathUpgrader(nodeEnvironment);
helper.upgrade(shardId, targetPath);
assertFalse(helper.needsUpgrading(shardId));
if (shardDataPaths.length > 1) {
for (Path shardPath : shardDataPaths) {
if (shardPath.equals(targetPath.getDataPath())) {
continue;
}
final Path translog = shardPath.resolve(ShardPath.TRANSLOG_FOLDER_NAME);
final Path idx = shardPath.resolve(ShardPath.INDEX_FOLDER_NAME);
final Path state = shardPath.resolve(MetaDataStateFormat.STATE_DIR_NAME);
assertFalse(Files.exists(translog));
assertFalse(Files.exists(idx));
assertFalse(Files.exists(state));
assertFalse(Files.exists(shardPath));
}
}
final ShardStateMetaData stateMetaData = ShardStateMetaData.FORMAT.loadLatestState(logger, targetPath.getShardStatePath());
assertEquals(metaStateVersion, stateMetaData.version);
assertTrue(stateMetaData.primary);
assertEquals(uuid, stateMetaData.indexUUID);
final Path translog = targetPath.getDataPath().resolve(ShardPath.TRANSLOG_FOLDER_NAME);
final Path idx = targetPath.getDataPath().resolve(ShardPath.INDEX_FOLDER_NAME);
Files.deleteIfExists(idx.resolve("write.lock"));
assertEquals(numTranslogFiles, FileSystemUtils.files(translog).length);
assertEquals(numIdxFiles, FileSystemUtils.files(idx).length);
final HashSet<Path> translogFiles = Sets.newHashSet(FileSystemUtils.files(translog));
for (int i = 0; i < numTranslogFiles; i++) {
final String name = Integer.toString(i);
translogFiles.contains(translog.resolve(name + ".translog"));
byte[] content = Files.readAllBytes(translog.resolve(name + ".translog"));
assertEquals(name , new String(content, Charsets.UTF_8));
}
final HashSet<Path> idxFiles = Sets.newHashSet(FileSystemUtils.files(idx));
for (int i = 0; i < numIdxFiles; i++) {
final String name = Integer.toString(i);
idxFiles.contains(idx.resolve(name + ".tst"));
byte[] content = Files.readAllBytes(idx.resolve(name + ".tst"));
assertEquals(name , new String(content, Charsets.UTF_8));
}
}
}
/**
* Run upgrade on a real bwc index
*/
public void testUpgradeRealIndex() throws IOException, URISyntaxException {
List<Path> indexes = new ArrayList<>();
Path dir = getDataPath("/" + OldIndexBackwardsCompatibilityTests.class.getPackage().getName().replace('.', '/')); // the files are in the same pkg as the OldIndexBackwardsCompatibilityTests test
try (DirectoryStream<Path> stream = Files.newDirectoryStream(dir, "index-*.zip")) {
for (Path path : stream) {
indexes.add(path);
}
}
CollectionUtil.introSort(indexes, new Comparator<Path>() {
@Override
public int compare(Path o1, Path o2) {
return o1.getFileName().compareTo(o2.getFileName());
}
});
final ShardId shardId = new ShardId("test", 0);
final Path path = randomFrom(indexes);
final Path indexFile = path;
final String indexName = indexFile.getFileName().toString().replace(".zip", "").toLowerCase(Locale.ROOT);
try (NodeEnvironment nodeEnvironment = newNodeEnvironment()) {
if (nodeEnvironment.nodeDataPaths().length == 1) {
MultiDataPathUpgrader helper = new MultiDataPathUpgrader(nodeEnvironment);
assertFalse(helper.needsUpgrading(shardId));
return;
}
Path unzipDir = createTempDir();
Path unzipDataDir = unzipDir.resolve("data");
// decompress the index
try (InputStream stream = Files.newInputStream(indexFile)) {
TestUtil.unzip(stream, unzipDir);
}
// check it is unique
assertTrue(Files.exists(unzipDataDir));
Path[] list = FileSystemUtils.files(unzipDataDir);
if (list.length != 1) {
throw new IllegalStateException("Backwards index must contain exactly one cluster but was " + list.length);
}
// the bwc scripts packs the indices under this path
Path src = list[0].resolve("nodes/0/indices/" + indexName);
assertTrue("[" + indexFile + "] missing index dir: " + src.toString(), Files.exists(src));
Path[] multiDataPath = new Path[nodeEnvironment.nodeDataPaths().length];
int i = 0;
for (NodeEnvironment.NodePath nodePath : nodeEnvironment.nodePaths()) {
multiDataPath[i++] = nodePath.indicesPath;
}
logger.info("--> injecting index [{}] into multiple data paths", indexName);
OldIndexBackwardsCompatibilityTests.copyIndex(logger, src, indexName, multiDataPath);
final ShardPath shardPath = new ShardPath(nodeEnvironment.availableShardPaths(new ShardId(indexName, 0))[0], nodeEnvironment.availableShardPaths(new ShardId(indexName, 0))[0], IndexMetaData.INDEX_UUID_NA_VALUE, new ShardId(indexName, 0));
logger.info("{}", FileSystemUtils.files(shardPath.resolveIndex()));
MultiDataPathUpgrader helper = new MultiDataPathUpgrader(nodeEnvironment);
helper.upgrade(new ShardId(indexName, 0), shardPath);
helper.checkIndex(shardPath);
assertFalse(helper.needsUpgrading(new ShardId(indexName, 0)));
}
}
public void testNeedsUpgrade() throws IOException {
try (NodeEnvironment nodeEnvironment = newNodeEnvironment()) {
String uuid = Strings.randomBase64UUID();
final ShardId shardId = new ShardId("foo", 0);
ShardStateMetaData.FORMAT.write(new ShardStateMetaData(1, true, uuid), 1, nodeEnvironment.availableShardPaths(shardId));
MultiDataPathUpgrader helper = new MultiDataPathUpgrader(nodeEnvironment);
boolean multiDataPaths = nodeEnvironment.nodeDataPaths().length > 1;
boolean needsUpgrading = helper.needsUpgrading(shardId);
if (multiDataPaths) {
assertTrue(needsUpgrading);
} else {
assertFalse(needsUpgrading);
}
}
}
public void testPickTargetShardPath() throws IOException {
try (NodeEnvironment nodeEnvironment = newNodeEnvironment()) {
final ShardId shard = new ShardId("foo", 0);
final Path[] paths = nodeEnvironment.availableShardPaths(shard);
if (paths.length == 1) {
MultiDataPathUpgrader helper = new MultiDataPathUpgrader(nodeEnvironment);
try {
helper.pickShardPath(new ShardId("foo", 0));
fail("one path needs no upgrading");
} catch (IllegalStateException ex) {
// only one path
}
} else {
final Map<Path, Tuple<Long, Long>> pathToSpace = new HashMap<>();
final Path expectedPath;
if (randomBoolean()) { // path with most of the file bytes
expectedPath = randomFrom(paths);
long[] used = new long[paths.length];
long sumSpaceUsed = 0;
for (int i = 0; i < used.length; i++) {
long spaceUsed = paths[i] == expectedPath ? randomIntBetween(101, 200) : randomIntBetween(10, 100);
sumSpaceUsed += spaceUsed;
used[i] = spaceUsed;
}
for (int i = 0; i < used.length; i++) {
long availalbe = randomIntBetween((int)(2*sumSpaceUsed-used[i]), 4 * (int)sumSpaceUsed);
pathToSpace.put(paths[i], new Tuple<>(availalbe, used[i]));
}
} else { // path with largest available space
expectedPath = randomFrom(paths);
long[] used = new long[paths.length];
long sumSpaceUsed = 0;
for (int i = 0; i < used.length; i++) {
long spaceUsed = randomIntBetween(10, 100);
sumSpaceUsed += spaceUsed;
used[i] = spaceUsed;
}
for (int i = 0; i < used.length; i++) {
long availalbe = paths[i] == expectedPath ? randomIntBetween((int)(sumSpaceUsed), (int)(2*sumSpaceUsed)) : randomIntBetween(0, (int)(sumSpaceUsed) - 1) ;
pathToSpace.put(paths[i], new Tuple<>(availalbe, used[i]));
}
}
MultiDataPathUpgrader helper = new MultiDataPathUpgrader(nodeEnvironment) {
@Override
protected long getUsabelSpace(NodeEnvironment.NodePath path) throws IOException {
return pathToSpace.get(path.resolve(shard)).v1();
}
@Override
protected long getSpaceUsedByShard(Path path) throws IOException {
return pathToSpace.get(path).v2();
}
};
String uuid = Strings.randomBase64UUID();
ShardStateMetaData.FORMAT.write(new ShardStateMetaData(1, true, uuid), 1, paths);
final ShardPath shardPath = helper.pickShardPath(new ShardId("foo", 0));
assertEquals(expectedPath, shardPath.getDataPath());
assertEquals(expectedPath, shardPath.getShardStatePath());
}
MultiDataPathUpgrader helper = new MultiDataPathUpgrader(nodeEnvironment) {
@Override
protected long getUsabelSpace(NodeEnvironment.NodePath path) throws IOException {
return randomIntBetween(0, 10);
}
@Override
protected long getSpaceUsedByShard(Path path) throws IOException {
return randomIntBetween(11, 20);
}
};
try {
helper.pickShardPath(new ShardId("foo", 0));
fail("not enough space");
} catch (IllegalStateException ex) {
// not enough space
}
}
}
}

View File

@ -36,7 +36,6 @@ import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
@ -314,11 +313,13 @@ public class NodeEnvironmentTests extends ElasticsearchTestCase {
assertFalse("no settings should mean no custom data path", NodeEnvironment.hasCustomDataPath(s1));
assertTrue("settings with path_data should have a custom data path", NodeEnvironment.hasCustomDataPath(s2));
assertThat(env.shardDataPaths(sid, s1), equalTo(env.shardPaths(sid)));
assertThat(env.shardDataPaths(sid, s2), equalTo(new Path[] {PathUtils.get("/tmp/foo/0/myindex/0")}));
assertThat(env.availableShardPaths(sid), equalTo(env.availableShardPaths(sid)));
assertFalse(NodeEnvironment.hasCustomDataPath(s1));
assertThat(env.resolveCustomLocation(s2, sid), equalTo(PathUtils.get("/tmp/foo/0/myindex/0")));
assertTrue(NodeEnvironment.hasCustomDataPath(s2));
assertThat("shard paths with a custom data_path should contain only regular paths",
env.shardPaths(sid),
env.availableShardPaths(sid),
equalTo(stringsToPaths(dataPaths, "elasticsearch/nodes/0/indices/myindex/0")));
assertThat("index paths uses the regular template",
@ -328,11 +329,11 @@ public class NodeEnvironmentTests extends ElasticsearchTestCase {
NodeEnvironment env2 = newNodeEnvironment(dataPaths,
ImmutableSettings.builder().put(NodeEnvironment.ADD_NODE_ID_TO_CUSTOM_PATH, false).build());
assertThat(env2.shardDataPaths(sid, s1), equalTo(env2.shardPaths(sid)));
assertThat(env2.shardDataPaths(sid, s2), equalTo(new Path[] {PathUtils.get("/tmp/foo/myindex/0")}));
assertThat(env2.availableShardPaths(sid), equalTo(env2.availableShardPaths(sid)));
assertThat(env2.resolveCustomLocation(s2, sid), equalTo(PathUtils.get("/tmp/foo/myindex/0")));
assertThat("shard paths with a custom data_path should contain only regular paths",
env2.shardPaths(sid),
env2.availableShardPaths(sid),
equalTo(stringsToPaths(dataPaths, "elasticsearch/nodes/0/indices/myindex/0")));
assertThat("index paths uses the regular template",

View File

@ -84,7 +84,6 @@ import org.elasticsearch.index.similarity.SimilarityLookupService;
import org.elasticsearch.index.store.DirectoryService;
import org.elasticsearch.index.store.DirectoryUtils;
import org.elasticsearch.index.store.Store;
import org.elasticsearch.index.store.distributor.LeastUsedDistributor;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.index.translog.TranslogSizeMatcher;
import org.elasticsearch.index.translog.fs.FsTranslog;
@ -214,8 +213,8 @@ public class InternalEngineTests extends ElasticsearchTestCase {
protected Store createStore(final Directory directory) throws IOException {
final DirectoryService directoryService = new DirectoryService(shardId, EMPTY_SETTINGS) {
@Override
public Directory[] build() throws IOException {
return new Directory[]{directory};
public Directory newDirectory() throws IOException {
return directory;
}
@Override
@ -223,7 +222,7 @@ public class InternalEngineTests extends ElasticsearchTestCase {
return 0;
}
};
return new Store(shardId, EMPTY_SETTINGS, directoryService, new LeastUsedDistributor(directoryService), new DummyShardLock(shardId));
return new Store(shardId, EMPTY_SETTINGS, directoryService, new DummyShardLock(shardId));
}
protected Translog createTranslog() throws IOException {

View File

@ -59,7 +59,6 @@ import org.elasticsearch.index.shard.ShardUtils;
import org.elasticsearch.index.store.DirectoryService;
import org.elasticsearch.index.store.DirectoryUtils;
import org.elasticsearch.index.store.Store;
import org.elasticsearch.index.store.distributor.LeastUsedDistributor;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.index.translog.fs.FsTranslog;
import org.elasticsearch.test.DummyShardLock;
@ -191,8 +190,8 @@ public class ShadowEngineTests extends ElasticsearchTestCase {
protected Store createStore(final Directory directory) throws IOException {
final DirectoryService directoryService = new DirectoryService(shardId, EMPTY_SETTINGS) {
@Override
public Directory[] build() throws IOException {
return new Directory[]{ directory };
public Directory newDirectory() throws IOException {
return directory;
}
@Override
@ -200,7 +199,7 @@ public class ShadowEngineTests extends ElasticsearchTestCase {
return 0;
}
};
return new Store(shardId, EMPTY_SETTINGS, directoryService, new LeastUsedDistributor(directoryService), new DummyShardLock(shardId));
return new Store(shardId, EMPTY_SETTINGS, directoryService, new DummyShardLock(shardId));
}
protected Translog createTranslog() throws IOException {

View File

@ -33,7 +33,6 @@ import org.elasticsearch.index.settings.IndexSettingsService;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.store.DirectoryService;
import org.elasticsearch.index.store.Store;
import org.elasticsearch.index.store.distributor.LeastUsedDistributor;
import org.elasticsearch.test.DummyShardLock;
import org.elasticsearch.test.ElasticsearchTestCase;
import org.junit.Test;
@ -315,8 +314,8 @@ public class MergePolicySettingsTest extends ElasticsearchTestCase {
protected Store createStore(Settings settings) throws IOException {
final DirectoryService directoryService = new DirectoryService(shardId, EMPTY_SETTINGS) {
@Override
public Directory[] build() throws IOException {
return new Directory[] { new RAMDirectory() } ;
public Directory newDirectory() throws IOException {
return new RAMDirectory() ;
}
@Override
@ -324,7 +323,7 @@ public class MergePolicySettingsTest extends ElasticsearchTestCase {
return 0;
}
};
return new Store(shardId, settings, directoryService, new LeastUsedDistributor(directoryService), new DummyShardLock(shardId));
return new Store(shardId, settings, directoryService, new DummyShardLock(shardId));
}
}

View File

@ -71,18 +71,18 @@ public class IndexShardTests extends ElasticsearchSingleNodeTest {
long version = between(1, Integer.MAX_VALUE / 2);
boolean primary = randomBoolean();
ShardStateMetaData state1 = new ShardStateMetaData(version, primary, "foo");
write(state1, env.shardPaths(id));
ShardStateMetaData shardStateMetaData = load(logger, env.shardPaths(id));
write(state1, env.availableShardPaths(id));
ShardStateMetaData shardStateMetaData = load(logger, env.availableShardPaths(id));
assertEquals(shardStateMetaData, state1);
ShardStateMetaData state2 = new ShardStateMetaData(version, primary, "foo");
write(state2, env.shardPaths(id));
shardStateMetaData = load(logger, env.shardPaths(id));
write(state2, env.availableShardPaths(id));
shardStateMetaData = load(logger, env.availableShardPaths(id));
assertEquals(shardStateMetaData, state1);
ShardStateMetaData state3 = new ShardStateMetaData(version + 1, primary, "foo");
write(state3, env.shardPaths(id));
shardStateMetaData = load(logger, env.shardPaths(id));
write(state3, env.availableShardPaths(id));
shardStateMetaData = load(logger, env.availableShardPaths(id));
assertEquals(shardStateMetaData, state3);
assertEquals("foo", state3.indexUUID);
}
@ -95,44 +95,44 @@ public class IndexShardTests extends ElasticsearchSingleNodeTest {
NodeEnvironment env = getInstanceFromNode(NodeEnvironment.class);
IndexService test = indicesService.indexService("test");
IndexShard shard = test.shard(0);
ShardStateMetaData shardStateMetaData = load(logger, env.shardPaths(shard.shardId));
ShardStateMetaData shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId));
assertEquals(getShardStateMetadata(shard), shardStateMetaData);
ShardRouting routing = new MutableShardRouting(shard.shardRouting, shard.shardRouting.version()+1);
shard.updateRoutingEntry(routing, true);
shardStateMetaData = load(logger, env.shardPaths(shard.shardId));
shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId));
assertEquals(shardStateMetaData, getShardStateMetadata(shard));
assertEquals(shardStateMetaData, new ShardStateMetaData(routing.version(), routing.primary(), shard.indexSettings.get(IndexMetaData.SETTING_UUID)));
routing = new MutableShardRouting(shard.shardRouting, shard.shardRouting.version()+1);
shard.updateRoutingEntry(routing, true);
shardStateMetaData = load(logger, env.shardPaths(shard.shardId));
shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId));
assertEquals(shardStateMetaData, getShardStateMetadata(shard));
assertEquals(shardStateMetaData, new ShardStateMetaData(routing.version(), routing.primary(), shard.indexSettings.get(IndexMetaData.SETTING_UUID)));
routing = new MutableShardRouting(shard.shardRouting, shard.shardRouting.version()+1);
shard.updateRoutingEntry(routing, true);
shardStateMetaData = load(logger, env.shardPaths(shard.shardId));
shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId));
assertEquals(shardStateMetaData, getShardStateMetadata(shard));
assertEquals(shardStateMetaData, new ShardStateMetaData(routing.version(), routing.primary(), shard.indexSettings.get(IndexMetaData.SETTING_UUID)));
// test if we still write it even if the shard is not active
MutableShardRouting inactiveRouting = new MutableShardRouting(shard.shardRouting.index(), shard.shardRouting.shardId().id(), shard.shardRouting.currentNodeId(), true, ShardRoutingState.INITIALIZING, shard.shardRouting.version() + 1);
shard.persistMetadata(inactiveRouting, shard.shardRouting);
shardStateMetaData = load(logger, env.shardPaths(shard.shardId));
shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId));
assertEquals("inactive shard state shouldn't be persisted", shardStateMetaData, getShardStateMetadata(shard));
assertEquals("inactive shard state shouldn't be persisted", shardStateMetaData, new ShardStateMetaData(routing.version(), routing.primary(), shard.indexSettings.get(IndexMetaData.SETTING_UUID)));
shard.updateRoutingEntry(new MutableShardRouting(shard.shardRouting, shard.shardRouting.version()+1), false);
shardStateMetaData = load(logger, env.shardPaths(shard.shardId));
shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId));
assertFalse("shard state persisted despite of persist=false", shardStateMetaData.equals(getShardStateMetadata(shard)));
assertEquals("shard state persisted despite of persist=false", shardStateMetaData, new ShardStateMetaData(routing.version(), routing.primary(), shard.indexSettings.get(IndexMetaData.SETTING_UUID)));
routing = new MutableShardRouting(shard.shardRouting, shard.shardRouting.version()+1);
shard.updateRoutingEntry(routing, true);
shardStateMetaData = load(logger, env.shardPaths(shard.shardId));
shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId));
assertEquals(shardStateMetaData, getShardStateMetadata(shard));
assertEquals(shardStateMetaData, new ShardStateMetaData(routing.version(), routing.primary(), shard.indexSettings.get(IndexMetaData.SETTING_UUID)));
}
@ -152,14 +152,14 @@ public class IndexShardTests extends ElasticsearchSingleNodeTest {
}
ShardRouting routing = shard.routingEntry();
ShardStateMetaData shardStateMetaData = load(logger, env.shardPaths(shard.shardId));
ShardStateMetaData shardStateMetaData = load(logger, env.availableShardPaths(shard.shardId));
assertEquals(shardStateMetaData, getShardStateMetadata(shard));
routing = new MutableShardRouting(shard.shardId.index().getName(), shard.shardId.id(), routing.currentNodeId(), routing.primary(), ShardRoutingState.INITIALIZING, shard.shardRouting.version()+1);
shard.updateRoutingEntry(routing, true);
shard.deleteShardState();
assertNull("no shard state expected after delete on initializing", load(logger, env.shardPaths(shard.shardId)));
assertNull("no shard state expected after delete on initializing", load(logger, env.availableShardPaths(shard.shardId)));

View File

@ -0,0 +1,83 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch 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.elasticsearch.index.shard;
import org.elasticsearch.ElasticsearchIllegalStateException;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.test.ElasticsearchTestCase;
import org.junit.Test;
import java.io.IOException;
import java.nio.file.Path;
import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder;
/**
*/
public class ShardPathTests extends ElasticsearchTestCase {
public void testLoadShardPath() throws IOException {
try (final NodeEnvironment env = newNodeEnvironment(settingsBuilder().build())) {
ImmutableSettings.Builder builder = settingsBuilder().put(IndexMetaData.SETTING_UUID, "0xDEADBEEF");
Settings settings = builder.build();
ShardId shardId = new ShardId("foo", 0);
Path[] paths = env.availableShardPaths(shardId);
Path path = randomFrom(paths);
ShardStateMetaData.FORMAT.write(new ShardStateMetaData(2, true, "0xDEADBEEF"), 2, path);
ShardPath shardPath = ShardPath.loadShardPath(logger, env, shardId, settings);
assertEquals(path, shardPath.getDataPath());
assertEquals("0xDEADBEEF", shardPath.getIndexUUID());
assertEquals("foo", shardPath.getShardId().getIndex());
assertEquals(path.resolve("translog"), shardPath.resolveTranslog());
assertEquals(path.resolve("index"), shardPath.resolveIndex());
}
}
@Test(expected = ElasticsearchIllegalStateException.class)
public void testFailLoadShardPathOnMultiState() throws IOException {
try (final NodeEnvironment env = newNodeEnvironment(settingsBuilder().build())) {
ImmutableSettings.Builder builder = settingsBuilder().put(IndexMetaData.SETTING_UUID, "0xDEADBEEF");
Settings settings = builder.build();
ShardId shardId = new ShardId("foo", 0);
Path[] paths = env.availableShardPaths(shardId);
assumeTrue("This test tests multi data.path but we only got one", paths.length > 1);
int id = randomIntBetween(1, 10);
ShardStateMetaData.FORMAT.write(new ShardStateMetaData(id, true, "0xDEADBEEF"), id, paths);
ShardPath.loadShardPath(logger, env, shardId, settings);
}
}
@Test(expected = ElasticsearchIllegalStateException.class)
public void testFailLoadShardPathIndexUUIDMissmatch() throws IOException {
try (final NodeEnvironment env = newNodeEnvironment(settingsBuilder().build())) {
ImmutableSettings.Builder builder = settingsBuilder().put(IndexMetaData.SETTING_UUID, "foobar");
Settings settings = builder.build();
ShardId shardId = new ShardId("foo", 0);
Path[] paths = env.availableShardPaths(shardId);
Path path = randomFrom(paths);
int id = randomIntBetween(1, 10);
ShardStateMetaData.FORMAT.write(new ShardStateMetaData(id, true, "0xDEADBEEF"), id, path);
ShardPath.loadShardPath(logger, env, shardId, settings);
}
}
}

View File

@ -532,11 +532,13 @@ public class CorruptedFileTest extends ElasticsearchIntegrationTest {
String path = info.getPath();
final String relativeDataLocationPath = "indices/test/" + Integer.toString(shardRouting.getId()) + "/index";
Path file = PathUtils.get(path).resolve(relativeDataLocationPath);
try (DirectoryStream<Path> stream = Files.newDirectoryStream(file)) {
for (Path item : stream) {
if (Files.isRegularFile(item) && "write.lock".equals(item.getFileName().toString()) == false) {
if (includePerCommitFiles || isPerSegmentFile(item.getFileName().toString())) {
files.add(item);
if (Files.exists(file)) { // multi data path might only have one path in use
try (DirectoryStream<Path> stream = Files.newDirectoryStream(file)) {
for (Path item : stream) {
if (Files.isRegularFile(item) && "write.lock".equals(item.getFileName().toString()) == false) {
if (includePerCommitFiles || isPerSegmentFile(item.getFileName().toString())) {
files.add(item);
}
}
}
}
@ -641,9 +643,11 @@ public class CorruptedFileTest extends ElasticsearchIntegrationTest {
for (FsStats.Info info : nodeStatses.getNodes()[0].getFs()) {
String path = info.getPath();
Path file = PathUtils.get(path).resolve("indices/test/" + Integer.toString(routing.getId()) + "/index");
try (DirectoryStream<Path> stream = Files.newDirectoryStream(file)) {
for (Path item : stream) {
files.add(item);
if (Files.exists(file)) { // multi data path might only have one path in use
try (DirectoryStream<Path> stream = Files.newDirectoryStream(file)) {
for (Path item : stream) {
files.add(item);
}
}
}
}

View File

@ -128,14 +128,15 @@ public class CorruptedTranslogTests extends ElasticsearchIntegrationTest {
String path = info.getPath();
final String relativeDataLocationPath = "indices/test/" + Integer.toString(shardRouting.getId()) + "/translog";
Path file = PathUtils.get(path).resolve(relativeDataLocationPath);
logger.info("--> path: {}", file);
try (DirectoryStream<Path> stream = Files.newDirectoryStream(file)) {
for (Path item : stream) {
logger.info("--> File: {}", item);
if (Files.isRegularFile(item) && item.getFileName().toString().startsWith("translog-")) {
files.add(item);
if (Files.exists(file)) {
logger.info("--> path: {}", file);
try (DirectoryStream<Path> stream = Files.newDirectoryStream(file)) {
for (Path item : stream) {
logger.info("--> File: {}", item);
if (Files.isRegularFile(item) && item.getFileName().toString().startsWith("translog-")) {
files.add(item);
}
}
}
}
}

View File

@ -1,208 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch 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.elasticsearch.index.store;
import com.carrotsearch.randomizedtesting.annotations.Listeners;
import com.carrotsearch.randomizedtesting.annotations.ThreadLeakLingering;
import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope;
import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
import com.carrotsearch.randomizedtesting.generators.RandomInts;
import com.carrotsearch.randomizedtesting.generators.RandomPicks;
import com.google.common.collect.ImmutableSet;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.store.BaseDirectoryTestCase;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FilterDirectory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.MockDirectoryWrapper;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util.TimeUnits;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.index.store.distributor.Distributor;
import org.elasticsearch.test.junit.listeners.LoggingListener;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.nio.file.NoSuchFileException;
import java.nio.file.Path;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashSet;
import java.util.Set;
@ThreadLeakScope(ThreadLeakScope.Scope.SUITE)
@ThreadLeakLingering(linger = 5000) // 5 sec lingering
@TimeoutSuite(millis = 5 * TimeUnits.MINUTE)
@Listeners(LoggingListener.class)
@LuceneTestCase.SuppressSysoutChecks(bugUrl = "we log a lot on purpose")
public class DistributorDirectoryTest extends BaseDirectoryTestCase {
protected final ESLogger logger = Loggers.getLogger(getClass());
@Override
protected Directory getDirectory(Path path) throws IOException {
Directory[] directories = new Directory[1 + random().nextInt(5)];
for (int i = 0; i < directories.length; i++) {
directories[i] = newDirectory();
if (directories[i] instanceof MockDirectoryWrapper) {
// TODO: fix this test to handle virus checker
((MockDirectoryWrapper) directories[i]).setEnableVirusScanner(false);
}
}
return new DistributorDirectory(directories);
}
// #7306: don't invoke the distributor when we are opening an already existing file
public void testDoNotCallDistributorOnRead() throws Exception {
Directory dir = newDirectory();
dir.createOutput("one.txt", IOContext.DEFAULT).close();
final Directory[] dirs = new Directory[] {dir};
Distributor distrib = new Distributor() {
@Override
public Directory primary() {
return dirs[0];
}
@Override
public Directory[] all() {
return dirs;
}
@Override
public synchronized Directory any() {
throw new IllegalStateException("any should not be called");
}
};
DistributorDirectory dd = new DistributorDirectory(distrib);
assertEquals(0, dd.fileLength("one.txt"));
dd.openInput("one.txt", IOContext.DEFAULT).close();
try {
dd.createOutput("three.txt", IOContext.DEFAULT).close();
fail("didn't hit expected exception");
} catch (IllegalStateException ise) {
// expected
}
dd.close();
}
public void testRenameFiles() throws IOException {
final int iters = 1 + random().nextInt(10);
for (int i = 0; i < iters; i++) {
Directory[] dirs = new Directory[1 + random().nextInt(5)];
for (int j=0; j < dirs.length; j++) {
MockDirectoryWrapper directory = newMockDirectory();
directory.setEnableVirusScanner(false);
directory.setCheckIndexOnClose(false);
dirs[j] = directory;
}
DistributorDirectory dd = new DistributorDirectory(dirs);
String file = RandomPicks.randomFrom(random(), Arrays.asList(Store.CHECKSUMS_PREFIX, IndexFileNames.OLD_SEGMENTS_GEN, IndexFileNames.SEGMENTS, IndexFileNames.PENDING_SEGMENTS));
String tmpFileName = RandomPicks.randomFrom(random(), Arrays.asList("recovery.", "foobar.", "test.")) + Math.max(0, Math.abs(random().nextLong())) + "." + file;
try (IndexOutput out = dd.createOutput(tmpFileName, IOContext.DEFAULT)) {
out.writeInt(1);
}
Directory theDir = null;
for (Directory d : dirs) {
try {
if (d.fileLength(tmpFileName) > 0) {
theDir = d;
break;
}
} catch (IOException ex) {
// nevermind
}
}
assertNotNull("file must be in at least one dir", theDir);
dd.renameFile(tmpFileName, file);
try {
dd.fileLength(tmpFileName);
fail("file ["+tmpFileName + "] was renamed but still exists");
} catch (FileNotFoundException | NoSuchFileException ex) {
// all is well
}
try {
theDir.fileLength(tmpFileName);
fail("file ["+tmpFileName + "] was renamed but still exists");
} catch (FileNotFoundException | NoSuchFileException ex) {
// all is well
}
assertEquals(theDir.fileLength(file), 4);
try (IndexOutput out = dd.createOutput("foo.bar", IOContext.DEFAULT)) {
out.writeInt(1);
}
assertNotNull(dd);
if (dd.getDirectory("foo.bar") != dd.getDirectory(file)) {
try {
dd.renameFile("foo.bar", file);
fail("target file already exists in a different directory");
} catch (IOException ex) {
// target file already exists
}
}
IOUtils.close(dd);
}
}
public void testSync() throws IOException {
final Set<String> syncedFiles = new HashSet<>();
final Directory[] directories = new Directory[RandomInts.randomIntBetween(random(), 1, 5)];
for (int i = 0; i < directories.length; ++i) {
final Directory dir = newDirectory();
directories[i] = new FilterDirectory(dir) {
@Override
public void sync(Collection<String> names) throws IOException {
super.sync(names);
syncedFiles.addAll(names);
}
};
}
final Directory directory = new DistributorDirectory(directories);
for (String file : Arrays.asList("a.bin", "b.bin")) {
try (IndexOutput out = directory.createOutput(file, IOContext.DEFAULT)) {
out.writeInt(random().nextInt());
}
}
// syncing on a missing file throws an exception
try {
directory.sync(Arrays.asList("a.bin", "c.bin"));
} catch (FileNotFoundException e) {
// expected
}
assertEquals(ImmutableSet.of(), syncedFiles);
// but syncing on existing files actually delegates
directory.sync(Arrays.asList("a.bin", "b.bin"));
assertEquals(ImmutableSet.of("a.bin", "b.bin"), syncedFiles);
directory.close();
}
}

View File

@ -1,213 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch 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.elasticsearch.index.store;
import com.carrotsearch.randomizedtesting.annotations.Listeners;
import com.carrotsearch.randomizedtesting.annotations.ThreadLeakLingering;
import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.ThreadedIndexingAndSearchingTestCase;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.MockDirectoryWrapper;
import org.apache.lucene.util.LuceneTestCase;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.index.store.distributor.Distributor;
import org.elasticsearch.test.junit.listeners.LoggingListener;
import org.elasticsearch.test.junit.listeners.ReproduceInfoPrinter;
import org.junit.Before;
import java.io.IOException;
import java.nio.file.Path;
import java.util.HashSet;
import java.util.Set;
import java.util.concurrent.ExecutorService;
/**
* This test is a copy of TestNRTThreads from lucene that puts some
* hard concurrent pressure on the directory etc. to ensure DistributorDirectory is behaving ok.
*/
@LuceneTestCase.SuppressCodecs({ "SimpleText", "Memory", "Direct" })
@ThreadLeakScope(ThreadLeakScope.Scope.SUITE)
@ThreadLeakLingering(linger = 5000) // 5 sec lingering
@Listeners({
ReproduceInfoPrinter.class,
LoggingListener.class
})
@LuceneTestCase.SuppressReproduceLine
@LuceneTestCase.SuppressSysoutChecks(bugUrl = "we log a lot on purpose")
@LuceneTestCase.SuppressFileSystems("ExtrasFS") // can easily create the same extra file in two subdirs
public class DistributorInTheWildTest extends ThreadedIndexingAndSearchingTestCase {
protected final ESLogger logger = Loggers.getLogger(getClass());
private boolean useNonNrtReaders = true;
@Override
@Before
public void setUp() throws Exception {
super.setUp();
useNonNrtReaders = random().nextBoolean();
}
@Override
protected void doSearching(ExecutorService es, long stopTime) throws Exception {
boolean anyOpenDelFiles = false;
DirectoryReader r = DirectoryReader.open(writer, true);
while (System.currentTimeMillis() < stopTime && !failed.get()) {
if (random().nextBoolean()) {
if (VERBOSE) {
logger.info("TEST: now reopen r=" + r);
}
final DirectoryReader r2 = DirectoryReader.openIfChanged(r);
if (r2 != null) {
r.close();
r = r2;
}
} else {
if (VERBOSE) {
logger.info("TEST: now close reader=" + r);
}
r.close();
writer.commit();
final Set<String> openDeletedFiles = getOpenDeletedFiles(dir);
if (openDeletedFiles.size() > 0) {
logger.info("OBD files: " + openDeletedFiles);
}
anyOpenDelFiles |= openDeletedFiles.size() > 0;
//assertEquals("open but deleted: " + openDeletedFiles, 0, openDeletedFiles.size());
if (VERBOSE) {
logger.info("TEST: now open");
}
r = DirectoryReader.open(writer, true);
}
if (VERBOSE) {
logger.info("TEST: got new reader=" + r);
}
//logger.info("numDocs=" + r.numDocs() + "
//openDelFileCount=" + dir.openDeleteFileCount());
if (r.numDocs() > 0) {
fixedSearcher = new IndexSearcher(r, es);
smokeTestSearcher(fixedSearcher);
runSearchThreads(System.currentTimeMillis() + 500);
}
}
r.close();
//logger.info("numDocs=" + r.numDocs() + " openDelFileCount=" + dir.openDeleteFileCount());
final Set<String> openDeletedFiles = getOpenDeletedFiles(dir);
if (openDeletedFiles.size() > 0) {
logger.info("OBD files: " + openDeletedFiles);
}
anyOpenDelFiles |= openDeletedFiles.size() > 0;
assertFalse("saw non-zero open-but-deleted count", anyOpenDelFiles);
}
private Set<String> getOpenDeletedFiles(Directory dir) throws IOException {
if (random().nextBoolean() && dir instanceof MockDirectoryWrapper) {
return ((MockDirectoryWrapper) dir).getOpenDeletedFiles();
}
DistributorDirectory d = DirectoryUtils.getLeaf(dir, DistributorDirectory.class, null);
Distributor distributor = d.getDistributor();
Set<String> set = new HashSet<>();
for (Directory subDir : distributor.all()) {
Set<String> openDeletedFiles = ((MockDirectoryWrapper) subDir).getOpenDeletedFiles();
set.addAll(openDeletedFiles);
}
return set;
}
@Override
protected Directory getDirectory(Directory in) {
assert in instanceof MockDirectoryWrapper;
if (!useNonNrtReaders) ((MockDirectoryWrapper) in).setAssertNoDeleteOpenFile(true);
Directory[] directories = new Directory[1 + random().nextInt(5)];
directories[0] = in;
for (int i = 1; i < directories.length; i++) {
final Path tempDir = createTempDir(getTestName());
directories[i] = newMockFSDirectory(tempDir); // some subclasses rely on this being MDW
if (!useNonNrtReaders) ((MockDirectoryWrapper) directories[i]).setAssertNoDeleteOpenFile(true);
}
for (Directory dir : directories) {
((MockDirectoryWrapper) dir).setCheckIndexOnClose(false);
}
try {
if (random().nextBoolean()) {
return new MockDirectoryWrapper(random(), new DistributorDirectory(directories));
} else {
return new DistributorDirectory(directories);
}
} catch (IOException ex) {
throw new RuntimeException(ex);
}
}
@Override
protected void doAfterWriter(ExecutorService es) throws Exception {
// Force writer to do reader pooling, always, so that
// all merged segments, even for merges before
// doSearching is called, are warmed:
DirectoryReader.open(writer, true).close();
}
private IndexSearcher fixedSearcher;
@Override
protected IndexSearcher getCurrentSearcher() throws Exception {
return fixedSearcher;
}
@Override
protected void releaseSearcher(IndexSearcher s) throws Exception {
if (s != fixedSearcher) {
// Final searcher:
s.getIndexReader().close();
}
}
@Override
protected IndexSearcher getFinalSearcher() throws Exception {
final IndexReader r2;
if (useNonNrtReaders) {
if (random().nextBoolean()) {
r2 = DirectoryReader.open(writer, true);
} else {
writer.commit();
r2 = DirectoryReader.open(dir);
}
} else {
r2 = DirectoryReader.open(writer, true);
}
return newSearcher(r2);
}
public void testNRTThreads() throws Exception {
runTest("TestNRTThreads");
}
}

View File

@ -38,9 +38,6 @@ import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.env.ShardLock;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.store.distributor.Distributor;
import org.elasticsearch.index.store.distributor.LeastUsedDistributor;
import org.elasticsearch.index.store.distributor.RandomWeightedDistributor;
import org.elasticsearch.test.DummyShardLock;
import org.elasticsearch.test.ElasticsearchTestCase;
import org.hamcrest.Matchers;
@ -63,7 +60,7 @@ public class StoreTest extends ElasticsearchTestCase {
public void testRefCount() throws IOException {
final ShardId shardId = new ShardId(new Index("index"), 1);
DirectoryService directoryService = new LuceneManagedDirectoryService(random());
Store store = new Store(shardId, ImmutableSettings.EMPTY, directoryService, randomDistributor(directoryService), new DummyShardLock(shardId));
Store store = new Store(shardId, ImmutableSettings.EMPTY, directoryService, new DummyShardLock(shardId));
int incs = randomIntBetween(1, 100);
for (int i = 0; i < incs; i++) {
if (randomBoolean()) {
@ -234,7 +231,7 @@ public class StoreTest extends ElasticsearchTestCase {
public void testWriteLegacyChecksums() throws IOException {
final ShardId shardId = new ShardId(new Index("index"), 1);
DirectoryService directoryService = new LuceneManagedDirectoryService(random());
Store store = new Store(shardId, ImmutableSettings.EMPTY, directoryService, randomDistributor(directoryService), new DummyShardLock(shardId));
Store store = new Store(shardId, ImmutableSettings.EMPTY, directoryService, new DummyShardLock(shardId));
// set default codec - all segments need checksums
final boolean usesOldCodec = randomBoolean();
IndexWriter writer = new IndexWriter(store.directory(), newIndexWriterConfig(random(), new MockAnalyzer(random())).setCodec(usesOldCodec ? new OldSIMockingCodec() : TestUtil.getDefaultCodec()));
@ -319,7 +316,7 @@ public class StoreTest extends ElasticsearchTestCase {
public void testNewChecksums() throws IOException {
final ShardId shardId = new ShardId(new Index("index"), 1);
DirectoryService directoryService = new LuceneManagedDirectoryService(random());
Store store = new Store(shardId, ImmutableSettings.EMPTY, directoryService, randomDistributor(directoryService), new DummyShardLock(shardId));
Store store = new Store(shardId, ImmutableSettings.EMPTY, directoryService, new DummyShardLock(shardId));
// set default codec - all segments need checksums
IndexWriter writer = new IndexWriter(store.directory(), newIndexWriterConfig(random(), new MockAnalyzer(random())).setCodec(TestUtil.getDefaultCodec()));
int docs = 1 + random().nextInt(100);
@ -379,7 +376,7 @@ public class StoreTest extends ElasticsearchTestCase {
public void testMixedChecksums() throws IOException {
final ShardId shardId = new ShardId(new Index("index"), 1);
DirectoryService directoryService = new LuceneManagedDirectoryService(random());
Store store = new Store(shardId, ImmutableSettings.EMPTY, directoryService, randomDistributor(directoryService), new DummyShardLock(shardId));
Store store = new Store(shardId, ImmutableSettings.EMPTY, directoryService, new DummyShardLock(shardId));
// this time random codec....
IndexWriter writer = new IndexWriter(store.directory(), newIndexWriterConfig(random(), new MockAnalyzer(random())).setCodec(TestUtil.getDefaultCodec()));
int docs = 1 + random().nextInt(100);
@ -471,7 +468,7 @@ public class StoreTest extends ElasticsearchTestCase {
public void testRenameFile() throws IOException {
final ShardId shardId = new ShardId(new Index("index"), 1);
DirectoryService directoryService = new LuceneManagedDirectoryService(random(), false);
Store store = new Store(shardId, ImmutableSettings.EMPTY, directoryService, randomDistributor(directoryService), new DummyShardLock(shardId));
Store store = new Store(shardId, ImmutableSettings.EMPTY, directoryService, new DummyShardLock(shardId));
{
IndexOutput output = store.directory().createOutput("foo.bar", IOContext.DEFAULT);
int iters = scaledRandomIntBetween(10, 100);
@ -505,27 +502,10 @@ public class StoreTest extends ElasticsearchTestCase {
CodecUtil.writeFooter(output);
output.close();
}
DistributorDirectory distributorDirectory = DirectoryUtils.getLeaf(store.directory(), DistributorDirectory.class);
if (distributorDirectory != null && distributorDirectory.getDirectory("foo.bar") != distributorDirectory.getDirectory("bar.foo")) {
try {
store.renameFile("foo.bar", "bar.foo");
fail("target file already exists in a different directory");
} catch (IOException ex) {
// expected
}
try (IndexInput input = store.directory().openInput("bar.foo", IOContext.DEFAULT)) {
assertThat(lastChecksum, equalTo(CodecUtil.checksumEntireFile(input)));
}
assertThat(store.directory().listAll().length, is(2));
assertDeleteContent(store, directoryService);
IOUtils.close(store);
} else {
store.renameFile("foo.bar", "bar.foo");
assertThat(store.directory().listAll().length, is(1));
assertDeleteContent(store, directoryService);
IOUtils.close(store);
}
store.renameFile("foo.bar", "bar.foo");
assertThat(store.directory().listAll().length, is(1));
assertDeleteContent(store, directoryService);
IOUtils.close(store);
}
public void testCheckIntegrity() throws IOException {
@ -684,13 +664,11 @@ public class StoreTest extends ElasticsearchTestCase {
deleteContent(store.directory());
assertThat(Arrays.toString(store.directory().listAll()), store.directory().listAll().length, equalTo(0));
assertThat(store.stats().sizeInBytes(), equalTo(0l));
for (Directory dir : service.build()) {
assertThat(dir.listAll().length, equalTo(0));
}
assertThat(service.newDirectory().listAll().length, equalTo(0));
}
private static final class LuceneManagedDirectoryService extends DirectoryService {
private final Directory[] dirs;
private final Directory dir;
private final Random random;
public LuceneManagedDirectoryService(Random random) {
@ -698,20 +676,17 @@ public class StoreTest extends ElasticsearchTestCase {
}
public LuceneManagedDirectoryService(Random random, boolean preventDoubleWrite) {
super(new ShardId("fake", 1), ImmutableSettings.EMPTY);
this.dirs = new Directory[1 + random.nextInt(5)];
for (int i = 0; i < dirs.length; i++) {
dirs[i] = newDirectory(random);
if (dirs[i] instanceof MockDirectoryWrapper) {
((MockDirectoryWrapper)dirs[i]).setPreventDoubleWrite(preventDoubleWrite);
dir = StoreTest.newDirectory(random);
if (dir instanceof MockDirectoryWrapper) {
((MockDirectoryWrapper)dir).setPreventDoubleWrite(preventDoubleWrite);
// TODO: fix this test to handle virus checker
((MockDirectoryWrapper)dirs[i]).setEnableVirusScanner(false);
((MockDirectoryWrapper)dir).setEnableVirusScanner(false);
}
}
this.random = random;
}
@Override
public Directory[] build() throws IOException {
return dirs;
public Directory newDirectory() throws IOException {
return dir;
}
@Override
@ -729,13 +704,6 @@ public class StoreTest extends ElasticsearchTestCase {
}
}
}
private Distributor randomDistributor(DirectoryService service) throws IOException {
return randomDistributor(random(), service);
}
private Distributor randomDistributor(Random random, DirectoryService service) throws IOException {
return random.nextBoolean() ? new LeastUsedDistributor(service) : new RandomWeightedDistributor(service);
}
/**
* Legacy indices without lucene CRC32 did never write or calculate checksums for segments_N files
@ -775,7 +743,7 @@ public class StoreTest extends ElasticsearchTestCase {
iwc.setMaxThreadStates(1);
final ShardId shardId = new ShardId(new Index("index"), 1);
DirectoryService directoryService = new LuceneManagedDirectoryService(random);
Store store = new Store(shardId, ImmutableSettings.EMPTY, directoryService, randomDistributor(random, directoryService), new DummyShardLock(shardId));
Store store = new Store(shardId, ImmutableSettings.EMPTY, directoryService, new DummyShardLock(shardId));
IndexWriter writer = new IndexWriter(store.directory(), iwc);
final boolean lotsOfSegments = rarely(random);
for (Document d : docs) {
@ -806,7 +774,7 @@ public class StoreTest extends ElasticsearchTestCase {
iwc.setMaxThreadStates(1);
final ShardId shardId = new ShardId(new Index("index"), 1);
DirectoryService directoryService = new LuceneManagedDirectoryService(random);
store = new Store(shardId, ImmutableSettings.EMPTY, directoryService, randomDistributor(random, directoryService), new DummyShardLock(shardId));
store = new Store(shardId, ImmutableSettings.EMPTY, directoryService, new DummyShardLock(shardId));
IndexWriter writer = new IndexWriter(store.directory(), iwc);
final boolean lotsOfSegments = rarely(random);
for (Document d : docs) {
@ -907,7 +875,7 @@ public class StoreTest extends ElasticsearchTestCase {
public void testCleanupFromSnapshot() throws IOException {
final ShardId shardId = new ShardId(new Index("index"), 1);
DirectoryService directoryService = new LuceneManagedDirectoryService(random());
Store store = new Store(shardId, ImmutableSettings.EMPTY, directoryService, randomDistributor(directoryService), new DummyShardLock(shardId));
Store store = new Store(shardId, ImmutableSettings.EMPTY, directoryService, new DummyShardLock(shardId));
// this time random codec....
IndexWriterConfig indexWriterConfig = newIndexWriterConfig(random(), new MockAnalyzer(random())).setCodec(TestUtil.getDefaultCodec());
// we keep all commits and that allows us clean based on multiple snapshots
@ -1016,7 +984,7 @@ public class StoreTest extends ElasticsearchTestCase {
final ShardId shardId = new ShardId(new Index("index"), 1);
DirectoryService directoryService = new LuceneManagedDirectoryService(random());
Store store = new Store(shardId, ImmutableSettings.EMPTY, directoryService, randomDistributor(directoryService), new DummyShardLock(shardId));
Store store = new Store(shardId, ImmutableSettings.EMPTY, directoryService, new DummyShardLock(shardId));
for (String file : metaDataMap.keySet()) {
try (IndexOutput output = store.directory().createOutput(file, IOContext.DEFAULT)) {
BytesRef bytesRef = new BytesRef(TestUtil.randomRealisticUnicodeString(random(), 10, 1024));
@ -1036,7 +1004,7 @@ public class StoreTest extends ElasticsearchTestCase {
final AtomicInteger count = new AtomicInteger(0);
final ShardLock lock = new DummyShardLock(shardId);
Store store = new Store(shardId, ImmutableSettings.EMPTY, directoryService, randomDistributor(directoryService), lock , new Store.OnClose() {
Store store = new Store(shardId, ImmutableSettings.EMPTY, directoryService, lock , new Store.OnClose() {
@Override
public void handle(ShardLock theLock) {
assertEquals(shardId, theLock.getShardId());
@ -1059,7 +1027,7 @@ public class StoreTest extends ElasticsearchTestCase {
final ShardId shardId = new ShardId(new Index("index"), 1);
DirectoryService directoryService = new LuceneManagedDirectoryService(random());
Settings settings = ImmutableSettings.builder().put(Store.INDEX_STORE_STATS_REFRESH_INTERVAL, TimeValue.timeValueMinutes(0)).build();
Store store = new Store(shardId, settings, directoryService, randomDistributor(directoryService), new DummyShardLock(shardId));
Store store = new Store(shardId, settings, directoryService, new DummyShardLock(shardId));
StoreStats stats = store.stats();
assertEquals(stats.getSize().bytes(), 0);

View File

@ -1,195 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch 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.elasticsearch.index.store.distributor;
import com.carrotsearch.randomizedtesting.LifecycleScope;
import org.apache.lucene.store.*;
import org.elasticsearch.common.settings.ImmutableSettings;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.store.DirectoryService;
import org.elasticsearch.test.ElasticsearchTestCase;
import org.junit.Test;
import java.io.File;
import java.io.IOException;
import java.nio.file.Path;
import java.nio.file.Paths;
import static org.hamcrest.Matchers.*;
/**
*/
public class DistributorTests extends ElasticsearchTestCase {
@Test
public void testLeastUsedDistributor() throws Exception {
FakeFsDirectory[] directories = new FakeFsDirectory[]{
new FakeFsDirectory("dir0", 10L),
new FakeFsDirectory("dir1", 20L),
new FakeFsDirectory("dir2", 30L)
};
FakeDirectoryService directoryService = new FakeDirectoryService(directories);
LeastUsedDistributor distributor = new LeastUsedDistributor(directoryService) {
@Override
protected long getUsableSpace(Directory directory) throws IOException {
return ((FakeFsDirectory)directory).useableSpace;
}
};
for (int i = 0; i < 5; i++) {
assertThat(distributor.any(), equalTo((Directory) directories[2]));
}
directories[2].setUsableSpace(5L);
for (int i = 0; i < 5; i++) {
assertThat(distributor.any(), equalTo((Directory) directories[1]));
}
directories[1].setUsableSpace(0L);
for (int i = 0; i < 5; i++) {
assertThat(distributor.any(), equalTo((Directory) directories[0]));
}
directories[0].setUsableSpace(10L);
directories[1].setUsableSpace(20L);
directories[2].setUsableSpace(20L);
for (FakeFsDirectory directory : directories) {
directory.resetAllocationCount();
}
for (int i = 0; i < 10000; i++) {
((FakeFsDirectory) distributor.any()).incrementAllocationCount();
}
assertThat(directories[0].getAllocationCount(), equalTo(0));
assertThat((double) directories[1].getAllocationCount() / directories[2].getAllocationCount(), closeTo(1.0, 0.5));
// Test failover scenario
for (FakeFsDirectory directory : directories) {
directory.resetAllocationCount();
}
directories[0].setUsableSpace(0L);
directories[1].setUsableSpace(0L);
directories[2].setUsableSpace(0L);
for (int i = 0; i < 10000; i++) {
((FakeFsDirectory) distributor.any()).incrementAllocationCount();
}
for (FakeFsDirectory directory : directories) {
assertThat(directory.getAllocationCount(), greaterThan(0));
}
assertThat((double) directories[0].getAllocationCount() / directories[2].getAllocationCount(), closeTo(1.0, 0.5));
assertThat((double) directories[1].getAllocationCount() / directories[2].getAllocationCount(), closeTo(1.0, 0.5));
}
@Test
public void testRandomWeightedDistributor() throws Exception {
FakeFsDirectory[] directories = new FakeFsDirectory[]{
new FakeFsDirectory("dir0", 10L),
new FakeFsDirectory("dir1", 20L),
new FakeFsDirectory("dir2", 30L)
};
FakeDirectoryService directoryService = new FakeDirectoryService(directories);
RandomWeightedDistributor randomWeightedDistributor = new RandomWeightedDistributor(directoryService) {
@Override
protected long getUsableSpace(Directory directory) throws IOException {
return ((FakeFsDirectory)directory).useableSpace;
}
};
for (int i = 0; i < 10000; i++) {
((FakeFsDirectory) randomWeightedDistributor.any()).incrementAllocationCount();
}
for (FakeFsDirectory directory : directories) {
assertThat(directory.getAllocationCount(), greaterThan(0));
}
assertThat((double) directories[1].getAllocationCount() / directories[0].getAllocationCount(), closeTo(2.0, 0.5));
assertThat((double) directories[2].getAllocationCount() / directories[0].getAllocationCount(), closeTo(3.0, 0.5));
for (FakeFsDirectory directory : directories) {
directory.resetAllocationCount();
}
directories[1].setUsableSpace(0L);
for (int i = 0; i < 1000; i++) {
((FakeFsDirectory) randomWeightedDistributor.any()).incrementAllocationCount();
}
assertThat(directories[0].getAllocationCount(), greaterThan(0));
assertThat(directories[1].getAllocationCount(), equalTo(0));
assertThat(directories[2].getAllocationCount(), greaterThan(0));
}
public class FakeDirectoryService extends DirectoryService {
private final Directory[] directories;
public FakeDirectoryService(Directory[] directories) {
super(new ShardId("fake", 1), ImmutableSettings.EMPTY);
this.directories = directories;
}
@Override
public Directory[] build() throws IOException {
return directories;
}
@Override
public long throttleTimeInNanos() {
return 0;
}
}
public class FakeFsDirectory extends FSDirectory {
public int allocationCount;
public long useableSpace;
public FakeFsDirectory(String path, long usableSpace) throws IOException {
super(createTempDir().resolve(path), NoLockFactory.INSTANCE);
allocationCount = 0;
this.useableSpace = usableSpace;
}
@Override
public IndexInput openInput(String name, IOContext context) throws IOException {
throw new UnsupportedOperationException("Shouldn't be called in the test");
}
public void setUsableSpace(long usableSpace) {
this.useableSpace = usableSpace;
}
public void incrementAllocationCount() {
allocationCount++;
}
public int getAllocationCount() {
return allocationCount;
}
public void resetAllocationCount() {
allocationCount = 0;
}
}
}

View File

@ -67,6 +67,7 @@ public abstract class AbstractSimpleTranslogTests extends ElasticsearchTestCase
@Before
public void setUp() throws Exception {
super.setUp();
// if a previous test failed we clean up things here
translogDir = createTempDir();
translog = create(translogDir);
translog.newTranslog(1);
@ -383,18 +384,14 @@ public abstract class AbstractSimpleTranslogTests extends ElasticsearchTestCase
}
public void assertFileIsPresent(Translog translog, long id) {
for (Path location : translog.locations()) {
if (Files.exists(location.resolve(translog.getFilename(id)))) {
return;
}
if(Files.exists(translog.location().resolve(translog.getFilename(id)))) {
return;
}
fail(translog.getFilename(id) + " is not present in any location: " + Arrays.toString(translog.locations()));
fail(translog.getFilename(id) + " is not present in any location: " + translog.location());
}
public void assertFileDeleted(Translog translog, long id) {
for (Path location : translog.locations()) {
assertFalse(Files.exists(location.resolve(translog.getFilename(id))));
}
assertFalse(Files.exists(translog.location().resolve(translog.getFilename(id))));
}
@Test

View File

@ -29,11 +29,10 @@ import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.gateway.GatewayMetaState;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardPath;
import org.elasticsearch.test.ElasticsearchSingleNodeTest;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Path;
import java.util.concurrent.TimeUnit;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
@ -44,6 +43,9 @@ public class IndicesServiceTest extends ElasticsearchSingleNodeTest {
public IndicesService getIndicesService() {
return getInstanceFromNode(IndicesService.class);
}
public NodeEnvironment getNodeEnvironment() {
return getInstanceFromNode(NodeEnvironment.class);
}
@Override
protected boolean resetNodeAfterTest() {
@ -87,17 +89,14 @@ public class IndicesServiceTest extends ElasticsearchSingleNodeTest {
assertNull(meta.index("test"));
createIndex("test");
test = createIndex("test");
client().prepareIndex("test", "type", "1").setSource("field", "value").setRefresh(true).get();
client().admin().indices().prepareFlush("test").get();
assertHitCount(client().prepareSearch("test").get(), 1);
IndexMetaData secondMetaData = clusterService.state().metaData().index("test");
assertAcked(client().admin().indices().prepareClose("test"));
NodeEnvironment nodeEnv = getInstanceFromNode(NodeEnvironment.class);
Path[] paths = nodeEnv.shardDataPaths(new ShardId("test", 0), clusterService.state().getMetaData().index("test").getSettings());
for (Path path : paths) {
assertTrue(Files.exists(path));
}
ShardPath path = ShardPath.loadShardPath(logger, getNodeEnvironment(), new ShardId(test.index(), 0), test.getIndexSettings());
assertTrue(path.exists());
try {
indicesService.deleteIndexStore("boom", secondMetaData, clusterService.state());
@ -106,9 +105,7 @@ public class IndicesServiceTest extends ElasticsearchSingleNodeTest {
// all good
}
for (Path path : paths) {
assertTrue(Files.exists(path));
}
assertTrue(path.exists());
// now delete the old one and make sure we resolve against the name
try {
@ -124,19 +121,20 @@ public class IndicesServiceTest extends ElasticsearchSingleNodeTest {
public void testPendingTasks() throws IOException {
IndicesService indicesService = getIndicesService();
IndexService test = createIndex("test");
NodeEnvironment nodeEnv = getInstanceFromNode(NodeEnvironment.class);
assertTrue(test.hasShard(0));
Path[] paths = nodeEnv.shardDataPaths(new ShardId(test.index(), 0), test.getIndexSettings());
ShardPath path = test.shard(0).shardPath();
assertTrue(test.shard(0).routingEntry().started());
ShardPath shardPath = ShardPath.loadShardPath(logger, getNodeEnvironment(), new ShardId(test.index(), 0), test.getIndexSettings());
assertEquals(shardPath, path);
try {
indicesService.processPendingDeletes(test.index(), test.getIndexSettings(), new TimeValue(0, TimeUnit.MILLISECONDS));
fail("can't get lock");
} catch (LockObtainFailedException ex) {
}
for (Path p : paths) {
assertTrue(Files.exists(p));
}
assertTrue(path.exists());
int numPending = 1;
if (randomBoolean()) {
indicesService.addPendingDelete(new ShardId(test.index(), 0), test.getIndexSettings());
@ -148,16 +146,14 @@ public class IndicesServiceTest extends ElasticsearchSingleNodeTest {
indicesService.addPendingDelete(test.index(), test.getIndexSettings());
}
assertAcked(client().admin().indices().prepareClose("test"));
for (Path p : paths) {
assertTrue(Files.exists(p));
}
assertTrue(path.exists());
assertEquals(indicesService.numPendingDeletes(test.index()), numPending);
// shard lock released... we can now delete
indicesService.processPendingDeletes(test.index(), test.getIndexSettings(), new TimeValue(0, TimeUnit.MILLISECONDS));
assertEquals(indicesService.numPendingDeletes(test.index()), 0);
for (Path p : paths) {
assertFalse(Files.exists(p));
}
assertFalse(path.exists());
if (randomBoolean()) {
indicesService.addPendingDelete(new ShardId(test.index(), 0), test.getIndexSettings());

View File

@ -45,9 +45,7 @@ import java.nio.file.Files;
import java.nio.file.Path;
import java.util.Arrays;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder;
import static org.elasticsearch.test.ElasticsearchIntegrationTest.Scope;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.hamcrest.Matchers.equalTo;
@ -58,6 +56,11 @@ import static org.hamcrest.Matchers.equalTo;
@ClusterScope(scope = Scope.TEST, numDataNodes = 0)
public class IndicesStoreIntegrationTests extends ElasticsearchIntegrationTest {
@Override
protected Settings nodeSettings(int nodeOrdinal) { // simplify this and only use a single data path
return ImmutableSettings.settingsBuilder().put(super.nodeSettings(nodeOrdinal)).put("path.data", "").build();
}
@Test
public void indexCleanup() throws Exception {
final String masterNode = internalCluster().startNode(ImmutableSettings.builder().put("node.data", false));
@ -247,12 +250,16 @@ public class IndicesStoreIntegrationTests extends ElasticsearchIntegrationTest {
private Path indexDirectory(String server, String index) {
NodeEnvironment env = internalCluster().getInstance(NodeEnvironment.class, server);
return env.indexPaths(new Index(index))[0];
final Path[] paths = env.indexPaths(new Index(index));
assert paths.length == 1;
return paths[0];
}
private Path shardDirectory(String server, String index, int shard) {
NodeEnvironment env = internalCluster().getInstance(NodeEnvironment.class, server);
return env.shardPaths(new ShardId(index, shard))[0];
final Path[] paths = env.availableShardPaths(new ShardId(index, shard));
assert paths.length == 1;
return paths[0];
}
private boolean waitForShardDeletion(final String server, final String index, final int shard) throws InterruptedException {

View File

@ -1,160 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch 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.elasticsearch.indices.store;
import org.apache.lucene.store.Directory;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.store.IndexStoreModule;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.junit.Test;
import java.io.IOException;
import java.nio.file.Path;
import java.util.Locale;
import java.util.Set;
import static org.elasticsearch.common.settings.ImmutableSettings.settingsBuilder;
import static org.hamcrest.Matchers.*;
/**
*
*/
public class SimpleDistributorTests extends ElasticsearchIntegrationTest {
@Test
public void testAvailableSpaceDetection() {
for (IndexStoreModule.Type store : IndexStoreModule.Type.values()) {
createIndexWithStoreType("test", store, StrictDistributor.class.getCanonicalName());
}
}
@Test
public void testDirectoryToString() throws IOException {
internalCluster().wipeTemplates(); // no random settings please
createIndexWithStoreType("test", IndexStoreModule.Type.NIOFS, "least_used");
String storeString = getStoreDirectory("test", 0).toString();
logger.info(storeString);
Path[] dataPaths = dataPaths();
assertThat(storeString.toLowerCase(Locale.ROOT), startsWith("store(least_used[rate_limited(niofs(" + dataPaths[0].toAbsolutePath().normalize().toString().toLowerCase(Locale.ROOT)));
if (dataPaths.length > 1) {
assertThat(storeString.toLowerCase(Locale.ROOT), containsString("), rate_limited(niofs(" + dataPaths[1].toAbsolutePath().normalize().toString().toLowerCase(Locale.ROOT)));
}
assertThat(storeString, endsWith(", type=MERGE, rate=20.0)])"));
createIndexWithStoreType("test", IndexStoreModule.Type.NIOFS, "random");
storeString = getStoreDirectory("test", 0).toString();
logger.info(storeString);
dataPaths = dataPaths();
assertThat(storeString.toLowerCase(Locale.ROOT), startsWith("store(random[rate_limited(niofs(" + dataPaths[0].toAbsolutePath().normalize().toString().toLowerCase(Locale.ROOT)));
if (dataPaths.length > 1) {
assertThat(storeString.toLowerCase(Locale.ROOT), containsString("), rate_limited(niofs(" + dataPaths[1].toAbsolutePath().normalize().toString().toLowerCase(Locale.ROOT)));
}
assertThat(storeString, endsWith(", type=MERGE, rate=20.0)])"));
createIndexWithStoreType("test", IndexStoreModule.Type.MMAPFS, "least_used");
storeString = getStoreDirectory("test", 0).toString();
logger.info(storeString);
dataPaths = dataPaths();
assertThat(storeString.toLowerCase(Locale.ROOT), startsWith("store(least_used[rate_limited(mmapfs(" + dataPaths[0].toAbsolutePath().normalize().toString().toLowerCase(Locale.ROOT)));
if (dataPaths.length > 1) {
assertThat(storeString.toLowerCase(Locale.ROOT), containsString("), rate_limited(mmapfs(" + dataPaths[1].toAbsolutePath().normalize().toString().toLowerCase(Locale.ROOT)));
}
assertThat(storeString, endsWith(", type=MERGE, rate=20.0)])"));
createIndexWithStoreType("test", IndexStoreModule.Type.SIMPLEFS, "least_used");
storeString = getStoreDirectory("test", 0).toString();
logger.info(storeString);
dataPaths = dataPaths();
assertThat(storeString.toLowerCase(Locale.ROOT), startsWith("store(least_used[rate_limited(simplefs(" + dataPaths[0].toAbsolutePath().normalize().toString().toLowerCase(Locale.ROOT)));
if (dataPaths.length > 1) {
assertThat(storeString.toLowerCase(Locale.ROOT), containsString("), rate_limited(simplefs(" + dataPaths[1].toAbsolutePath().normalize().toString().toLowerCase(Locale.ROOT)));
}
assertThat(storeString, endsWith(", type=MERGE, rate=20.0)])"));
createIndexWithStoreType("test", IndexStoreModule.Type.DEFAULT, "least_used");
storeString = getStoreDirectory("test", 0).toString();
logger.info(storeString);
dataPaths = dataPaths();
assertThat(storeString.toLowerCase(Locale.ROOT), startsWith("store(least_used[rate_limited(default(mmapfs(" + dataPaths[0].toAbsolutePath().normalize().toString().toLowerCase(Locale.ROOT)));
assertThat(storeString.toLowerCase(Locale.ROOT), containsString("),niofs(" + dataPaths[0].toAbsolutePath().normalize().toString().toLowerCase(Locale.ROOT)));
if (dataPaths.length > 1) {
assertThat(storeString.toLowerCase(Locale.ROOT), containsString("), rate_limited(default(mmapfs(" + dataPaths[1].toAbsolutePath().normalize().toString().toLowerCase(Locale.ROOT)));
}
assertThat(storeString, endsWith(", type=MERGE, rate=20.0)])"));
createIndexWithoutRateLimitingStoreType("test", IndexStoreModule.Type.NIOFS, "least_used");
storeString = getStoreDirectory("test", 0).toString();
logger.info(storeString);
dataPaths = dataPaths();
assertThat(storeString.toLowerCase(Locale.ROOT), startsWith("store(least_used[niofs(" + dataPaths[0].toAbsolutePath().normalize().toString().toLowerCase(Locale.ROOT)));
if (dataPaths.length > 1) {
assertThat(storeString.toLowerCase(Locale.ROOT), containsString("), niofs(" + dataPaths[1].toAbsolutePath().normalize().toString().toLowerCase(Locale.ROOT)));
}
assertThat(storeString, endsWith(")])"));
}
private void createIndexWithStoreType(String index, IndexStoreModule.Type storeType, String distributor) {
cluster().wipeIndices(index);
client().admin().indices().prepareCreate(index)
.setSettings(settingsBuilder()
.put("index.store.distributor", distributor)
.put("index.store.type", storeType.name())
.put("index.number_of_replicas", 0)
.put("index.number_of_shards", 1)
.put("index.store.throttle.type", "merge")
.put("index.store.throttle.max_bytes_per_sec", "20mb")
)
.execute().actionGet();
assertThat(client().admin().cluster().prepareHealth("test").setWaitForGreenStatus().execute().actionGet().isTimedOut(), equalTo(false));
}
private void createIndexWithoutRateLimitingStoreType(String index, IndexStoreModule.Type storeType, String distributor) {
cluster().wipeIndices(index);
client().admin().indices().prepareCreate(index)
.setSettings(settingsBuilder()
.put("index.store.distributor", distributor)
.put("index.store.type", storeType)
.put("index.store.throttle.type", "none")
.put("index.number_of_replicas", 0)
.put("index.number_of_shards", 1)
)
.execute().actionGet();
assertThat(client().admin().cluster().prepareHealth("test").setWaitForGreenStatus().execute().actionGet().isTimedOut(), equalTo(false));
}
private Path[] dataPaths() {
Set<String> nodes = internalCluster().nodesInclude("test");
assertThat(nodes.isEmpty(), equalTo(false));
NodeEnvironment env = internalCluster().getInstance(NodeEnvironment.class, nodes.iterator().next());
return env.nodeDataPaths();
}
private Directory getStoreDirectory(String index, int shardId) {
Set<String> nodes = internalCluster().nodesInclude("test");
assertThat(nodes.isEmpty(), equalTo(false));
IndicesService indicesService = internalCluster().getInstance(IndicesService.class, nodes.iterator().next());
IndexShard indexShard = indicesService.indexService(index).shardSafe(shardId);
return indexShard.store().directory();
}
}

View File

@ -1,55 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch 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.elasticsearch.indices.store;
import org.apache.lucene.store.Directory;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.index.store.DirectoryService;
import org.elasticsearch.index.store.distributor.AbstractDistributor;
import java.io.IOException;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.greaterThan;
/**
*
*/
public class StrictDistributor extends AbstractDistributor {
@Inject
public StrictDistributor(DirectoryService directoryService) throws IOException {
super(directoryService);
}
@Override
public Directory doAny() throws IOException {
for (Directory delegate : delegates) {
assertThat(getUsableSpace(delegate), greaterThan(0L));
}
return primary();
}
@Override
public String name() {
return "strict";
}
}

View File

@ -32,7 +32,6 @@ import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.search.SearchPhaseExecutionException;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.search.SearchType;
import org.elasticsearch.client.Client;
import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterService;
@ -596,23 +595,25 @@ public class RelocationTests extends ElasticsearchIntegrationTest {
logger.info("--> verifying no temporary recoveries are left");
for (String node : internalCluster().getNodeNames()) {
NodeEnvironment nodeEnvironment = internalCluster().getInstance(NodeEnvironment.class, node);
for (final Path shardLoc : nodeEnvironment.shardPaths(new ShardId(indexName, 0))) {
assertBusy(new Runnable() {
@Override
public void run() {
try {
Files.walkFileTree(shardLoc, new SimpleFileVisitor<Path>() {
@Override
public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException {
assertThat("found a temporary recovery file: " + file, file.getFileName().toString(), not(startsWith("recovery.")));
return FileVisitResult.CONTINUE;
}
});
} catch (IOException e) {
throw new AssertionError("failed to walk file tree starting at [" + shardLoc + "]", e);
for (final Path shardLoc : nodeEnvironment.availableShardPaths(new ShardId(indexName, 0))) {
if (Files.exists(shardLoc)) {
assertBusy(new Runnable() {
@Override
public void run() {
try {
Files.walkFileTree(shardLoc, new SimpleFileVisitor<Path>() {
@Override
public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException {
assertThat("found a temporary recovery file: " + file, file.getFileName().toString(), not(startsWith("recovery.")));
return FileVisitResult.CONTINUE;
}
});
} catch (IOException e) {
throw new AssertionError("failed to walk file tree starting at [" + shardLoc + "]", e);
}
}
}
});
});
}
}
}
}

View File

@ -581,5 +581,4 @@ public abstract class ElasticsearchTestCase extends LuceneTestCase {
return threadGroup.getName();
}
}
}

View File

@ -31,6 +31,7 @@ import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardPath;
import org.elasticsearch.index.store.DirectoryService;
import org.elasticsearch.index.store.IndexStore;
import org.elasticsearch.index.store.fs.*;
@ -96,31 +97,24 @@ public class MockDirectoryHelper {
return w;
}
public Directory[] wrapAllInplace(Directory[] dirs) {
for (int i = 0; i < dirs.length; i++) {
dirs[i] = wrap(dirs[i]);
}
return dirs;
}
public FsDirectoryService randomDirectorService(IndexStore indexStore) {
public FsDirectoryService randomDirectorService(IndexStore indexStore, ShardPath path) {
if ((Constants.WINDOWS || Constants.SUN_OS) && Constants.JRE_IS_64BIT && MMapDirectory.UNMAP_SUPPORTED) {
return new MmapFsDirectoryService(shardId, indexSettings, indexStore);
return new MmapFsDirectoryService(shardId, indexSettings, indexStore, path);
} else if (Constants.WINDOWS) {
return new SimpleFsDirectoryService(shardId, indexSettings, indexStore);
return new SimpleFsDirectoryService(shardId, indexSettings, indexStore, path);
}
switch (random.nextInt(4)) {
case 2:
return new DefaultFsDirectoryService(shardId, indexSettings, indexStore);
return new DefaultFsDirectoryService(shardId, indexSettings, indexStore, path);
case 1:
return new MmapFsDirectoryService(shardId, indexSettings, indexStore);
return new MmapFsDirectoryService(shardId, indexSettings, indexStore, path);
case 0:
if (random.nextInt(10) == 0) {
// use simplefs less, it synchronizes all threads reads
return new SimpleFsDirectoryService(shardId, indexSettings, indexStore);
return new SimpleFsDirectoryService(shardId, indexSettings, indexStore, path);
}
default:
return new NioFsDirectoryService(shardId, indexSettings, indexStore);
return new NioFsDirectoryService(shardId, indexSettings, indexStore, path);
}
}
@ -170,7 +164,7 @@ public class MockDirectoryHelper {
/**
* Returns true if {@link #in} must sync its files.
* Currently, only {@link NRTCachingDirectory} requires sync'ing its files
* because otherwise they are cached in an internal {@link RAMDirectory}. If
* because otherwise they are cached in an internal {@link org.apache.lucene.store.RAMDirectory}. If
* other directories require that too, they should be added to this method.
*/
private boolean mustSync() {

View File

@ -34,13 +34,9 @@ import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.settings.IndexSettings;
import org.elasticsearch.index.shard.IndexShardException;
import org.elasticsearch.index.shard.IndexShardState;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.*;
import org.elasticsearch.index.store.IndexStore;
import org.elasticsearch.index.store.Store;
import org.elasticsearch.index.store.distributor.Distributor;
import org.elasticsearch.index.store.fs.FsDirectoryService;
import org.elasticsearch.indices.IndicesLifecycle;
import org.elasticsearch.indices.IndicesService;
@ -66,14 +62,14 @@ public class MockFSDirectoryService extends FsDirectoryService {
private final boolean checkIndexOnClose;
@Inject
public MockFSDirectoryService(final ShardId shardId, @IndexSettings Settings indexSettings, IndexStore indexStore, final IndicesService service) {
super(shardId, indexSettings, indexStore);
public MockFSDirectoryService(final ShardId shardId, @IndexSettings Settings indexSettings, IndexStore indexStore, final IndicesService service, final ShardPath path) {
super(shardId, indexSettings, indexStore, path);
final long seed = indexSettings.getAsLong(ElasticsearchIntegrationTest.SETTING_INDEX_SEED, 0l);
Random random = new Random(seed);
helper = new MockDirectoryHelper(shardId, indexSettings, logger, random, seed);
checkIndexOnClose = indexSettings.getAsBoolean(CHECK_INDEX_ON_CLOSE, true);
delegateService = helper.randomDirectorService(indexStore);
delegateService = helper.randomDirectorService(indexStore, path);
if (checkIndexOnClose) {
final IndicesLifecycle.Listener listener = new IndicesLifecycle.Listener() {
@ -112,9 +108,11 @@ public class MockFSDirectoryService extends FsDirectoryService {
}
}
@Override
public Directory[] build() throws IOException {
return delegateService.build();
public Directory newDirectory() throws IOException {
return helper.wrap(delegateService.newDirectory());
}
@Override
@ -175,9 +173,4 @@ public class MockFSDirectoryService extends FsDirectoryService {
public long throttleTimeInNanos() {
return delegateService.throttleTimeInNanos();
}
@Override
public Directory newFromDistributor(Distributor distributor) throws IOException {
return helper.wrap(super.newFromDistributor(distributor));
}
}