Minimize the usage of guava classes in interfaces, return types, arguments

Using these classes in interfaces forces downstream users to use shaded APIs.
We should minimize the usage of shaded APIs and keep them internal.
This commit is contained in:
Simon Willnauer 2015-06-04 21:22:16 +02:00
parent 6dd6ce2f8f
commit 6a90c64202
40 changed files with 172 additions and 133 deletions

View File

@ -28,6 +28,7 @@ import org.elasticsearch.common.settings.Settings;
import java.io.IOException; import java.io.IOException;
import java.util.Iterator; import java.util.Iterator;
import java.util.List;
/** /**
* Get repositories response * Get repositories response
@ -49,7 +50,7 @@ public class GetRepositoriesResponse extends ActionResponse implements Iterable<
* *
* @return list or repositories * @return list or repositories
*/ */
public ImmutableList<RepositoryMetaData> repositories() { public List<RepositoryMetaData> repositories() {
return repositories; return repositories;
} }

View File

@ -29,6 +29,7 @@ import org.elasticsearch.common.xcontent.XContentBuilderString;
import org.elasticsearch.snapshots.SnapshotInfo; import org.elasticsearch.snapshots.SnapshotInfo;
import java.io.IOException; import java.io.IOException;
import java.util.List;
/** /**
* Get snapshots response * Get snapshots response
@ -49,7 +50,7 @@ public class GetSnapshotsResponse extends ActionResponse implements ToXContent {
* *
* @return the list of snapshots * @return the list of snapshots
*/ */
public ImmutableList<SnapshotInfo> getSnapshots() { public List<SnapshotInfo> getSnapshots() {
return snapshots; return snapshots;
} }

View File

