More progress
This commit is contained in:
parent
a378cc6866
commit
46d10f1b6f
|
@ -19,8 +19,11 @@
|
|||
|
||||
package org.elasticsearch.cluster;
|
||||
|
||||
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
|
||||
|
||||
import org.elasticsearch.cluster.ClusterState.Custom;
|
||||
import org.elasticsearch.cluster.metadata.SnapshotId;
|
||||
import org.elasticsearch.common.collect.ImmutableOpenMap;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
|
@ -31,12 +34,7 @@ import java.io.IOException;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import static java.util.Collections.emptyMap;
|
||||
import static java.util.Collections.unmodifiableMap;
|
||||
|
||||
/**
|
||||
* Meta data about restore processes that are currently executing
|
||||
|
@ -115,7 +113,7 @@ public class RestoreInProgress extends AbstractDiffable<Custom> implements Custo
|
|||
public static class Entry {
|
||||
private final State state;
|
||||
private final SnapshotId snapshotId;
|
||||
private final Map<ShardId, ShardRestoreStatus> shards;
|
||||
private final ImmutableOpenMap<ShardId, ShardRestoreStatus> shards;
|
||||
private final List<String> indices;
|
||||
|
||||
/**
|
||||
|
@ -124,14 +122,14 @@ public class RestoreInProgress extends AbstractDiffable<Custom> implements Custo
|
|||
* @param snapshotId snapshot id
|
||||
* @param state current state of the restore process
|
||||
* @param indices list of indices being restored
|
||||
* @param shards list of shards being restored and thier current restore status
|
||||
* @param shards map of shards being restored to their current restore status
|
||||
*/
|
||||
public Entry(SnapshotId snapshotId, State state, List<String> indices, Map<ShardId, ShardRestoreStatus> shards) {
|
||||
public Entry(SnapshotId snapshotId, State state, List<String> indices, ImmutableOpenMap<ShardId, ShardRestoreStatus> shards) {
|
||||
this.snapshotId = snapshotId;
|
||||
this.state = state;
|
||||
this.indices = indices;
|
||||
if (shards == null) {
|
||||
this.shards = emptyMap();
|
||||
this.shards = ImmutableOpenMap.of();
|
||||
} else {
|
||||
this.shards = shards;
|
||||
}
|
||||
|
@ -151,7 +149,7 @@ public class RestoreInProgress extends AbstractDiffable<Custom> implements Custo
|
|||
*
|
||||
* @return list of shards
|
||||
*/
|
||||
public Map<ShardId, ShardRestoreStatus> shards() {
|
||||
public ImmutableOpenMap<ShardId, ShardRestoreStatus> shards() {
|
||||
return this.shards;
|
||||
}
|
||||
|
||||
|
@ -419,14 +417,14 @@ public class RestoreInProgress extends AbstractDiffable<Custom> implements Custo
|
|||
for (int j = 0; j < indices; j++) {
|
||||
indexBuilder.add(in.readString());
|
||||
}
|
||||
Map<ShardId, ShardRestoreStatus> builder = new HashMap<>();
|
||||
ImmutableOpenMap.Builder<ShardId, ShardRestoreStatus> builder = ImmutableOpenMap.builder();
|
||||
int shards = in.readVInt();
|
||||
for (int j = 0; j < shards; j++) {
|
||||
ShardId shardId = ShardId.readShardId(in);
|
||||
ShardRestoreStatus shardState = ShardRestoreStatus.readShardRestoreStatus(in);
|
||||
builder.put(shardId, shardState);
|
||||
}
|
||||
entries[i] = new Entry(snapshotId, state, Collections.unmodifiableList(indexBuilder), unmodifiableMap(builder));
|
||||
entries[i] = new Entry(snapshotId, state, Collections.unmodifiableList(indexBuilder), builder.build());
|
||||
}
|
||||
return new RestoreInProgress(entries);
|
||||
}
|
||||
|
@ -445,9 +443,9 @@ public class RestoreInProgress extends AbstractDiffable<Custom> implements Custo
|
|||
out.writeString(index);
|
||||
}
|
||||
out.writeVInt(entry.shards().size());
|
||||
for (Map.Entry<ShardId, ShardRestoreStatus> shardEntry : entry.shards().entrySet()) {
|
||||
shardEntry.getKey().writeTo(out);
|
||||
shardEntry.getValue().writeTo(out);
|
||||
for (ObjectObjectCursor<ShardId, ShardRestoreStatus> shardEntry : entry.shards()) {
|
||||
shardEntry.key.writeTo(out);
|
||||
shardEntry.value.writeTo(out);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -486,9 +484,9 @@ public class RestoreInProgress extends AbstractDiffable<Custom> implements Custo
|
|||
builder.endArray();
|
||||
builder.startArray("shards");
|
||||
{
|
||||
for (Map.Entry<ShardId, ShardRestoreStatus> shardEntry : entry.shards.entrySet()) {
|
||||
ShardId shardId = shardEntry.getKey();
|
||||
ShardRestoreStatus status = shardEntry.getValue();
|
||||
for (ObjectObjectCursor<ShardId, ShardRestoreStatus> shardEntry : entry.shards) {
|
||||
ShardId shardId = shardEntry.key;
|
||||
ShardRestoreStatus status = shardEntry.value;
|
||||
builder.startObject();
|
||||
{
|
||||
builder.field("index", shardId.getIndex());
|
||||
|
|
|
@ -19,11 +19,11 @@
|
|||
|
||||
package org.elasticsearch.common.collect;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import static java.util.Collections.unmodifiableMap;
|
||||
|
||||
/**
|
||||
*
|
||||
*/
|
||||
|
@ -83,7 +83,8 @@ public class MapBuilder<K, V> {
|
|||
return this.map;
|
||||
}
|
||||
|
||||
public Map<K, V> immutableMap() {
|
||||
return unmodifiableMap(new HashMap<>(map));
|
||||
public ImmutableMap<K, V> immutableMap() {
|
||||
// Note that this whole method is going to have to go next but we're changing it like this here just to keep the commit smaller.
|
||||
return ImmutableMap.<K, V>builder().putAll(map).build();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -19,8 +19,6 @@
|
|||
|
||||
package org.elasticsearch.index;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
|
||||
import org.apache.lucene.util.Accountable;
|
||||
import org.apache.lucene.util.IOUtils;
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
|
@ -75,6 +73,7 @@ import java.util.concurrent.TimeUnit;
|
|||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import static java.util.Collections.emptyMap;
|
||||
import static java.util.Collections.unmodifiableMap;
|
||||
import static org.elasticsearch.common.collect.MapBuilder.newMapBuilder;
|
||||
|
||||
/**
|
||||
|
@ -410,11 +409,11 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
|
|||
return;
|
||||
}
|
||||
logger.debug("[{}] closing... (reason: [{}])", shardId, reason);
|
||||
HashMap<Integer, IndexShardInjectorPair> tmpShardsMap = new HashMap<>(shards);
|
||||
IndexShardInjectorPair indexShardInjectorPair = tmpShardsMap.remove(shardId);
|
||||
HashMap<Integer, IndexShardInjectorPair> newShards = new HashMap<>(shards);
|
||||
IndexShardInjectorPair indexShardInjectorPair = newShards.remove(shardId);
|
||||
indexShard = indexShardInjectorPair.getIndexShard();
|
||||
shardInjector = indexShardInjectorPair.getInjector();
|
||||
shards = ImmutableMap.copyOf(tmpShardsMap);
|
||||
shards = unmodifiableMap(newShards);
|
||||
closeShardInjector(reason, sId, shardInjector, indexShard);
|
||||
logger.debug("[{}] closed (reason: [{}])", shardId, reason);
|
||||
}
|
||||
|
|
|
@ -18,7 +18,6 @@
|
|||
*/
|
||||
package org.elasticsearch.indices.flush;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.common.util.iterable.Iterables;
|
||||
import org.elasticsearch.common.xcontent.ToXContent;
|
||||
|
@ -30,6 +29,8 @@ import java.io.IOException;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import static java.util.Collections.unmodifiableMap;
|
||||
|
||||
/**
|
||||
* The result of performing a sync flush operation on all shards of multiple indices
|
||||
*/
|
||||
|
@ -40,7 +41,10 @@ public class IndicesSyncedFlushResult implements ToXContent {
|
|||
|
||||
|
||||
public IndicesSyncedFlushResult(Map<String, List<ShardsSyncedFlushResult>> shardsResultPerIndex) {
|
||||
this.shardsResultPerIndex = ImmutableMap.copyOf(shardsResultPerIndex);
|
||||
// shardsResultPerIndex is never modified after it is passed to this
|
||||
// constructor so this is safe even though shardsResultPerIndex is a
|
||||
// ConcurrentHashMap
|
||||
this.shardsResultPerIndex = unmodifiableMap(shardsResultPerIndex);
|
||||
this.shardCounts = calculateShardCounts(Iterables.flatten(shardsResultPerIndex.values()));
|
||||
}
|
||||
|
||||
|
|
|
@ -37,7 +37,7 @@ public class IndicesQueriesRegistry extends AbstractComponent {
|
|||
private Map<String, QueryParser<?>> queryParsers;
|
||||
|
||||
@Inject
|
||||
public IndicesQueriesRegistry(Settings settings, Set<QueryParser<?>> injectedQueryParsers, NamedWriteableRegistry namedWriteableRegistry) {
|
||||
public IndicesQueriesRegistry(Settings settings, Set<QueryParser> injectedQueryParsers, NamedWriteableRegistry namedWriteableRegistry) {
|
||||
super(settings);
|
||||
Map<String, QueryParser<?>> queryParsers = new HashMap<>();
|
||||
for (QueryParser<?> queryParser : injectedQueryParsers) {
|
||||
|
|
|
@ -54,6 +54,7 @@ import org.elasticsearch.cluster.settings.ClusterDynamicSettings;
|
|||
import org.elasticsearch.cluster.settings.DynamicSettings;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.collect.ImmutableOpenMap;
|
||||
import org.elasticsearch.common.collect.Tuple;
|
||||
import org.elasticsearch.common.component.AbstractComponent;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
|
@ -89,8 +90,6 @@ import java.util.Set;
|
|||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.CopyOnWriteArrayList;
|
||||
|
||||
import static java.util.Collections.emptyMap;
|
||||
import static java.util.Collections.unmodifiableMap;
|
||||
import static java.util.Collections.unmodifiableSet;
|
||||
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_AUTO_EXPAND_REPLICAS;
|
||||
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_CREATION_DATE;
|
||||
|
@ -233,11 +232,11 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
|
|||
MetaData.Builder mdBuilder = MetaData.builder(currentState.metaData());
|
||||
ClusterBlocks.Builder blocks = ClusterBlocks.builder().blocks(currentState.blocks());
|
||||
RoutingTable.Builder rtBuilder = RoutingTable.builder(currentState.routingTable());
|
||||
Map<ShardId, RestoreInProgress.ShardRestoreStatus> shards;
|
||||
ImmutableOpenMap<ShardId, RestoreInProgress.ShardRestoreStatus> shards;
|
||||
Set<String> aliases = new HashSet<>();
|
||||
if (!renamedIndices.isEmpty()) {
|
||||
// We have some indices to restore
|
||||
Map<ShardId, RestoreInProgress.ShardRestoreStatus> shardsBuilder = new HashMap<>();
|
||||
ImmutableOpenMap.Builder<ShardId, RestoreInProgress.ShardRestoreStatus> shardsBuilder = ImmutableOpenMap.builder();
|
||||
for (Map.Entry<String, String> indexEntry : renamedIndices.entrySet()) {
|
||||
String index = indexEntry.getValue();
|
||||
boolean partial = checkPartial(index);
|
||||
|
@ -308,11 +307,11 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
|
|||
}
|
||||
}
|
||||
|
||||
shards = unmodifiableMap(shardsBuilder);
|
||||
shards = shardsBuilder.build();
|
||||
RestoreInProgress.Entry restoreEntry = new RestoreInProgress.Entry(snapshotId, RestoreInProgress.State.INIT, Collections.unmodifiableList(new ArrayList<>(renamedIndices.keySet())), shards);
|
||||
builder.putCustom(RestoreInProgress.TYPE, new RestoreInProgress(restoreEntry));
|
||||
} else {
|
||||
shards = emptyMap();
|
||||
shards = ImmutableOpenMap.of();
|
||||
}
|
||||
|
||||
checkAliasNameConflicts(renamedIndices, aliases);
|
||||
|
@ -533,7 +532,7 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
|
|||
|
||||
clusterService.submitStateUpdateTask("update snapshot state", new ClusterStateUpdateTask() {
|
||||
private final List<UpdateIndexShardRestoreStatusRequest> drainedRequests = new ArrayList<>();
|
||||
private Map<SnapshotId, Tuple<RestoreInfo, Map<ShardId, ShardRestoreStatus>>> batchedRestoreInfo = null;
|
||||
private Map<SnapshotId, Tuple<RestoreInfo, ImmutableOpenMap<ShardId, ShardRestoreStatus>>> batchedRestoreInfo = null;
|
||||
|
||||
@Override
|
||||
public ClusterState execute(ClusterState currentState) {
|
||||
|
@ -556,7 +555,7 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
|
|||
int changedCount = 0;
|
||||
final List<RestoreInProgress.Entry> entries = new ArrayList<>();
|
||||
for (RestoreInProgress.Entry entry : restore.entries()) {
|
||||
Map<ShardId, ShardRestoreStatus> shards = null;
|
||||
ImmutableOpenMap.Builder<ShardId, ShardRestoreStatus> shardsBuilder = null;
|
||||
|
||||
for (int i = 0; i < batchSize; i++) {
|
||||
final UpdateIndexShardRestoreStatusRequest updateSnapshotState = drainedRequests.get(i);
|
||||
|
@ -564,17 +563,18 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
|
|||
|
||||
if (entry.snapshotId().equals(updateSnapshotState.snapshotId())) {
|
||||
logger.trace("[{}] Updating shard [{}] with status [{}]", updateSnapshotState.snapshotId(), updateSnapshotState.shardId(), updateSnapshotState.status().state());
|
||||
if (shards == null) {
|
||||
shards = new HashMap<>(entry.shards());
|
||||
if (shardsBuilder == null) {
|
||||
shardsBuilder = ImmutableOpenMap.builder(entry.shards());
|
||||
}
|
||||
shards.put(updateSnapshotState.shardId(), updateSnapshotState.status());
|
||||
shardsBuilder.put(updateSnapshotState.shardId(), updateSnapshotState.status());
|
||||
changedCount++;
|
||||
}
|
||||
}
|
||||
|
||||
if (shards != null) {
|
||||
if (shardsBuilder != null) {
|
||||
ImmutableOpenMap<ShardId, ShardRestoreStatus> shards = shardsBuilder.build();
|
||||
if (!completed(shards)) {
|
||||
entries.add(new RestoreInProgress.Entry(entry.snapshotId(), RestoreInProgress.State.STARTED, entry.indices(), unmodifiableMap(shards)));
|
||||
entries.add(new RestoreInProgress.Entry(entry.snapshotId(), RestoreInProgress.State.STARTED, entry.indices(), shards));
|
||||
} else {
|
||||
logger.info("restore [{}] is done", entry.snapshotId());
|
||||
if (batchedRestoreInfo == null) {
|
||||
|
@ -611,15 +611,15 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
|
|||
@Override
|
||||
public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) {
|
||||
if (batchedRestoreInfo != null) {
|
||||
for (final Entry<SnapshotId, Tuple<RestoreInfo, Map<ShardId, ShardRestoreStatus>>> entry : batchedRestoreInfo.entrySet()) {
|
||||
for (final Entry<SnapshotId, Tuple<RestoreInfo, ImmutableOpenMap<ShardId, ShardRestoreStatus>>> entry : batchedRestoreInfo.entrySet()) {
|
||||
final SnapshotId snapshotId = entry.getKey();
|
||||
final RestoreInfo restoreInfo = entry.getValue().v1();
|
||||
final Map<ShardId, ShardRestoreStatus> shards = entry.getValue().v2();
|
||||
final ImmutableOpenMap<ShardId, ShardRestoreStatus> shards = entry.getValue().v2();
|
||||
RoutingTable routingTable = newState.getRoutingTable();
|
||||
final List<ShardId> waitForStarted = new ArrayList<>();
|
||||
for (Map.Entry<ShardId, ShardRestoreStatus> shard : shards.entrySet()) {
|
||||
if (shard.getValue().state() == RestoreInProgress.State.SUCCESS ) {
|
||||
ShardId shardId = shard.getKey();
|
||||
for (ObjectObjectCursor<ShardId, ShardRestoreStatus> shard : shards) {
|
||||
if (shard.value.state() == RestoreInProgress.State.SUCCESS ) {
|
||||
ShardId shardId = shard.key;
|
||||
ShardRouting shardRouting = findPrimaryShard(routingTable, shardId);
|
||||
if (shardRouting != null && !shardRouting.active()) {
|
||||
logger.trace("[{}][{}] waiting for the shard to start", snapshotId, shardId);
|
||||
|
@ -679,19 +679,19 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
|
|||
});
|
||||
}
|
||||
|
||||
private boolean completed(Map<ShardId, RestoreInProgress.ShardRestoreStatus> shards) {
|
||||
for (RestoreInProgress.ShardRestoreStatus status : shards.values()) {
|
||||
if (!status.state().completed()) {
|
||||
private boolean completed(ImmutableOpenMap<ShardId, RestoreInProgress.ShardRestoreStatus> shards) {
|
||||
for (ObjectCursor<RestoreInProgress.ShardRestoreStatus> status : shards.values()) {
|
||||
if (!status.value.state().completed()) {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
private int failedShards(Map<ShardId, RestoreInProgress.ShardRestoreStatus> shards) {
|
||||
private int failedShards(ImmutableOpenMap<ShardId, RestoreInProgress.ShardRestoreStatus> shards) {
|
||||
int failedShards = 0;
|
||||
for (RestoreInProgress.ShardRestoreStatus status : shards.values()) {
|
||||
if (status.state() == RestoreInProgress.State.FAILURE) {
|
||||
for (ObjectCursor<RestoreInProgress.ShardRestoreStatus> status : shards.values()) {
|
||||
if (status.value.state() == RestoreInProgress.State.FAILURE) {
|
||||
failedShards++;
|
||||
}
|
||||
}
|
||||
|
@ -746,13 +746,13 @@ public class RestoreService extends AbstractComponent implements ClusterStateLis
|
|||
// Some indices were deleted, let's make sure all indices that we are restoring still exist
|
||||
for (RestoreInProgress.Entry entry : restore.entries()) {
|
||||
List<ShardId> shardsToFail = null;
|
||||
for (Map.Entry<ShardId, ShardRestoreStatus> shard : entry.shards().entrySet()) {
|
||||
if (!shard.getValue().state().completed()) {
|
||||
if (!event.state().metaData().hasIndex(shard.getKey().getIndex())) {
|
||||
for (ObjectObjectCursor<ShardId, ShardRestoreStatus> shard : entry.shards()) {
|
||||
if (!shard.value.state().completed()) {
|
||||
if (!event.state().metaData().hasIndex(shard.key.getIndex())) {
|
||||
if (shardsToFail == null) {
|
||||
shardsToFail = new ArrayList<>();
|
||||
}
|
||||
shardsToFail.add(shard.getKey());
|
||||
shardsToFail.add(shard.key);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -58,7 +58,6 @@ import java.util.Collections;
|
|||
import java.util.List;
|
||||
|
||||
import static java.util.Collections.emptyList;
|
||||
import static java.util.Collections.emptyMap;
|
||||
import static org.elasticsearch.cluster.metadata.AliasMetaData.newAliasMetaDataBuilder;
|
||||
import static org.elasticsearch.cluster.routing.RandomShardRoutingMutator.randomChange;
|
||||
import static org.elasticsearch.cluster.routing.RandomShardRoutingMutator.randomReason;
|
||||
|
@ -680,7 +679,7 @@ public class ClusterStateDiffIT extends ESIntegTestCase {
|
|||
new SnapshotId(randomName("repo"), randomName("snap")),
|
||||
RestoreInProgress.State.fromValue((byte) randomIntBetween(0, 3)),
|
||||
emptyList(),
|
||||
emptyMap()));
|
||||
ImmutableOpenMap.of()));
|
||||
default:
|
||||
throw new IllegalArgumentException("Shouldn't be here");
|
||||
}
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.elasticsearch.cluster.serialization;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
|
||||
import org.elasticsearch.cluster.AbstractDiffable;
|
||||
import org.elasticsearch.cluster.Diff;
|
||||
import org.elasticsearch.cluster.DiffableUtils;
|
||||
|
@ -36,6 +37,7 @@ import java.io.IOException;
|
|||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import static java.util.Collections.unmodifiableMap;
|
||||
import static org.hamcrest.CoreMatchers.equalTo;
|
||||
|
||||
public class DiffableTests extends ESTestCase {
|
||||
|
@ -52,7 +54,7 @@ public class DiffableTests extends ESTestCase {
|
|||
map.remove("bar");
|
||||
map.put("baz", new TestDiffable("4"));
|
||||
map.put("new", new TestDiffable("5"));
|
||||
ImmutableMap<String, TestDiffable> after = ImmutableMap.copyOf(map);
|
||||
Map<String, TestDiffable> after = unmodifiableMap(new HashMap<>(map));
|
||||
Diff diff = DiffableUtils.diff(before, after);
|
||||
BytesStreamOutput out = new BytesStreamOutput();
|
||||
diff.writeTo(out);
|
||||
|
|
|
@ -18,9 +18,13 @@
|
|||
*/
|
||||
package org.elasticsearch.test.rest.section;
|
||||
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import java.util.*;
|
||||
import static java.util.Collections.unmodifiableMap;
|
||||
|
||||
/**
|
||||
* Represents a test fragment that contains the information needed to call an api
|
||||
|
@ -41,7 +45,7 @@ public class ApiCallSection {
|
|||
|
||||
public Map<String, String> getParams() {
|
||||
//make sure we never modify the parameters once returned
|
||||
return ImmutableMap.copyOf(params);
|
||||
return unmodifiableMap(params);
|
||||
}
|
||||
|
||||
public void addParam(String key, String value) {
|
||||
|
|
Loading…
Reference in New Issue