@ -36,6 +36,8 @@ import org.elasticsearch.snapshots.SnapshotsService;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.TransportService;
import java.util.List;
/** /**
* Transport Action for get snapshots operation * Transport Action for get snapshots operation
*/ */
@ -69,12 +71,12 @@ public class TransportGetSnapshotsAction extends TransportMasterNodeAction<GetSn
try { try {
ImmutableList.Builder<SnapshotInfo> snapshotInfoBuilder = ImmutableList.builder(); ImmutableList.Builder<SnapshotInfo> snapshotInfoBuilder = ImmutableList.builder();
if (isAllSnapshots(request.snapshots())) { if (isAllSnapshots(request.snapshots())) {
ImmutableList<Snapshot> snapshots = snapshotsService.snapshots(request.repository()); List<Snapshot> snapshots = snapshotsService.snapshots(request.repository());
for (Snapshot snapshot : snapshots) { for (Snapshot snapshot : snapshots) {
snapshotInfoBuilder.add(new SnapshotInfo(snapshot)); snapshotInfoBuilder.add(new SnapshotInfo(snapshot));
} }
} else if (isCurrentSnapshots(request.snapshots())) { } else if (isCurrentSnapshots(request.snapshots())) {
ImmutableList<Snapshot> snapshots = snapshotsService.currentSnapshots(request.repository()); List<Snapshot> snapshots = snapshotsService.currentSnapshots(request.repository());
for (Snapshot snapshot : snapshots) { for (Snapshot snapshot : snapshots) {
snapshotInfoBuilder.add(new SnapshotInfo(snapshot)); snapshotInfoBuilder.add(new SnapshotInfo(snapshot));
} }

View File

@ -41,6 +41,7 @@ import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.TransportService;
import java.io.IOException; import java.io.IOException;
import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
@ -81,7 +82,7 @@ public class TransportSnapshotsStatusAction extends TransportMasterNodeAction<Sn
protected void masterOperation(final SnapshotsStatusRequest request, protected void masterOperation(final SnapshotsStatusRequest request,
final ClusterState state, final ClusterState state,
final ActionListener<SnapshotsStatusResponse> listener) throws Exception { final ActionListener<SnapshotsStatusResponse> listener) throws Exception {
ImmutableList<SnapshotMetaData.Entry> currentSnapshots = snapshotsService.currentSnapshots(request.repository(), request.snapshots()); List<SnapshotMetaData.Entry> currentSnapshots = snapshotsService.currentSnapshots(request.repository(), request.snapshots());
if (currentSnapshots.isEmpty()) { if (currentSnapshots.isEmpty()) {
listener.onResponse(buildResponse(request, currentSnapshots, null)); listener.onResponse(buildResponse(request, currentSnapshots, null));
@ -110,7 +111,7 @@ public class TransportSnapshotsStatusAction extends TransportMasterNodeAction<Sn
@Override @Override
public void onResponse(TransportNodesSnapshotsStatus.NodesSnapshotStatus nodeSnapshotStatuses) { public void onResponse(TransportNodesSnapshotsStatus.NodesSnapshotStatus nodeSnapshotStatuses) {
try { try {
ImmutableList<SnapshotMetaData.Entry> currentSnapshots = List<SnapshotMetaData.Entry> currentSnapshots =
snapshotsService.currentSnapshots(request.repository(), request.snapshots()); snapshotsService.currentSnapshots(request.repository(), request.snapshots());
listener.onResponse(buildResponse(request, currentSnapshots, nodeSnapshotStatuses)); listener.onResponse(buildResponse(request, currentSnapshots, nodeSnapshotStatuses));
} catch (Throwable e) { } catch (Throwable e) {
@ -130,7 +131,7 @@ public class TransportSnapshotsStatusAction extends TransportMasterNodeAction<Sn
} }
private SnapshotsStatusResponse buildResponse(SnapshotsStatusRequest request, ImmutableList<SnapshotMetaData.Entry> currentSnapshots, private SnapshotsStatusResponse buildResponse(SnapshotsStatusRequest request, List<SnapshotMetaData.Entry> currentSnapshots,
TransportNodesSnapshotsStatus.NodesSnapshotStatus nodeSnapshotStatuses) throws IOException { TransportNodesSnapshotsStatus.NodesSnapshotStatus nodeSnapshotStatuses) throws IOException {
// First process snapshot that are currently processed // First process snapshot that are currently processed
ImmutableList.Builder<SnapshotStatus> builder = ImmutableList.builder(); ImmutableList.Builder<SnapshotStatus> builder = ImmutableList.builder();

View File

@ -19,8 +19,6 @@
package org.elasticsearch.client.transport; package org.elasticsearch.client.transport;
import com.google.common.collect.ImmutableList;
import org.elasticsearch.Version; import org.elasticsearch.Version;
import org.elasticsearch.action.Action; import org.elasticsearch.action.Action;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
@ -56,6 +54,7 @@ import org.elasticsearch.transport.TransportModule;
import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.TransportService;
import org.elasticsearch.transport.netty.NettyTransport; import org.elasticsearch.transport.netty.NettyTransport;
import java.util.List;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import static org.elasticsearch.common.settings.Settings.settingsBuilder; import static org.elasticsearch.common.settings.Settings.settingsBuilder;
@ -180,7 +179,7 @@ public class TransportClient extends AbstractClient {
* Returns the current registered transport addresses to use (added using * Returns the current registered transport addresses to use (added using
* {@link #addTransportAddress(org.elasticsearch.common.transport.TransportAddress)}. * {@link #addTransportAddress(org.elasticsearch.common.transport.TransportAddress)}.
*/ */
public ImmutableList<TransportAddress> transportAddresses() { public List<TransportAddress> transportAddresses() {
return nodesService.transportAddresses(); return nodesService.transportAddresses();
} }
@ -190,7 +189,7 @@ public class TransportClient extends AbstractClient {
* <p>The nodes include all the nodes that are currently alive based on the transport * <p>The nodes include all the nodes that are currently alive based on the transport
* addresses provided. * addresses provided.
*/ */
public ImmutableList<DiscoveryNode> connectedNodes() { public List<DiscoveryNode> connectedNodes() {
return nodesService.connectedNodes(); return nodesService.connectedNodes();
} }
@ -198,14 +197,14 @@ public class TransportClient extends AbstractClient {
* The list of filtered nodes that were not connected to, for example, due to * The list of filtered nodes that were not connected to, for example, due to
* mismatch in cluster name. * mismatch in cluster name.
*/ */
public ImmutableList<DiscoveryNode> filteredNodes() { public List<DiscoveryNode> filteredNodes() {
return nodesService.filteredNodes(); return nodesService.filteredNodes();
} }
/** /**
* Returns the listed nodes in the transport client (ones added to it). * Returns the listed nodes in the transport client (ones added to it).
*/ */
public ImmutableList<DiscoveryNode> listedNodes() { public List<DiscoveryNode> listedNodes() {
return nodesService.listedNodes(); return nodesService.listedNodes();
} }

View File

@ -77,8 +77,8 @@ public class TransportClientNodesService extends AbstractComponent {
private final Object mutex = new Object(); private final Object mutex = new Object();
private volatile ImmutableList<DiscoveryNode> nodes = ImmutableList.of(); private volatile List<DiscoveryNode> nodes = ImmutableList.of();
private volatile ImmutableList<DiscoveryNode> filteredNodes = ImmutableList.of(); private volatile List<DiscoveryNode> filteredNodes = ImmutableList.of();
private final AtomicInteger tempNodeIdGenerator = new AtomicInteger(); private final AtomicInteger tempNodeIdGenerator = new AtomicInteger();
@ -118,7 +118,7 @@ public class TransportClientNodesService extends AbstractComponent {
this.nodesSamplerFuture = threadPool.schedule(nodesSamplerInterval, ThreadPool.Names.GENERIC, new ScheduledNodeSampler()); this.nodesSamplerFuture = threadPool.schedule(nodesSamplerInterval, ThreadPool.Names.GENERIC, new ScheduledNodeSampler());
} }
public ImmutableList<TransportAddress> transportAddresses() { public List<TransportAddress> transportAddresses() {
ImmutableList.Builder<TransportAddress> lstBuilder = ImmutableList.builder(); ImmutableList.Builder<TransportAddress> lstBuilder = ImmutableList.builder();
for (DiscoveryNode listedNode : listedNodes) { for (DiscoveryNode listedNode : listedNodes) {
lstBuilder.add(listedNode.address()); lstBuilder.add(listedNode.address());
@ -126,15 +126,15 @@ public class TransportClientNodesService extends AbstractComponent {
return lstBuilder.build(); return lstBuilder.build();
} }
public ImmutableList<DiscoveryNode> connectedNodes() { public List<DiscoveryNode> connectedNodes() {
return this.nodes; return this.nodes;
} }
public ImmutableList<DiscoveryNode> filteredNodes() { public List<DiscoveryNode> filteredNodes() {
return this.filteredNodes; return this.filteredNodes;
} }
public ImmutableList<DiscoveryNode> listedNodes() { public List<DiscoveryNode> listedNodes() {
return this.listedNodes; return this.listedNodes;
} }
@ -193,7 +193,7 @@ public class TransportClientNodesService extends AbstractComponent {
} }
public <Response> void execute(NodeListenerCallback<Response> callback, ActionListener<Response> listener) { public <Response> void execute(NodeListenerCallback<Response> callback, ActionListener<Response> listener) {
ImmutableList<DiscoveryNode> nodes = this.nodes; List<DiscoveryNode> nodes = this.nodes;
ensureNodesAreAvailable(nodes); ensureNodesAreAvailable(nodes);
int index = getNodeNumber(); int index = getNodeNumber();
RetryListener<Response> retryListener = new RetryListener<>(callback, listener, nodes, index); RetryListener<Response> retryListener = new RetryListener<>(callback, listener, nodes, index);
@ -209,12 +209,12 @@ public class TransportClientNodesService extends AbstractComponent {
public static class RetryListener<Response> implements ActionListener<Response> { public static class RetryListener<Response> implements ActionListener<Response> {
private final NodeListenerCallback<Response> callback; private final NodeListenerCallback<Response> callback;
private final ActionListener<Response> listener; private final ActionListener<Response> listener;
private final ImmutableList<DiscoveryNode> nodes; private final List<DiscoveryNode> nodes;
private final int index; private final int index;
private volatile int i; private volatile int i;
public RetryListener(NodeListenerCallback<Response> callback, ActionListener<Response> listener, ImmutableList<DiscoveryNode> nodes, int index) { public RetryListener(NodeListenerCallback<Response> callback, ActionListener<Response> listener, List<DiscoveryNode> nodes, int index) {
this.callback = callback; this.callback = callback;
this.listener = listener; this.listener = listener;
this.nodes = nodes; this.nodes = nodes;
@ -274,7 +274,7 @@ public class TransportClientNodesService extends AbstractComponent {
return index; return index;
} }
private void ensureNodesAreAvailable(ImmutableList<DiscoveryNode> nodes) { private void ensureNodesAreAvailable(List<DiscoveryNode> nodes) {
if (nodes.isEmpty()) { if (nodes.isEmpty()) {
String message = String.format(Locale.ROOT, "None of the configured nodes are available: %s", nodes); String message = String.format(Locale.ROOT, "None of the configured nodes are available: %s", nodes);
throw new NoNodeAvailableException(message); throw new NoNodeAvailableException(message);
@ -297,7 +297,7 @@ public class TransportClientNodesService extends AbstractComponent {
* validates a set of potentially newly discovered nodes and returns an immutable * validates a set of potentially newly discovered nodes and returns an immutable
* list of the nodes that has passed. * list of the nodes that has passed.
*/ */
protected ImmutableList<DiscoveryNode> validateNewNodes(Set<DiscoveryNode> nodes) { protected List<DiscoveryNode> validateNewNodes(Set<DiscoveryNode> nodes) {
for (Iterator<DiscoveryNode> it = nodes.iterator(); it.hasNext(); ) { for (Iterator<DiscoveryNode> it = nodes.iterator(); it.hasNext(); ) {
DiscoveryNode node = it.next(); DiscoveryNode node = it.next();
if (!transportService.nodeConnected(node)) { if (!transportService.nodeConnected(node)) {

View File

@ -31,6 +31,7 @@ import org.elasticsearch.index.shard.ShardId;
import java.io.IOException; import java.io.IOException;
import java.util.EnumSet; import java.util.EnumSet;
import java.util.List;
import java.util.Map; import java.util.Map;
/** /**
@ -67,7 +68,7 @@ public class RestoreMetaData extends AbstractDiffable<MetaData.Custom> implement
* *
* @return list of currently running restore processes * @return list of currently running restore processes
*/ */
public ImmutableList<Entry> entries() { public List<Entry> entries() {
return this.entries; return this.entries;
} }

View File

@ -33,6 +33,7 @@ import org.elasticsearch.index.shard.ShardId;
import java.io.IOException; import java.io.IOException;
import java.util.EnumSet; import java.util.EnumSet;
import java.util.List;
import java.util.Map; import java.util.Map;
import static com.google.common.collect.Maps.newHashMap; import static com.google.common.collect.Maps.newHashMap;
@ -319,7 +320,7 @@ public class SnapshotMetaData extends AbstractDiffable<Custom> implements MetaDa
this.entries = ImmutableList.copyOf(entries); this.entries = ImmutableList.copyOf(entries);
} }
public ImmutableList<Entry> entries() { public List<Entry> entries() {
return this.entries; return this.entries;
} }

View File

@ -508,7 +508,7 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
return !removed.isEmpty(); return !removed.isEmpty();
} }
public ImmutableList<DiscoveryNode> removedNodes() { public List<DiscoveryNode> removedNodes() {
return removed; return removed;
} }
@ -516,7 +516,7 @@ public class DiscoveryNodes extends AbstractDiffable<DiscoveryNodes> implements
return !added.isEmpty(); return !added.isEmpty();
} }
public ImmutableList<DiscoveryNode> addedNodes() { public List<DiscoveryNode> addedNodes() {
return added; return added;
} }

View File

@ -63,10 +63,10 @@ public class IndexShardRoutingTable implements Iterable<ShardRouting> {
final boolean primaryAllocatedPostApi; final boolean primaryAllocatedPostApi;
IndexShardRoutingTable(ShardId shardId, ImmutableList<ShardRouting> shards, boolean primaryAllocatedPostApi) { IndexShardRoutingTable(ShardId shardId, List<ShardRouting> shards, boolean primaryAllocatedPostApi) {
this.shardId = shardId; this.shardId = shardId;
this.shuffler = new RotationShardShuffler(ThreadLocalRandom.current().nextInt()); this.shuffler = new RotationShardShuffler(ThreadLocalRandom.current().nextInt());
this.shards = shards; this.shards = ImmutableList.copyOf(shards);
this.primaryAllocatedPostApi = primaryAllocatedPostApi; this.primaryAllocatedPostApi = primaryAllocatedPostApi;
ShardRouting primary = null; ShardRouting primary = null;
@ -173,7 +173,7 @@ public class IndexShardRoutingTable implements Iterable<ShardRouting> {
} }
@Override @Override
public UnmodifiableIterator<ShardRouting> iterator() { public Iterator<ShardRouting> iterator() {
return shards.iterator(); return shards.iterator();
} }
@ -192,56 +192,56 @@ public class IndexShardRoutingTable implements Iterable<ShardRouting> {
} }
/** /**
* Returns a {@link ImmutableList} of shards * Returns a {@link List} of shards
* *
* @return a {@link ImmutableList} of shards * @return a {@link List} of shards
*/ */
public ImmutableList<ShardRouting> shards() { public List<ShardRouting> shards() {
return this.shards; return this.shards;
} }
/** /**
* Returns a {@link ImmutableList} of shards * Returns a {@link List} of shards
* *
* @return a {@link ImmutableList} of shards * @return a {@link List} of shards
*/ */
public ImmutableList<ShardRouting> getShards() { public List<ShardRouting> getShards() {
return shards(); return shards();
} }
/** /**
* Returns a {@link ImmutableList} of active shards * Returns a {@link List} of active shards
* *
* @return a {@link ImmutableList} of shards * @return a {@link List} of shards
*/ */
public ImmutableList<ShardRouting> activeShards() { public List<ShardRouting> activeShards() {
return this.activeShards; return this.activeShards;
} }
/** /**
* Returns a {@link ImmutableList} of active shards * Returns a {@link List} of active shards
* *
* @return a {@link ImmutableList} of shards * @return a {@link List} of shards
*/ */
public ImmutableList<ShardRouting> getActiveShards() { public List<ShardRouting> getActiveShards() {
return activeShards(); return activeShards();
} }
/** /**
* Returns a {@link ImmutableList} of assigned shards * Returns a {@link List} of assigned shards
* *
* @return a {@link ImmutableList} of shards * @return a {@link List} of shards
*/ */
public ImmutableList<ShardRouting> assignedShards() { public List<ShardRouting> assignedShards() {
return this.assignedShards; return this.assignedShards;
} }
/** /**
* Returns a {@link ImmutableList} of assigned shards * Returns a {@link List} of assigned shards
* *
* @return a {@link ImmutableList} of shards * @return a {@link List} of shards
*/ */
public ImmutableList<ShardRouting> getAssignedShards() { public List<ShardRouting> getAssignedShards() {
return this.assignedShards; return this.assignedShards;
} }

View File

@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableMap;
import java.io.IOException; import java.io.IOException;
import java.io.InputStream; import java.io.InputStream;
import java.io.OutputStream; import java.io.OutputStream;
import java.util.Map;
/** /**
* *
@ -59,12 +60,12 @@ public interface BlobContainer {
/** /**
* Lists all blobs in the container * Lists all blobs in the container
*/ */
ImmutableMap<String, BlobMetaData> listBlobs() throws IOException; Map<String, BlobMetaData> listBlobs() throws IOException;
/** /**
* Lists all blobs in the container that match specified prefix * Lists all blobs in the container that match specified prefix
*/ */
ImmutableMap<String, BlobMetaData> listBlobsByPrefix(String blobNamePrefix) throws IOException; Map<String, BlobMetaData> listBlobsByPrefix(String blobNamePrefix) throws IOException;
/** /**
* Atomically renames source blob into target blob * Atomically renames source blob into target blob

View File

@ -25,6 +25,7 @@ import org.elasticsearch.common.blobstore.BlobMetaData;
import org.elasticsearch.common.blobstore.BlobPath; import org.elasticsearch.common.blobstore.BlobPath;
import java.io.IOException; import java.io.IOException;
import java.util.Map;
/** /**
* *
@ -44,7 +45,7 @@ public abstract class AbstractBlobContainer implements BlobContainer {
@Override @Override
public void deleteBlobsByPrefix(final String blobNamePrefix) throws IOException { public void deleteBlobsByPrefix(final String blobNamePrefix) throws IOException {
ImmutableMap<String, BlobMetaData> blobs = listBlobsByPrefix(blobNamePrefix); Map<String, BlobMetaData> blobs = listBlobsByPrefix(blobNamePrefix);
for (BlobMetaData blob : blobs.values()) { for (BlobMetaData blob : blobs.values()) {
deleteBlob(blob.name()); deleteBlob(blob.name());
} }

View File

@ -24,13 +24,17 @@ import com.carrotsearch.hppc.FloatArrayList;
import com.carrotsearch.hppc.LongArrayList; import com.carrotsearch.hppc.LongArrayList;
import com.carrotsearch.hppc.ObjectArrayList; import com.carrotsearch.hppc.ObjectArrayList;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterators;
import com.google.common.collect.Lists;
import org.apache.lucene.util.*; import org.apache.lucene.util.*;
import org.elasticsearch.common.inject.Module;
import java.util.*; import java.util.*;
/** Collections-related utility methods. */ /** Collections-related utility methods. */
public enum CollectionUtils { public enum CollectionUtils {
; CollectionUtils;
public static void sort(LongArrayList list) { public static void sort(LongArrayList list) {
sort(list.buffer, list.size()); sort(list.buffer, list.size());
@ -357,5 +361,15 @@ public enum CollectionUtils {
} }
/**
* Combines multiple iterators into a single iterator.
*/
public static <T> Iterator<T> concat(Iterator<? extends T>... iterators) {
return Iterators.concat(iterators);
}
public static <E> ArrayList<E> newArrayList(E... elements) {
return Lists.newArrayList(elements);
}
} }

View File

@ -152,7 +152,7 @@ public class Analysis {
.put("_turkish_", TurkishAnalyzer.getDefaultStopSet()) .put("_turkish_", TurkishAnalyzer.getDefaultStopSet())
.immutableMap(); .immutableMap();
public static CharArraySet parseWords(Environment env, Settings settings, String name, CharArraySet defaultWords, ImmutableMap<String, Set<?>> namedWords, boolean ignoreCase) { public static CharArraySet parseWords(Environment env, Settings settings, String name, CharArraySet defaultWords, Map<String, Set<?>> namedWords, boolean ignoreCase) {
String value = settings.get(name); String value = settings.get(name);
if (value != null) { if (value != null) {
if ("_none_".equals(value)) { if ("_none_".equals(value)) {
@ -184,7 +184,7 @@ public class Analysis {
return parseWords(env, settings, "stopwords", defaultStopWords, namedStopWords, ignoreCase); return parseWords(env, settings, "stopwords", defaultStopWords, namedStopWords, ignoreCase);
} }
private static CharArraySet resolveNamedWords(Collection<String> words, ImmutableMap<String, Set<?>> namedWords, boolean ignoreCase) { private static CharArraySet resolveNamedWords(Collection<String> words, Map<String, Set<?>> namedWords, boolean ignoreCase) {
if (namedWords == null) { if (namedWords == null) {
return new CharArraySet(words, ignoreCase); return new CharArraySet(words, ignoreCase);
} }

View File

@ -105,11 +105,11 @@ public class CommitPoint {
return this.type; return this.type;
} }
public ImmutableList<FileInfo> indexFiles() { public List<FileInfo> indexFiles() {
return this.indexFiles; return this.indexFiles;
} }
public ImmutableList<FileInfo> translogFiles() { public List<FileInfo> translogFiles() {
return this.translogFiles; return this.translogFiles;
} }

View File

@ -49,7 +49,7 @@ public class CommitPoints implements Iterable<CommitPoint> {
this.commitPoints = ImmutableList.copyOf(commitPoints); this.commitPoints = ImmutableList.copyOf(commitPoints);
} }
public ImmutableList<CommitPoint> commits() { public List<CommitPoint> commits() {
return this.commitPoints; return this.commitPoints;
} }

View File

@ -43,6 +43,7 @@ import org.elasticsearch.index.mapper.object.RootObjectMapper;
import java.io.Closeable; import java.io.Closeable;
import java.io.IOException; import java.io.IOException;
import java.util.Collections; import java.util.Collections;
import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
@ -598,7 +599,7 @@ class DocumentParser implements Closeable {
} }
/** Creates instances of the fields that the current field should be copied to */ /** Creates instances of the fields that the current field should be copied to */
private static void parseCopyFields(ParseContext context, FieldMapper fieldMapper, ImmutableList<String> copyToFields) throws IOException { private static void parseCopyFields(ParseContext context, FieldMapper fieldMapper, List<String> copyToFields) throws IOException {
if (!context.isWithinCopyTo() && copyToFields.isEmpty() == false) { if (!context.isWithinCopyTo() && copyToFields.isEmpty() == false) {
context = context.createCopyToContext(); context = context.createCopyToContext();
for (String field : copyToFields) { for (String field : copyToFields) {

View File

@ -891,7 +891,7 @@ public abstract class AbstractFieldMapper implements FieldMapper {
} }
} }
public ImmutableList<String> copyToFields() { public List<String> copyToFields() {
return copyToFields; return copyToFields;
} }
} }

View File

@ -318,7 +318,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
* Delete shard snapshot * Delete shard snapshot
*/ */
public void delete() { public void delete() {
final ImmutableMap<String, BlobMetaData> blobs; final Map<String, BlobMetaData> blobs;
try { try {
blobs = blobContainer.listBlobs(); blobs = blobContainer.listBlobs();
} catch (IOException e) { } catch (IOException e) {
@ -372,7 +372,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
* @param fileListGeneration the generation number of the snapshot index file * @param fileListGeneration the generation number of the snapshot index file
* @param blobs list of blobs in the container * @param blobs list of blobs in the container
*/ */
protected void finalize(List<SnapshotFiles> snapshots, int fileListGeneration, ImmutableMap<String, BlobMetaData> blobs) { protected void finalize(List<SnapshotFiles> snapshots, int fileListGeneration, Map<String, BlobMetaData> blobs) {
BlobStoreIndexShardSnapshots newSnapshots = new BlobStoreIndexShardSnapshots(snapshots); BlobStoreIndexShardSnapshots newSnapshots = new BlobStoreIndexShardSnapshots(snapshots);
// delete old index files first // delete old index files first
for (String blobName : blobs.keySet()) { for (String blobName : blobs.keySet()) {
@ -437,7 +437,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
* @param blobs list of blobs in the repository * @param blobs list of blobs in the repository
* @return next available blob number * @return next available blob number
*/ */
protected long findLatestFileNameGeneration(ImmutableMap<String, BlobMetaData> blobs) { protected long findLatestFileNameGeneration(Map<String, BlobMetaData> blobs) {
long generation = -1; long generation = -1;
for (String name : blobs.keySet()) { for (String name : blobs.keySet()) {
if (!name.startsWith(DATA_BLOB_PREFIX)) { if (!name.startsWith(DATA_BLOB_PREFIX)) {
@ -462,7 +462,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
* @param blobs list of blobs in repository * @param blobs list of blobs in repository
* @return tuple of BlobStoreIndexShardSnapshots and the last snapshot index generation * @return tuple of BlobStoreIndexShardSnapshots and the last snapshot index generation
*/ */
protected Tuple<BlobStoreIndexShardSnapshots, Integer> buildBlobStoreIndexShardSnapshots(ImmutableMap<String, BlobMetaData> blobs) { protected Tuple<BlobStoreIndexShardSnapshots, Integer> buildBlobStoreIndexShardSnapshots(Map<String, BlobMetaData> blobs) {
int latest = -1; int latest = -1;
for (String name : blobs.keySet()) { for (String name : blobs.keySet()) {
if (name.startsWith(SNAPSHOT_INDEX_PREFIX)) { if (name.startsWith(SNAPSHOT_INDEX_PREFIX)) {
@ -538,7 +538,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
logger.debug("[{}] [{}] snapshot to [{}] ...", shardId, snapshotId, repositoryName); logger.debug("[{}] [{}] snapshot to [{}] ...", shardId, snapshotId, repositoryName);
store.incRef(); store.incRef();
try { try {
final ImmutableMap<String, BlobMetaData> blobs; final Map<String, BlobMetaData> blobs;
try { try {
blobs = blobContainer.listBlobs(); blobs = blobContainer.listBlobs();
} catch (IOException e) { } catch (IOException e) {
@ -570,7 +570,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
logger.trace("[{}] [{}] Processing [{}]", shardId, snapshotId, fileName); logger.trace("[{}] [{}] Processing [{}]", shardId, snapshotId, fileName);
final StoreFileMetaData md = metadata.get(fileName); final StoreFileMetaData md = metadata.get(fileName);
FileInfo existingFileInfo = null; FileInfo existingFileInfo = null;
ImmutableList<FileInfo> filesInfo = snapshots.findPhysicalIndexFiles(fileName); List<FileInfo> filesInfo = snapshots.findPhysicalIndexFiles(fileName);
if (filesInfo != null) { if (filesInfo != null) {
for (FileInfo fileInfo : filesInfo) { for (FileInfo fileInfo : filesInfo) {
try { try {
@ -696,7 +696,7 @@ public class BlobStoreIndexShardRepository extends AbstractComponent implements
* @param blobs list of blobs * @param blobs list of blobs
* @return true if file exists in the list of blobs * @return true if file exists in the list of blobs
*/ */
private boolean snapshotFileExistsInBlobs(BlobStoreIndexShardSnapshot.FileInfo fileInfo, ImmutableMap<String, BlobMetaData> blobs) { private boolean snapshotFileExistsInBlobs(BlobStoreIndexShardSnapshot.FileInfo fileInfo, Map<String, BlobMetaData> blobs) {
BlobMetaData blobMetaData = blobs.get(fileInfo.name()); BlobMetaData blobMetaData = blobs.get(fileInfo.name());
if (blobMetaData != null) { if (blobMetaData != null) {
return blobMetaData.length() == fileInfo.length(); return blobMetaData.length() == fileInfo.length();

View File

@ -364,7 +364,7 @@ public class BlobStoreIndexShardSnapshot {
* *
* @return list of files * @return list of files
*/ */
public ImmutableList<FileInfo> indexFiles() { public List<FileInfo> indexFiles() {
return indexFiles; return indexFiles;
} }

View File

@ -108,7 +108,7 @@ public class BlobStoreIndexShardSnapshots implements Iterable<SnapshotFiles>, To
* *
* @return list of snapshots * @return list of snapshots
*/ */
public ImmutableList<SnapshotFiles> snapshots() { public List<SnapshotFiles> snapshots() {
return this.shardSnapshots; return this.shardSnapshots;
} }
@ -118,7 +118,7 @@ public class BlobStoreIndexShardSnapshots implements Iterable<SnapshotFiles>, To
* @param physicalName original name * @param physicalName original name
* @return a list of file infos that match specified physical file or null if the file is not present in any of snapshots * @return a list of file infos that match specified physical file or null if the file is not present in any of snapshots
*/ */
public ImmutableList<FileInfo> findPhysicalIndexFiles(String physicalName) { public List<FileInfo> findPhysicalIndexFiles(String physicalName) {
return physicalFiles.get(physicalName); return physicalFiles.get(physicalName);
} }

View File

@ -21,6 +21,7 @@ package org.elasticsearch.index.snapshots.blobstore;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot.FileInfo; import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot.FileInfo;
import java.util.List;
import java.util.Map; import java.util.Map;
import static com.google.common.collect.Maps.newHashMap; import static com.google.common.collect.Maps.newHashMap;
@ -32,7 +33,7 @@ public class SnapshotFiles {
private final String snapshot; private final String snapshot;
private final ImmutableList<FileInfo> indexFiles; private final List<FileInfo> indexFiles;
private Map<String, FileInfo> physicalFiles = null; private Map<String, FileInfo> physicalFiles = null;
@ -40,7 +41,7 @@ public class SnapshotFiles {
return snapshot; return snapshot;
} }
public SnapshotFiles(String snapshot, ImmutableList<FileInfo> indexFiles ) { public SnapshotFiles(String snapshot, List<FileInfo> indexFiles ) {
this.snapshot = snapshot; this.snapshot = snapshot;
this.indexFiles = indexFiles; this.indexFiles = indexFiles;
} }
@ -48,7 +49,7 @@ public class SnapshotFiles {
/** /**
* Returns a list of file in the snapshot * Returns a list of file in the snapshot
*/ */
public ImmutableList<FileInfo> indexFiles() { public List<FileInfo> indexFiles() {
return indexFiles; return indexFiles;
} }

View File

@ -415,7 +415,7 @@ public class PluginsService extends AbstractComponent {
} }
} }
private ImmutableList<Tuple<PluginInfo,Plugin>> loadPluginsFromClasspath(Settings settings) { private List<Tuple<PluginInfo,Plugin>> loadPluginsFromClasspath(Settings settings) {
ImmutableList.Builder<Tuple<PluginInfo, Plugin>> plugins = ImmutableList.builder(); ImmutableList.Builder<Tuple<PluginInfo, Plugin>> plugins = ImmutableList.builder();
// Trying JVM plugins: looking for es-plugin.properties files // Trying JVM plugins: looking for es-plugin.properties files

View File

@ -26,6 +26,7 @@ import org.elasticsearch.snapshots.Snapshot;
import org.elasticsearch.snapshots.SnapshotShardFailure; import org.elasticsearch.snapshots.SnapshotShardFailure;
import java.io.IOException; import java.io.IOException;
import java.util.List;
/** /**
* Snapshot repository interface. * Snapshot repository interface.
@ -36,7 +37,7 @@ import java.io.IOException;
* <p/> * <p/>
* Typical snapshot usage pattern: * Typical snapshot usage pattern:
* <ul> * <ul>
* <li>Master calls {@link #initializeSnapshot(org.elasticsearch.cluster.metadata.SnapshotId, com.google.common.collect.ImmutableList, org.elasticsearch.cluster.metadata.MetaData)} * <li>Master calls {@link #initializeSnapshot(org.elasticsearch.cluster.metadata.SnapshotId, List, org.elasticsearch.cluster.metadata.MetaData)}
* with list of indices that will be included into the snapshot</li> * with list of indices that will be included into the snapshot</li>
* <li>Data nodes call {@link org.elasticsearch.index.snapshots.IndexShardRepository#snapshot(org.elasticsearch.cluster.metadata.SnapshotId, org.elasticsearch.index.shard.ShardId, org.elasticsearch.index.deletionpolicy.SnapshotIndexCommit, org.elasticsearch.index.snapshots.IndexShardSnapshotStatus)} for each shard</li> * <li>Data nodes call {@link org.elasticsearch.index.snapshots.IndexShardRepository#snapshot(org.elasticsearch.cluster.metadata.SnapshotId, org.elasticsearch.index.shard.ShardId, org.elasticsearch.index.deletionpolicy.SnapshotIndexCommit, org.elasticsearch.index.snapshots.IndexShardSnapshotStatus)} for each shard</li>
* <li>When all shard calls return master calls {@link #finalizeSnapshot} * <li>When all shard calls return master calls {@link #finalizeSnapshot}
@ -62,14 +63,14 @@ public interface Repository extends LifecycleComponent<Repository> {
* @param indices list of indices * @param indices list of indices
* @return information about snapshot * @return information about snapshot
*/ */
MetaData readSnapshotMetaData(SnapshotId snapshotId, ImmutableList<String> indices) throws IOException; MetaData readSnapshotMetaData(SnapshotId snapshotId, List<String> indices) throws IOException;
/** /**
* Returns the list of snapshots currently stored in the repository * Returns the list of snapshots currently stored in the repository
* *
* @return snapshot list * @return snapshot list
*/ */
ImmutableList<SnapshotId> snapshots(); List<SnapshotId> snapshots();
/** /**
* Starts snapshotting process * Starts snapshotting process
@ -78,7 +79,7 @@ public interface Repository extends LifecycleComponent<Repository> {
* @param indices list of indices to be snapshotted * @param indices list of indices to be snapshotted
* @param metaData cluster metadata * @param metaData cluster metadata
*/ */
void initializeSnapshot(SnapshotId snapshotId, ImmutableList<String> indices, MetaData metaData); void initializeSnapshot(SnapshotId snapshotId, List<String> indices, MetaData metaData);
/** /**
* Finalizes snapshotting process * Finalizes snapshotting process
@ -91,7 +92,7 @@ public interface Repository extends LifecycleComponent<Repository> {
* @param shardFailures list of shard failures * @param shardFailures list of shard failures
* @return snapshot description * @return snapshot description
*/ */
Snapshot finalizeSnapshot(SnapshotId snapshotId, ImmutableList<String> indices, long startTime, String failure, int totalShards, ImmutableList<SnapshotShardFailure> shardFailures); Snapshot finalizeSnapshot(SnapshotId snapshotId, List<String> indices, long startTime, String failure, int totalShards, List<SnapshotShardFailure> shardFailures);
/** /**
* Deletes snapshot * Deletes snapshot

View File

@ -75,6 +75,7 @@ import java.io.InputStream;
import java.io.OutputStream; import java.io.OutputStream;
import java.nio.file.NoSuchFileException; import java.nio.file.NoSuchFileException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -235,7 +236,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent<Rep
* {@inheritDoc} * {@inheritDoc}
*/ */
@Override @Override
public void initializeSnapshot(SnapshotId snapshotId, ImmutableList<String> indices, MetaData metaData) { public void initializeSnapshot(SnapshotId snapshotId, List<String> indices, MetaData metaData) {
try { try {
String snapshotBlobName = snapshotBlobName(snapshotId); String snapshotBlobName = snapshotBlobName(snapshotId);
if (snapshotsBlobContainer.blobExists(snapshotBlobName)) { if (snapshotsBlobContainer.blobExists(snapshotBlobName)) {
@ -268,7 +269,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent<Rep
*/ */
@Override @Override
public void deleteSnapshot(SnapshotId snapshotId) { public void deleteSnapshot(SnapshotId snapshotId) {
ImmutableList<String> indices = ImmutableList.of(); List<String> indices = Collections.EMPTY_LIST;
try { try {
indices = readSnapshot(snapshotId).indices(); indices = readSnapshot(snapshotId).indices();
} catch (SnapshotMissingException ex) { } catch (SnapshotMissingException ex) {
@ -288,7 +289,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent<Rep
snapshotsBlobContainer.deleteBlob(blobName); snapshotsBlobContainer.deleteBlob(blobName);
snapshotsBlobContainer.deleteBlob(metaDataBlobName(snapshotId)); snapshotsBlobContainer.deleteBlob(metaDataBlobName(snapshotId));
// Delete snapshot from the snapshot list // Delete snapshot from the snapshot list
ImmutableList<SnapshotId> snapshotIds = snapshots(); List<SnapshotId> snapshotIds = snapshots();
if (snapshotIds.contains(snapshotId)) { if (snapshotIds.contains(snapshotId)) {
ImmutableList.Builder<SnapshotId> builder = ImmutableList.builder(); ImmutableList.Builder<SnapshotId> builder = ImmutableList.builder();
for (SnapshotId id : snapshotIds) { for (SnapshotId id : snapshotIds) {
@ -352,7 +353,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent<Rep
* {@inheritDoc} * {@inheritDoc}
*/ */
@Override @Override
public Snapshot finalizeSnapshot(SnapshotId snapshotId, ImmutableList<String> indices, long startTime, String failure, int totalShards, ImmutableList<SnapshotShardFailure> shardFailures) { public Snapshot finalizeSnapshot(SnapshotId snapshotId, List<String> indices, long startTime, String failure, int totalShards, List<SnapshotShardFailure> shardFailures) {
try { try {
String tempBlobName = tempSnapshotBlobName(snapshotId); String tempBlobName = tempSnapshotBlobName(snapshotId);
String blobName = snapshotBlobName(snapshotId); String blobName = snapshotBlobName(snapshotId);
@ -361,7 +362,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent<Rep
writeSnapshot(blobStoreSnapshot, output); writeSnapshot(blobStoreSnapshot, output);
} }
snapshotsBlobContainer.move(tempBlobName, blobName); snapshotsBlobContainer.move(tempBlobName, blobName);
ImmutableList<SnapshotId> snapshotIds = snapshots(); List<SnapshotId> snapshotIds = snapshots();
if (!snapshotIds.contains(snapshotId)) { if (!snapshotIds.contains(snapshotId)) {
snapshotIds = ImmutableList.<SnapshotId>builder().addAll(snapshotIds).add(snapshotId).build(); snapshotIds = ImmutableList.<SnapshotId>builder().addAll(snapshotIds).add(snapshotId).build();
} }
@ -376,10 +377,10 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent<Rep
* {@inheritDoc} * {@inheritDoc}
*/ */
@Override @Override
public ImmutableList<SnapshotId> snapshots() { public List<SnapshotId> snapshots() {
try { try {
List<SnapshotId> snapshots = newArrayList(); List<SnapshotId> snapshots = newArrayList();
ImmutableMap<String, BlobMetaData> blobs; Map<String, BlobMetaData> blobs;
try { try {
blobs = snapshotsBlobContainer.listBlobsByPrefix(SNAPSHOT_PREFIX); blobs = snapshotsBlobContainer.listBlobsByPrefix(SNAPSHOT_PREFIX);
} catch (UnsupportedOperationException ex) { } catch (UnsupportedOperationException ex) {
@ -401,7 +402,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent<Rep
* {@inheritDoc} * {@inheritDoc}
*/ */
@Override @Override
public MetaData readSnapshotMetaData(SnapshotId snapshotId, ImmutableList<String> indices) throws IOException { public MetaData readSnapshotMetaData(SnapshotId snapshotId, List<String> indices) throws IOException {
return readSnapshotMetaData(snapshotId, indices, false); return readSnapshotMetaData(snapshotId, indices, false);
} }
@ -421,7 +422,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent<Rep
} }
} }
private MetaData readSnapshotMetaData(SnapshotId snapshotId, ImmutableList<String> indices, boolean ignoreIndexErrors) throws IOException { private MetaData readSnapshotMetaData(SnapshotId snapshotId, List<String> indices, boolean ignoreIndexErrors) throws IOException {
MetaData metaData; MetaData metaData;
try (InputStream blob = snapshotsBlobContainer.openInput(metaDataBlobName(snapshotId))) { try (InputStream blob = snapshotsBlobContainer.openInput(metaDataBlobName(snapshotId))) {
byte[] data = ByteStreams.toByteArray(blob); byte[] data = ByteStreams.toByteArray(blob);
@ -597,7 +598,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent<Rep
* @param snapshots list of snapshot ids * @param snapshots list of snapshot ids
* @throws IOException I/O errors * @throws IOException I/O errors
*/ */
protected void writeSnapshotList(ImmutableList<SnapshotId> snapshots) throws IOException { protected void writeSnapshotList(List<SnapshotId> snapshots) throws IOException {
BytesStreamOutput bStream = new BytesStreamOutput(); BytesStreamOutput bStream = new BytesStreamOutput();
StreamOutput stream = compressIfNeeded(bStream); StreamOutput stream = compressIfNeeded(bStream);
XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON, stream); XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON, stream);
@ -623,7 +624,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent<Rep
* @return list of snapshots in the repository * @return list of snapshots in the repository
* @throws IOException I/O errors * @throws IOException I/O errors
*/ */
protected ImmutableList<SnapshotId> readSnapshotList() throws IOException { protected List<SnapshotId> readSnapshotList() throws IOException {
try (InputStream blob = snapshotsBlobContainer.openInput(SNAPSHOTS_FILE)) { try (InputStream blob = snapshotsBlobContainer.openInput(SNAPSHOTS_FILE)) {
final byte[] data = ByteStreams.toByteArray(blob); final byte[] data = ByteStreams.toByteArray(blob);
ArrayList<SnapshotId> snapshots = new ArrayList<>(); ArrayList<SnapshotId> snapshots = new ArrayList<>();

View File

@ -33,6 +33,7 @@ import org.elasticsearch.repositories.blobstore.BlobStoreRepository;
import java.io.IOException; import java.io.IOException;
import java.net.URL; import java.net.URL;
import java.util.List;
/** /**
* Read-only URL-based implementation of the BlobStoreRepository * Read-only URL-based implementation of the BlobStoreRepository
@ -90,7 +91,7 @@ public class URLRepository extends BlobStoreRepository {
} }
@Override @Override
public ImmutableList<SnapshotId> snapshots() { public List<SnapshotId> snapshots() {
if (listDirectories) { if (listDirectories) {
return super.snapshots(); return super.snapshots();
} else { } else {

View File

@ -28,6 +28,7 @@ import org.elasticsearch.common.xcontent.XContentBuilderString;
import org.elasticsearch.rest.RestStatus; import org.elasticsearch.rest.RestStatus;
import java.io.IOException; import java.io.IOException;
import java.util.List;
/** /**
* Information about successfully completed restore operation. * Information about successfully completed restore operation.
@ -69,7 +70,7 @@ public class RestoreInfo implements ToXContent, Streamable {
* *
* @return list of restored indices * @return list of restored indices
*/ */
public ImmutableList<String> indices() { public List<String> indices() {
return indices; return indices;
} }

View File

@ -156,7 +156,7 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
Repository repository = repositoriesService.repository(request.repository()); Repository repository = repositoriesService.repository(request.repository());
final SnapshotId snapshotId = new SnapshotId(request.repository(), request.name()); final SnapshotId snapshotId = new SnapshotId(request.repository(), request.name());
final Snapshot snapshot = repository.readSnapshot(snapshotId); final Snapshot snapshot = repository.readSnapshot(snapshotId);
ImmutableList<String> filteredIndices = SnapshotUtils.filterIndices(snapshot.indices(), request.indices(), request.indicesOptions()); List<String> filteredIndices = SnapshotUtils.filterIndices(snapshot.indices(), request.indices(), request.indicesOptions());
MetaData metaDataIn = repository.readSnapshotMetaData(snapshotId, filteredIndices); MetaData metaDataIn = repository.readSnapshotMetaData(snapshotId, filteredIndices);
final MetaData metaData; final MetaData metaData;
@ -658,7 +658,7 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
return failedShards; return failedShards;
} }
private Map<String, String> renamedIndices(RestoreRequest request, ImmutableList<String> filteredIndices) { private Map<String, String> renamedIndices(RestoreRequest request, List<String> filteredIndices) {
Map<String, String> renamedIndices = newHashMap(); Map<String, String> renamedIndices = newHashMap();
for (String index : filteredIndices) { for (String index : filteredIndices) {
String renamedIndex = index; String renamedIndex = index;

View File

@ -28,6 +28,7 @@ import org.elasticsearch.common.xcontent.XContentParser;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List;
/** /**
* Represent information about snapshot * Represent information about snapshot
@ -42,7 +43,7 @@ public class Snapshot implements Comparable<Snapshot>, ToXContent {
private final String reason; private final String reason;
private final ImmutableList<String> indices; private final List<String> indices;
private final long startTime; private final long startTime;
@ -52,12 +53,12 @@ public class Snapshot implements Comparable<Snapshot>, ToXContent {
private final int successfulShards; private final int successfulShards;
private final ImmutableList<SnapshotShardFailure> shardFailures; private final List<SnapshotShardFailure> shardFailures;
private final static ImmutableList<SnapshotShardFailure> NO_FAILURES = ImmutableList.of(); private final static List<SnapshotShardFailure> NO_FAILURES = ImmutableList.of();
private Snapshot(String name, ImmutableList<String> indices, SnapshotState state, String reason, Version version, long startTime, long endTime, private Snapshot(String name, List<String> indices, SnapshotState state, String reason, Version version, long startTime, long endTime,
int totalShard, int successfulShards, ImmutableList<SnapshotShardFailure> shardFailures) { int totalShard, int successfulShards, List<SnapshotShardFailure> shardFailures) {
assert name != null; assert name != null;
assert indices != null; assert indices != null;
assert state != null; assert state != null;
@ -75,17 +76,17 @@ public class Snapshot implements Comparable<Snapshot>, ToXContent {
} }
public Snapshot(String name, ImmutableList<String> indices, long startTime) { public Snapshot(String name, List<String> indices, long startTime) {
this(name, indices, SnapshotState.IN_PROGRESS, null, Version.CURRENT, startTime, 0L, 0, 0, NO_FAILURES); this(name, indices, SnapshotState.IN_PROGRESS, null, Version.CURRENT, startTime, 0L, 0, 0, NO_FAILURES);
} }
public Snapshot(String name, ImmutableList<String> indices, long startTime, String reason, long endTime, public Snapshot(String name, List<String> indices, long startTime, String reason, long endTime,
int totalShard, ImmutableList<SnapshotShardFailure> shardFailures) { int totalShard, List<SnapshotShardFailure> shardFailures) {
this(name, indices, snapshotState(reason, shardFailures), reason, Version.CURRENT, this(name, indices, snapshotState(reason, shardFailures), reason, Version.CURRENT,
startTime, endTime, totalShard, totalShard - shardFailures.size(), shardFailures); startTime, endTime, totalShard, totalShard - shardFailures.size(), shardFailures);
} }
private static SnapshotState snapshotState(String reason, ImmutableList<SnapshotShardFailure> shardFailures) { private static SnapshotState snapshotState(String reason, List<SnapshotShardFailure> shardFailures) {
if (reason == null) { if (reason == null) {
if (shardFailures.isEmpty()) { if (shardFailures.isEmpty()) {
return SnapshotState.SUCCESS; return SnapshotState.SUCCESS;
@ -138,7 +139,7 @@ public class Snapshot implements Comparable<Snapshot>, ToXContent {
* *
* @return list of indices * @return list of indices
*/ */
public ImmutableList<String> indices() { public List<String> indices() {
return indices; return indices;
} }
@ -183,7 +184,7 @@ public class Snapshot implements Comparable<Snapshot>, ToXContent {
/** /**
* Returns shard failures * Returns shard failures
*/ */
public ImmutableList<SnapshotShardFailure> shardFailures() { public List<SnapshotShardFailure> shardFailures() {
return shardFailures; return shardFailures;
} }
@ -275,7 +276,7 @@ public class Snapshot implements Comparable<Snapshot>, ToXContent {
long endTime = 0; long endTime = 0;
int totalShard = 0; int totalShard = 0;
int successfulShards = 0; int successfulShards = 0;
ImmutableList<SnapshotShardFailure> shardFailures = NO_FAILURES; List<SnapshotShardFailure> shardFailures = NO_FAILURES;
XContentParser.Token token = parser.currentToken(); XContentParser.Token token = parser.currentToken();
if (token == XContentParser.Token.START_OBJECT) { if (token == XContentParser.Token.START_OBJECT) {

View File

@ -19,6 +19,8 @@
package org.elasticsearch.snapshots; package org.elasticsearch.snapshots;
import java.io.IOException; import java.io.IOException;
import java.util.List;
import org.elasticsearch.action.ShardOperationFailedException; import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.StreamOutput;
@ -44,7 +46,7 @@ public class SnapshotInfo implements ToXContent, Streamable {
private String reason; private String reason;
private ImmutableList<String> indices; private List<String> indices;
private long startTime; private long startTime;
@ -54,7 +56,7 @@ public class SnapshotInfo implements ToXContent, Streamable {
private int successfulShards; private int successfulShards;
private ImmutableList<SnapshotShardFailure> shardFailures; private List<SnapshotShardFailure> shardFailures;
SnapshotInfo() { SnapshotInfo() {
@ -109,7 +111,7 @@ public class SnapshotInfo implements ToXContent, Streamable {
* *
* @return list of indices * @return list of indices
*/ */
public ImmutableList<String> indices() { public List<String> indices() {
return indices; return indices;
} }
@ -165,7 +167,7 @@ public class SnapshotInfo implements ToXContent, Streamable {
* *
* @return shard failures * @return shard failures
*/ */
public ImmutableList<SnapshotShardFailure> shardFailures() { public List<SnapshotShardFailure> shardFailures() {
return shardFailures; return shardFailures;
} }

View File

@ -25,6 +25,7 @@ import org.elasticsearch.index.Index;
import org.elasticsearch.indices.IndexMissingException; import org.elasticsearch.indices.IndexMissingException;
import java.util.HashSet; import java.util.HashSet;
import java.util.List;
import java.util.Set; import java.util.Set;
/** /**
@ -40,7 +41,7 @@ public class SnapshotUtils {
* @param indicesOptions ignore indices flag * @param indicesOptions ignore indices flag
* @return filtered out indices * @return filtered out indices
*/ */
public static ImmutableList<String> filterIndices(ImmutableList<String> availableIndices, String[] selectedIndices, IndicesOptions indicesOptions) { public static List<String> filterIndices(List<String> availableIndices, String[] selectedIndices, IndicesOptions indicesOptions) {
if (selectedIndices == null || selectedIndices.length == 0) { if (selectedIndices == null || selectedIndices.length == 0) {
return availableIndices; return availableIndices;
} }

View File

@ -80,7 +80,7 @@ import static com.google.common.collect.Sets.newHashSet;
* <li>When cluster state is updated the {@link #beginSnapshot(ClusterState, SnapshotMetaData.Entry, boolean, CreateSnapshotListener)} method * <li>When cluster state is updated the {@link #beginSnapshot(ClusterState, SnapshotMetaData.Entry, boolean, CreateSnapshotListener)} method
* kicks in and initializes the snapshot in the repository and then populates list of shards that needs to be snapshotted in cluster state</li> * kicks in and initializes the snapshot in the repository and then populates list of shards that needs to be snapshotted in cluster state</li>
* <li>Each data node is watching for these shards and when new shards scheduled for snapshotting appear in the cluster state, data nodes * <li>Each data node is watching for these shards and when new shards scheduled for snapshotting appear in the cluster state, data nodes
* start processing them through {@link #processIndexShardSnapshots(SnapshotMetaData)} method</li> * start processing them through {@link SnapshotsService#processIndexShardSnapshots(ClusterChangedEvent)} method</li>
* <li>Once shard snapshot is created data node updates state of the shard in the cluster state using the {@link #updateIndexShardSnapshotStatus(UpdateIndexShardSnapshotStatusRequest)} method</li> * <li>Once shard snapshot is created data node updates state of the shard in the cluster state using the {@link #updateIndexShardSnapshotStatus(UpdateIndexShardSnapshotStatusRequest)} method</li>
* <li>When last shard is completed master node in {@link #innerUpdateSnapshotState} method marks the snapshot as completed</li> * <li>When last shard is completed master node in {@link #innerUpdateSnapshotState} method marks the snapshot as completed</li>
* <li>After cluster state is updated, the {@link #endSnapshot(SnapshotMetaData.Entry)} finalizes snapshot in the repository, * <li>After cluster state is updated, the {@link #endSnapshot(SnapshotMetaData.Entry)} finalizes snapshot in the repository,
@ -135,7 +135,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
* @throws SnapshotMissingException if snapshot is not found * @throws SnapshotMissingException if snapshot is not found
*/ */
public Snapshot snapshot(SnapshotId snapshotId) { public Snapshot snapshot(SnapshotId snapshotId) {
ImmutableList<SnapshotMetaData.Entry> entries = currentSnapshots(snapshotId.getRepository(), new String[]{snapshotId.getSnapshot()}); List<SnapshotMetaData.Entry> entries = currentSnapshots(snapshotId.getRepository(), new String[]{snapshotId.getSnapshot()});
if (!entries.isEmpty()) { if (!entries.isEmpty()) {
return inProgressSnapshot(entries.iterator().next()); return inProgressSnapshot(entries.iterator().next());
} }
@ -148,14 +148,14 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
* @param repositoryName repository name * @param repositoryName repository name
* @return list of snapshots * @return list of snapshots
*/ */
public ImmutableList<Snapshot> snapshots(String repositoryName) { public List<Snapshot> snapshots(String repositoryName) {
Set<Snapshot> snapshotSet = newHashSet(); Set<Snapshot> snapshotSet = newHashSet();
ImmutableList<SnapshotMetaData.Entry> entries = currentSnapshots(repositoryName, null); List<SnapshotMetaData.Entry> entries = currentSnapshots(repositoryName, null);
for (SnapshotMetaData.Entry entry : entries) { for (SnapshotMetaData.Entry entry : entries) {
snapshotSet.add(inProgressSnapshot(entry)); snapshotSet.add(inProgressSnapshot(entry));
} }
Repository repository = repositoriesService.repository(repositoryName); Repository repository = repositoriesService.repository(repositoryName);
ImmutableList<SnapshotId> snapshotIds = repository.snapshots(); List<SnapshotId> snapshotIds = repository.snapshots();
for (SnapshotId snapshotId : snapshotIds) { for (SnapshotId snapshotId : snapshotIds) {
snapshotSet.add(repository.readSnapshot(snapshotId)); snapshotSet.add(repository.readSnapshot(snapshotId));
} }
@ -170,9 +170,9 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
* @param repositoryName repository name * @param repositoryName repository name
* @return list of snapshots * @return list of snapshots
*/ */
public ImmutableList<Snapshot> currentSnapshots(String repositoryName) { public List<Snapshot> currentSnapshots(String repositoryName) {
List<Snapshot> snapshotList = newArrayList(); List<Snapshot> snapshotList = newArrayList();
ImmutableList<SnapshotMetaData.Entry> entries = currentSnapshots(repositoryName, null); List<SnapshotMetaData.Entry> entries = currentSnapshots(repositoryName, null);
for (SnapshotMetaData.Entry entry : entries) { for (SnapshotMetaData.Entry entry : entries) {
snapshotList.add(inProgressSnapshot(entry)); snapshotList.add(inProgressSnapshot(entry));
} }
@ -421,7 +421,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
* @param snapshots optional list of snapshots that will be used as a filter * @param snapshots optional list of snapshots that will be used as a filter
* @return list of metadata for currently running snapshots * @return list of metadata for currently running snapshots
*/ */
public ImmutableList<SnapshotMetaData.Entry> currentSnapshots(String repository, String[] snapshots) { public List<SnapshotMetaData.Entry> currentSnapshots(String repository, String[] snapshots) {
MetaData metaData = clusterService.state().metaData(); MetaData metaData = clusterService.state().metaData();
SnapshotMetaData snapshotMetaData = metaData.custom(SnapshotMetaData.TYPE); SnapshotMetaData snapshotMetaData = metaData.custom(SnapshotMetaData.TYPE);
if (snapshotMetaData == null || snapshotMetaData.entries().isEmpty()) { if (snapshotMetaData == null || snapshotMetaData.entries().isEmpty()) {
@ -524,7 +524,7 @@ public class SnapshotsService extends AbstractLifecycleComponent<SnapshotsServic
} }
private SnapshotShardFailure findShardFailure(ImmutableList<SnapshotShardFailure> shardFailures, ShardId shardId) { private SnapshotShardFailure findShardFailure(List<SnapshotShardFailure> shardFailures, ShardId shardId) {
for (SnapshotShardFailure shardFailure : shardFailures) { for (SnapshotShardFailure shardFailure : shardFailures) {
if (shardId.getIndex().equals(shardFailure.index()) && shardId.getId() == shardFailure.shardId()) { if (shardId.getIndex().equals(shardFailure.index()) && shardId.getId() == shardFailure.shardId()) {
return shardFailure; return shardFailure;

View File

@ -90,7 +90,7 @@ public class BlobStoreTest extends ElasticsearchTestCase {
generatedBlobs.put(name, (long) length); generatedBlobs.put(name, (long) length);
byte[] data = createRandomBlob(container, name, length); byte[] data = createRandomBlob(container, name, length);
ImmutableMap<String, BlobMetaData> blobs = container.listBlobs(); Map<String, BlobMetaData> blobs = container.listBlobs();
assertThat(blobs.size(), equalTo(numberOfFooBlobs + numberOfBarBlobs)); assertThat(blobs.size(), equalTo(numberOfFooBlobs + numberOfBarBlobs));
for (Map.Entry<String, Long> generated : generatedBlobs.entrySet()) { for (Map.Entry<String, Long> generated : generatedBlobs.entrySet()) {
BlobMetaData blobMetaData = blobs.get(generated.getKey()); BlobMetaData blobMetaData = blobs.get(generated.getKey());

View File

@ -220,7 +220,7 @@ public class CopyToMapperTests extends ElasticsearchSingleNodeTest {
DocumentMapperParser parser = createIndex("test").mapperService().documentMapperParser(); DocumentMapperParser parser = createIndex("test").mapperService().documentMapperParser();
DocumentMapper docMapperBefore = parser.parse(mappingBefore); DocumentMapper docMapperBefore = parser.parse(mappingBefore);
ImmutableList<String> fields = docMapperBefore.mappers().getMapper("copy_test").copyTo().copyToFields(); List<String> fields = docMapperBefore.mappers().getMapper("copy_test").copyTo().copyToFields();
assertThat(fields.size(), equalTo(2)); assertThat(fields.size(), equalTo(2));
assertThat(fields.get(0), equalTo("foo")); assertThat(fields.get(0), equalTo("foo"));

View File

@ -41,6 +41,7 @@ import java.nio.file.Files;
import java.nio.file.Path; import java.nio.file.Path;
import java.nio.file.SimpleFileVisitor; import java.nio.file.SimpleFileVisitor;
import java.nio.file.attribute.BasicFileAttributes; import java.nio.file.attribute.BasicFileAttributes;
import java.util.List;
import java.util.concurrent.CountDownLatch; import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
@ -100,7 +101,7 @@ public abstract class AbstractSnapshotTests extends ElasticsearchIntegrationTest
long start = System.currentTimeMillis(); long start = System.currentTimeMillis();
SnapshotId snapshotId = new SnapshotId(repository, snapshot); SnapshotId snapshotId = new SnapshotId(repository, snapshot);
while (System.currentTimeMillis() - start < timeout.millis()) { while (System.currentTimeMillis() - start < timeout.millis()) {
ImmutableList<SnapshotInfo> snapshotInfos = client().admin().cluster().prepareGetSnapshots(repository).setSnapshots(snapshot).get().getSnapshots(); List<SnapshotInfo> snapshotInfos = client().admin().cluster().prepareGetSnapshots(repository).setSnapshots(snapshot).get().getSnapshots();
assertThat(snapshotInfos.size(), equalTo(1)); assertThat(snapshotInfos.size(), equalTo(1));
if (snapshotInfos.get(0).state().completed()) { if (snapshotInfos.get(0).state().completed()) {
// Make sure that snapshot clean up operations are finished // Make sure that snapshot clean up operations are finished

View File

@ -39,6 +39,7 @@ import org.elasticsearch.test.ElasticsearchIntegrationTest;
import org.junit.Test; import org.junit.Test;
import java.nio.file.Path; import java.nio.file.Path;
import java.util.List;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertThrows; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertThrows;
@ -116,7 +117,7 @@ public class RepositoriesTests extends AbstractSnapshotTests {
assertThat(repositoriesResponse.repositories().size(), equalTo(0)); assertThat(repositoriesResponse.repositories().size(), equalTo(0));
} }
private RepositoryMetaData findRepository(ImmutableList<RepositoryMetaData> repositories, String name) { private RepositoryMetaData findRepository(List<RepositoryMetaData> repositories, String name) {
for (RepositoryMetaData repository : repositories) { for (RepositoryMetaData repository : repositories) {
if (repository.name().equals(name)) { if (repository.name().equals(name)) {
return repository; return repository;

View File

@ -1135,7 +1135,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
logger.info("Number of failed shards [{}]", snapshotInfo.shardFailures().size()); logger.info("Number of failed shards [{}]", snapshotInfo.shardFailures().size());
logger.info("--> done"); logger.info("--> done");
ImmutableList<SnapshotInfo> snapshotInfos = client().admin().cluster().prepareGetSnapshots("test-repo").setSnapshots("test-snap").get().getSnapshots(); List<SnapshotInfo> snapshotInfos = client().admin().cluster().prepareGetSnapshots("test-repo").setSnapshots("test-snap").get().getSnapshots();
assertThat(snapshotInfos.size(), equalTo(1)); assertThat(snapshotInfos.size(), equalTo(1));
assertThat(snapshotInfos.get(0).state(), equalTo(SnapshotState.SUCCESS)); assertThat(snapshotInfos.get(0).state(), equalTo(SnapshotState.SUCCESS));
@ -1218,7 +1218,7 @@ public class SharedClusterSnapshotRestoreTests extends AbstractSnapshotTests {
logger.info("Number of failed shards [{}]", snapshotInfo.shardFailures().size()); logger.info("Number of failed shards [{}]", snapshotInfo.shardFailures().size());
logger.info("--> done"); logger.info("--> done");
ImmutableList<SnapshotInfo> snapshotInfos = client().admin().cluster().prepareGetSnapshots("test-repo").setSnapshots("test-snap").get().getSnapshots(); List<SnapshotInfo> snapshotInfos = client().admin().cluster().prepareGetSnapshots("test-repo").setSnapshots("test-snap").get().getSnapshots();
assertThat(snapshotInfos.size(), equalTo(1)); assertThat(snapshotInfos.size(), equalTo(1));
assertThat(snapshotInfos.get(0).state(), equalTo(SnapshotState.SUCCESS)); assertThat(snapshotInfos.get(0).state(), equalTo(SnapshotState.SUCCESS));

View File

@ -23,6 +23,8 @@ import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.test.ElasticsearchTestCase; import org.elasticsearch.test.ElasticsearchTestCase;
import org.junit.Test; import org.junit.Test;
import java.util.List;
import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.containsInAnyOrder;
@ -50,8 +52,8 @@ public class SnapshotUtilsTests extends ElasticsearchTestCase {
} }
private void assertIndexNameFiltering(String[] indices, String[] filter, IndicesOptions indicesOptions, String[] expected) { private void assertIndexNameFiltering(String[] indices, String[] filter, IndicesOptions indicesOptions, String[] expected) {
ImmutableList<String> indicesList = ImmutableList.copyOf(indices); List<String> indicesList = ImmutableList.copyOf(indices);
ImmutableList<String> actual = SnapshotUtils.filterIndices(indicesList, filter, indicesOptions); List<String> actual = SnapshotUtils.filterIndices(indicesList, filter, indicesOptions);
assertThat(actual, containsInAnyOrder(expected)); assertThat(actual, containsInAnyOrder(expected));
} }
} }

View File

@ -26,6 +26,7 @@ import org.elasticsearch.common.blobstore.BlobPath;
import java.io.IOException; import java.io.IOException;
import java.io.InputStream; import java.io.InputStream;
import java.io.OutputStream; import java.io.OutputStream;
import java.util.Map;
/** /**
* *
@ -68,12 +69,12 @@ public class BlobContainerWrapper implements BlobContainer {
} }
@Override @Override
public ImmutableMap<String, BlobMetaData> listBlobs() throws IOException { public Map<String, BlobMetaData> listBlobs() throws IOException {
return delegate.listBlobs(); return delegate.listBlobs();
} }
@Override @Override
public ImmutableMap<String, BlobMetaData> listBlobsByPrefix(String blobNamePrefix) throws IOException { public Map<String, BlobMetaData> listBlobsByPrefix(String blobNamePrefix) throws IOException {
return delegate.listBlobsByPrefix(blobNamePrefix); return delegate.listBlobsByPrefix(blobNamePrefix);
} }

View File

@ -46,6 +46,8 @@ import java.io.UnsupportedEncodingException;
import java.nio.file.Path; import java.nio.file.Path;
import java.security.MessageDigest; import java.security.MessageDigest;
import java.security.NoSuchAlgorithmException; import java.security.NoSuchAlgorithmException;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicLong;
@ -95,7 +97,7 @@ public class MockRepository extends FsRepository {
} }
@Override @Override
public void initializeSnapshot(SnapshotId snapshotId, ImmutableList<String> indices, MetaData metaData) { public void initializeSnapshot(SnapshotId snapshotId, List<String> indices, MetaData metaData) {
if (blockOnInitialization ) { if (blockOnInitialization ) {
blockExecution(); blockExecution();
} }
@ -292,13 +294,13 @@ public class MockRepository extends FsRepository {
} }
@Override @Override
public ImmutableMap<String, BlobMetaData> listBlobs() throws IOException { public Map<String, BlobMetaData> listBlobs() throws IOException {
maybeIOExceptionOrBlock(""); maybeIOExceptionOrBlock("");
return super.listBlobs(); return super.listBlobs();
} }
@Override @Override
public ImmutableMap<String, BlobMetaData> listBlobsByPrefix(String blobNamePrefix) throws IOException { public Map<String, BlobMetaData> listBlobsByPrefix(String blobNamePrefix) throws IOException {
maybeIOExceptionOrBlock(blobNamePrefix); maybeIOExceptionOrBlock(blobNamePrefix);
return super.listBlobsByPrefix(blobNamePrefix); return super.listBlobsByPrefix(blobNamePrefix);
} }