Remove some usages of MapBuilder#immutableMap (#41593)
This method can be replaced by some convenience collection factory methods available in the JDK now. This commit does not attempt to replace all usages, only enough that the size of the commit is kept to a reasonable size.
This commit is contained in:
parent
b4c6643e71
commit
545e56d1d1
|
@ -129,26 +129,23 @@ public class GCEFixture extends AbstractHttpFixture {
|
|||
// https://cloud.google.com/compute/docs/reference/rest/v1/instances
|
||||
handlers.insert(authPath(HttpGet.METHOD_NAME, "/compute/v1/projects/{project}/zones/{zone}/instances"),
|
||||
request -> {
|
||||
final List items = new ArrayList();
|
||||
final var items = new ArrayList<Map<String, Object>>();
|
||||
int count = 0;
|
||||
for (String address : Files.readAllLines(nodes)) {
|
||||
count++;
|
||||
items.add(MapBuilder.<String, Object>newMapBuilder()
|
||||
.put("id", Long.toString(9309873766405L + count))
|
||||
.put("description", "ES node" + count)
|
||||
.put("name", "test" + count)
|
||||
.put("kind", "compute#instance")
|
||||
.put("machineType", "n1-standard-1")
|
||||
.put("networkInterfaces",
|
||||
Collections.singletonList(MapBuilder.<String, Object>newMapBuilder()
|
||||
.put("accessConfigs", Collections.emptyList())
|
||||
.put("name", "nic0")
|
||||
.put("network", "default")
|
||||
.put("networkIP", address)
|
||||
.immutableMap()))
|
||||
.put("status", "RUNNING")
|
||||
.put("zone", ZONE)
|
||||
.immutableMap());
|
||||
items.add(Map.of(
|
||||
"id", Long.toString(9309873766405L + count),
|
||||
"description", "ES node" + count,
|
||||
"name", "test" + count,
|
||||
"kind", "compute#instance",
|
||||
"machineType", "n1-standard-1",
|
||||
"networkInterfaces", List.of(Map.of(
|
||||
"accessConfigs", Collections.emptyList(),
|
||||
"name", "nic0",
|
||||
"network", "default",
|
||||
"networkIP", address)),
|
||||
"status", "RUNNING",
|
||||
"zone", ZONE));
|
||||
}
|
||||
|
||||
final String json = Strings.toString(jsonBuilder()
|
||||
|
|
|
@ -34,13 +34,11 @@ import com.microsoft.azure.storage.blob.CloudBlobContainer;
|
|||
import com.microsoft.azure.storage.blob.CloudBlockBlob;
|
||||
import com.microsoft.azure.storage.blob.DeleteSnapshotsOption;
|
||||
import com.microsoft.azure.storage.blob.ListBlobItem;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.apache.logging.log4j.message.ParameterizedMessage;
|
||||
import org.elasticsearch.common.blobstore.BlobMetaData;
|
||||
import org.elasticsearch.common.blobstore.support.PlainBlobMetaData;
|
||||
import org.elasticsearch.common.collect.MapBuilder;
|
||||
import org.elasticsearch.common.collect.Tuple;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.settings.SettingsException;
|
||||
|
@ -55,6 +53,7 @@ import java.net.URISyntaxException;
|
|||
import java.nio.file.FileAlreadyExistsException;
|
||||
import java.security.InvalidKeyException;
|
||||
import java.util.EnumSet;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.function.Supplier;
|
||||
|
||||
|
@ -136,7 +135,7 @@ public class AzureStorageService {
|
|||
*/
|
||||
public Map<String, AzureStorageSettings> refreshAndClearCache(Map<String, AzureStorageSettings> clientsSettings) {
|
||||
final Map<String, AzureStorageSettings> prevSettings = this.storageSettings;
|
||||
this.storageSettings = MapBuilder.newMapBuilder(clientsSettings).immutableMap();
|
||||
this.storageSettings = Map.copyOf(clientsSettings);
|
||||
// clients are built lazily by {@link client(String)}
|
||||
return prevSettings;
|
||||
}
|
||||
|
@ -220,7 +219,7 @@ public class AzureStorageService {
|
|||
// NOTE: this should be here: if (prefix == null) prefix = "";
|
||||
// however, this is really inefficient since deleteBlobsByPrefix enumerates everything and
|
||||
// then does a prefix match on the result; it should just call listBlobsByPrefix with the prefix!
|
||||
final MapBuilder<String, BlobMetaData> blobsBuilder = MapBuilder.newMapBuilder();
|
||||
final var blobsBuilder = new HashMap<String, BlobMetaData>();
|
||||
final EnumSet<BlobListingDetails> enumBlobListingDetails = EnumSet.of(BlobListingDetails.METADATA);
|
||||
final Tuple<CloudBlobClient, Supplier<OperationContext>> client = client(account);
|
||||
final CloudBlobContainer blobContainer = client.v1().getContainerReference(container);
|
||||
|
@ -239,7 +238,7 @@ public class AzureStorageService {
|
|||
blobsBuilder.put(name, new PlainBlobMetaData(name, properties.getLength()));
|
||||
}
|
||||
});
|
||||
return blobsBuilder.immutableMap();
|
||||
return Map.copyOf(blobsBuilder);
|
||||
}
|
||||
|
||||
public void writeBlob(String account, String container, String blobName, InputStream inputStream, long blobSize,
|
||||
|
|
|
@ -22,7 +22,6 @@ package org.elasticsearch.repositories.azure;
|
|||
import com.microsoft.azure.storage.LocationMode;
|
||||
import com.microsoft.azure.storage.RetryPolicy;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.collect.MapBuilder;
|
||||
import org.elasticsearch.common.settings.SecureSetting;
|
||||
import org.elasticsearch.common.settings.SecureString;
|
||||
import org.elasticsearch.common.settings.Setting;
|
||||
|
@ -31,6 +30,7 @@ import org.elasticsearch.common.settings.Setting.Property;
|
|||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.settings.SettingsException;
|
||||
import org.elasticsearch.common.unit.TimeValue;
|
||||
|
||||
import java.net.InetAddress;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.Proxy;
|
||||
|
@ -234,13 +234,13 @@ final class AzureStorageSettings {
|
|||
|
||||
static Map<String, AzureStorageSettings> overrideLocationMode(Map<String, AzureStorageSettings> clientsSettings,
|
||||
LocationMode locationMode) {
|
||||
final MapBuilder<String, AzureStorageSettings> mapBuilder = new MapBuilder<>();
|
||||
final var map = new HashMap<String, AzureStorageSettings>();
|
||||
for (final Map.Entry<String, AzureStorageSettings> entry : clientsSettings.entrySet()) {
|
||||
final AzureStorageSettings azureSettings = new AzureStorageSettings(entry.getValue().account, entry.getValue().key,
|
||||
entry.getValue().endpointSuffix, entry.getValue().timeout, entry.getValue().maxRetries, entry.getValue().proxy,
|
||||
locationMode);
|
||||
mapBuilder.put(entry.getKey(), azureSettings);
|
||||
map.put(entry.getKey(), azureSettings);
|
||||
}
|
||||
return mapBuilder.immutableMap();
|
||||
return Map.copyOf(map);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,7 +24,6 @@ import com.microsoft.azure.storage.StorageException;
|
|||
import com.microsoft.azure.storage.blob.CloudBlobClient;
|
||||
import org.elasticsearch.common.blobstore.BlobMetaData;
|
||||
import org.elasticsearch.common.blobstore.support.PlainBlobMetaData;
|
||||
import org.elasticsearch.common.collect.MapBuilder;
|
||||
import org.elasticsearch.common.collect.Tuple;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.core.internal.io.Streams;
|
||||
|
@ -38,6 +37,7 @@ import java.net.URISyntaxException;
|
|||
import java.nio.file.FileAlreadyExistsException;
|
||||
import java.nio.file.NoSuchFileException;
|
||||
import java.security.AccessController;
|
||||
import java.util.HashMap;
|
||||
import java.util.Locale;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
@ -91,7 +91,7 @@ public class AzureStorageServiceMock extends AzureStorageService {
|
|||
|
||||
@Override
|
||||
public Map<String, BlobMetaData> listBlobsByPrefix(String account, String container, String keyPath, String prefix) {
|
||||
MapBuilder<String, BlobMetaData> blobsBuilder = MapBuilder.newMapBuilder();
|
||||
final var blobsBuilder = new HashMap<String, BlobMetaData>();
|
||||
blobs.forEach((String blobName, ByteArrayOutputStream bos) -> {
|
||||
final String checkBlob;
|
||||
if (keyPath != null && !keyPath.isEmpty()) {
|
||||
|
@ -104,7 +104,7 @@ public class AzureStorageServiceMock extends AzureStorageService {
|
|||
blobsBuilder.put(blobName, new PlainBlobMetaData(checkBlob, bos.size()));
|
||||
}
|
||||
});
|
||||
return blobsBuilder.immutableMap();
|
||||
return Map.copyOf(blobsBuilder);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -91,10 +91,12 @@ import java.util.concurrent.atomic.AtomicBoolean;
|
|||
import java.util.function.Consumer;
|
||||
import java.util.function.LongSupplier;
|
||||
import java.util.function.Supplier;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
import static java.util.Collections.emptyMap;
|
||||
import static java.util.Collections.unmodifiableMap;
|
||||
import static org.elasticsearch.common.collect.MapBuilder.newMapBuilder;
|
||||
import static java.util.Map.entry;
|
||||
|
||||
public class IndexService extends AbstractIndexComponent implements IndicesClusterStateService.AllocatedIndex<IndexShard> {
|
||||
|
||||
|
@ -428,7 +430,8 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
|
|||
circuitBreakerService);
|
||||
eventListener.indexShardStateChanged(indexShard, null, indexShard.state(), "shard created");
|
||||
eventListener.afterIndexShardCreated(indexShard);
|
||||
shards = newMapBuilder(shards).put(shardId.id(), indexShard).immutableMap();
|
||||
shards = Stream.concat(shards.entrySet().stream(), Stream.of(entry(shardId.id(), indexShard)))
|
||||
.collect(Collectors.toUnmodifiableMap(Map.Entry::getKey, Map.Entry::getValue));
|
||||
success = true;
|
||||
return indexShard;
|
||||
} catch (ShardLockObtainFailedException e) {
|
||||
|
|
|
@ -25,9 +25,9 @@ import org.apache.lucene.codecs.lucene50.Lucene50PostingsFormat.FSTLoadMode;
|
|||
import org.apache.lucene.codecs.lucene50.Lucene50StoredFieldsFormat.Mode;
|
||||
import org.apache.lucene.codecs.lucene80.Lucene80Codec;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.collect.MapBuilder;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
|
@ -46,7 +46,7 @@ public class CodecService {
|
|||
public static final String LUCENE_DEFAULT_CODEC = "lucene_default";
|
||||
|
||||
public CodecService(@Nullable MapperService mapperService, Logger logger) {
|
||||
final MapBuilder<String, Codec> codecs = MapBuilder.<String, Codec>newMapBuilder();
|
||||
final var codecs = new HashMap<String, Codec>();
|
||||
if (mapperService == null) {
|
||||
codecs.put(DEFAULT_CODEC, new Lucene80Codec());
|
||||
codecs.put(BEST_COMPRESSION_CODEC, new Lucene80Codec(Mode.BEST_COMPRESSION, FSTLoadMode.AUTO));
|
||||
|
@ -60,7 +60,7 @@ public class CodecService {
|
|||
for (String codec : Codec.availableCodecs()) {
|
||||
codecs.put(codec, Codec.forName(codec));
|
||||
}
|
||||
this.codecs = codecs.immutableMap();
|
||||
this.codecs = Map.copyOf(codecs);
|
||||
}
|
||||
|
||||
public Codec codec(String name) {
|
||||
|
|
|
@ -19,7 +19,6 @@
|
|||
package org.elasticsearch.index.engine;
|
||||
|
||||
import org.apache.lucene.index.SegmentInfos;
|
||||
import org.elasticsearch.common.collect.MapBuilder;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.io.stream.Streamable;
|
||||
|
@ -28,9 +27,12 @@ import org.elasticsearch.common.xcontent.ToXContentFragment;
|
|||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Base64;
|
||||
import java.util.Map;
|
||||
|
||||
import static java.util.Map.entry;
|
||||
|
||||
/** a class the returns dynamic information with respect to the last commit point of this shard */
|
||||
public final class CommitStats implements Streamable, ToXContentFragment {
|
||||
|
||||
|
@ -41,7 +43,7 @@ public final class CommitStats implements Streamable, ToXContentFragment {
|
|||
|
||||
public CommitStats(SegmentInfos segmentInfos) {
|
||||
// clone the map to protect against concurrent changes
|
||||
userData = MapBuilder.<String, String>newMapBuilder().putAll(segmentInfos.getUserData()).immutableMap();
|
||||
userData = Map.copyOf(segmentInfos.getUserData());
|
||||
// lucene calls the current generation, last generation.
|
||||
generation = segmentInfos.getLastGeneration();
|
||||
id = Base64.getEncoder().encodeToString(segmentInfos.getId());
|
||||
|
@ -92,11 +94,13 @@ public final class CommitStats implements Streamable, ToXContentFragment {
|
|||
|
||||
@Override
|
||||
public void readFrom(StreamInput in) throws IOException {
|
||||
MapBuilder<String, String> builder = MapBuilder.newMapBuilder();
|
||||
for (int i = in.readVInt(); i > 0; i--) {
|
||||
builder.put(in.readString(), in.readString());
|
||||
final int length = in.readVInt();
|
||||
final var entries = new ArrayList<Map.Entry<String, String>>(length);
|
||||
for (int i = length; i > 0; i--) {
|
||||
entries.add(entry(in.readString(), in.readString()));
|
||||
}
|
||||
userData = builder.immutableMap();
|
||||
// noinspection unchecked
|
||||
userData = Map.ofEntries(entries.toArray((Map.Entry<String, String>[])new Map.Entry[0]));
|
||||
generation = in.readLong();
|
||||
id = in.readOptionalString();
|
||||
numDocs = in.readInt();
|
||||
|
|
|
@ -23,7 +23,6 @@ import org.elasticsearch.ElasticsearchParseException;
|
|||
import org.elasticsearch.common.CheckedConsumer;
|
||||
import org.elasticsearch.common.bytes.BytesArray;
|
||||
import org.elasticsearch.common.bytes.BytesReference;
|
||||
import org.elasticsearch.common.collect.MapBuilder;
|
||||
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
|
||||
import org.elasticsearch.common.xcontent.XContentParser;
|
||||
import org.elasticsearch.common.xcontent.XContentType;
|
||||
|
@ -126,16 +125,17 @@ public class RestRequestTests extends ESTestCase {
|
|||
contentRestRequest("", emptyMap()).contentOrSourceParam());
|
||||
assertEquals("request body or source parameter is required", e.getMessage());
|
||||
assertEquals(new BytesArray("stuff"), contentRestRequest("stuff", emptyMap()).contentOrSourceParam().v2());
|
||||
assertEquals(new BytesArray("stuff"),
|
||||
contentRestRequest("stuff", MapBuilder.<String, String>newMapBuilder()
|
||||
.put("source", "stuff2").put("source_content_type", "application/json").immutableMap()).contentOrSourceParam().v2());
|
||||
assertEquals(
|
||||
new BytesArray("stuff"),
|
||||
contentRestRequest(
|
||||
"stuff",
|
||||
Map.of("source", "stuff2", "source_content_type", "application/json"))
|
||||
.contentOrSourceParam().v2());
|
||||
assertEquals(new BytesArray("{\"foo\": \"stuff\"}"),
|
||||
contentRestRequest("", MapBuilder.<String, String>newMapBuilder()
|
||||
.put("source", "{\"foo\": \"stuff\"}").put("source_content_type", "application/json").immutableMap())
|
||||
contentRestRequest("", Map.of("source", "{\"foo\": \"stuff\"}", "source_content_type", "application/json"))
|
||||
.contentOrSourceParam().v2());
|
||||
e = expectThrows(IllegalStateException.class, () ->
|
||||
contentRestRequest("", MapBuilder.<String, String>newMapBuilder()
|
||||
.put("source", "stuff2").immutableMap()).contentOrSourceParam());
|
||||
contentRestRequest("", Map.of("source", "stuff2")).contentOrSourceParam());
|
||||
assertEquals("source and source_content_type parameters are required", e.getMessage());
|
||||
}
|
||||
|
||||
|
@ -152,8 +152,12 @@ public class RestRequestTests extends ESTestCase {
|
|||
assertEquals("request body or source parameter is required", e.getMessage());
|
||||
assertEquals(emptyMap(), contentRestRequest("{}", emptyMap()).contentOrSourceParamParser().map());
|
||||
assertEquals(emptyMap(), contentRestRequest("{}", singletonMap("source", "stuff2")).contentOrSourceParamParser().map());
|
||||
assertEquals(emptyMap(), contentRestRequest("", MapBuilder.<String, String>newMapBuilder()
|
||||
.put("source", "{}").put("source_content_type", "application/json").immutableMap()).contentOrSourceParamParser().map());
|
||||
assertEquals(
|
||||
emptyMap(),
|
||||
contentRestRequest(
|
||||
"",
|
||||
Map.of("source", "{}", "source_content_type", "application/json"))
|
||||
.contentOrSourceParamParser().map());
|
||||
}
|
||||
|
||||
public void testWithContentOrSourceParamParserOrNull() throws IOException {
|
||||
|
@ -161,10 +165,10 @@ public class RestRequestTests extends ESTestCase {
|
|||
contentRestRequest("{}", emptyMap()).withContentOrSourceParamParserOrNull(parser -> assertEquals(emptyMap(), parser.map()));
|
||||
contentRestRequest("{}", singletonMap("source", "stuff2")).withContentOrSourceParamParserOrNull(parser ->
|
||||
assertEquals(emptyMap(), parser.map()));
|
||||
contentRestRequest("", MapBuilder.<String, String>newMapBuilder().put("source_content_type", "application/json")
|
||||
.put("source", "{}").immutableMap())
|
||||
.withContentOrSourceParamParserOrNull(parser ->
|
||||
assertEquals(emptyMap(), parser.map()));
|
||||
contentRestRequest(
|
||||
"",
|
||||
Map.of("source_content_type", "application/json", "source", "{}"))
|
||||
.withContentOrSourceParamParserOrNull(parser -> assertEquals(emptyMap(), parser.map()));
|
||||
}
|
||||
|
||||
public void testContentTypeParsing() {
|
||||
|
@ -221,13 +225,15 @@ public class RestRequestTests extends ESTestCase {
|
|||
contentRestRequest("", emptyMap()).requiredContent());
|
||||
assertEquals("request body is required", e.getMessage());
|
||||
assertEquals(new BytesArray("stuff"), contentRestRequest("stuff", emptyMap()).requiredContent());
|
||||
assertEquals(new BytesArray("stuff"),
|
||||
contentRestRequest("stuff", MapBuilder.<String, String>newMapBuilder()
|
||||
.put("source", "stuff2").put("source_content_type", "application/json").immutableMap()).requiredContent());
|
||||
e = expectThrows(ElasticsearchParseException.class, () ->
|
||||
contentRestRequest("", MapBuilder.<String, String>newMapBuilder()
|
||||
.put("source", "{\"foo\": \"stuff\"}").put("source_content_type", "application/json").immutableMap())
|
||||
.requiredContent());
|
||||
assertEquals(
|
||||
new BytesArray("stuff"),
|
||||
contentRestRequest("stuff", Map.of("source", "stuff2", "source_content_type", "application/json")).requiredContent());
|
||||
e = expectThrows(
|
||||
ElasticsearchParseException.class,
|
||||
() -> contentRestRequest(
|
||||
"",
|
||||
Map.of("source", "{\"foo\": \"stuff\"}", "source_content_type", "application/json"))
|
||||
.requiredContent());
|
||||
assertEquals("request body is required", e.getMessage());
|
||||
e = expectThrows(IllegalStateException.class, () ->
|
||||
contentRestRequest("test", null, Collections.emptyMap()).requiredContent());
|
||||
|
|
|
@ -21,7 +21,6 @@ package org.elasticsearch.indices.analysis;
|
|||
|
||||
import org.apache.lucene.analysis.util.TokenFilterFactory;
|
||||
import org.apache.lucene.analysis.util.TokenizerFactory;
|
||||
import org.elasticsearch.common.collect.MapBuilder;
|
||||
import org.elasticsearch.index.analysis.HunspellTokenFilterFactory;
|
||||
import org.elasticsearch.index.analysis.ShingleTokenFilterFactory;
|
||||
import org.elasticsearch.index.analysis.StandardTokenizerFactory;
|
||||
|
@ -39,6 +38,7 @@ import java.util.regex.Matcher;
|
|||
import java.util.regex.Pattern;
|
||||
|
||||
import static java.util.Collections.emptyMap;
|
||||
import static java.util.Map.entry;
|
||||
|
||||
/**
|
||||
* Alerts us if new analysis components are added to Lucene, so we don't miss them.
|
||||
|
@ -61,164 +61,156 @@ public abstract class AnalysisFactoryTestCase extends ESTestCase {
|
|||
return sb.toString();
|
||||
}
|
||||
|
||||
static final Map<String,Class<?>> KNOWN_TOKENIZERS = new MapBuilder<String,Class<?>>()
|
||||
static final Map<String,Class<?>> KNOWN_TOKENIZERS = Map.ofEntries(
|
||||
// exposed in ES
|
||||
.put("classic", MovedToAnalysisCommon.class)
|
||||
.put("edgengram", MovedToAnalysisCommon.class)
|
||||
.put("keyword", MovedToAnalysisCommon.class)
|
||||
.put("letter", MovedToAnalysisCommon.class)
|
||||
.put("ngram", MovedToAnalysisCommon.class)
|
||||
.put("pathhierarchy", MovedToAnalysisCommon.class)
|
||||
.put("pattern", MovedToAnalysisCommon.class)
|
||||
.put("simplepattern", MovedToAnalysisCommon.class)
|
||||
.put("simplepatternsplit", MovedToAnalysisCommon.class)
|
||||
.put("standard", StandardTokenizerFactory.class)
|
||||
.put("thai", MovedToAnalysisCommon.class)
|
||||
.put("uax29urlemail", MovedToAnalysisCommon.class)
|
||||
.put("whitespace", MovedToAnalysisCommon.class)
|
||||
|
||||
entry("classic", MovedToAnalysisCommon.class),
|
||||
entry("edgengram", MovedToAnalysisCommon.class),
|
||||
entry("keyword", MovedToAnalysisCommon.class),
|
||||
entry("letter", MovedToAnalysisCommon.class),
|
||||
entry("ngram", MovedToAnalysisCommon.class),
|
||||
entry("pathhierarchy", MovedToAnalysisCommon.class),
|
||||
entry("pattern", MovedToAnalysisCommon.class),
|
||||
entry("simplepattern", MovedToAnalysisCommon.class),
|
||||
entry("simplepatternsplit", MovedToAnalysisCommon.class),
|
||||
entry("standard", StandardTokenizerFactory.class),
|
||||
entry("thai", MovedToAnalysisCommon.class),
|
||||
entry("uax29urlemail", MovedToAnalysisCommon.class),
|
||||
entry("whitespace", MovedToAnalysisCommon.class),
|
||||
// this one "seems to mess up offsets". probably shouldn't be a tokenizer...
|
||||
.put("wikipedia", Void.class)
|
||||
.immutableMap();
|
||||
entry("wikipedia", Void.class));
|
||||
|
||||
static final Map<String,Class<?>> KNOWN_TOKENFILTERS = new MapBuilder<String,Class<?>>()
|
||||
static final Map<String,Class<?>> KNOWN_TOKENFILTERS = Map.ofEntries(
|
||||
// exposed in ES
|
||||
.put("apostrophe", MovedToAnalysisCommon.class)
|
||||
.put("arabicnormalization", MovedToAnalysisCommon.class)
|
||||
.put("arabicstem", MovedToAnalysisCommon.class)
|
||||
.put("asciifolding", MovedToAnalysisCommon.class)
|
||||
.put("bengalinormalization", MovedToAnalysisCommon.class)
|
||||
.put("bengalistem", MovedToAnalysisCommon.class)
|
||||
.put("brazilianstem", MovedToAnalysisCommon.class)
|
||||
.put("bulgarianstem", MovedToAnalysisCommon.class)
|
||||
.put("cjkbigram", MovedToAnalysisCommon.class)
|
||||
.put("cjkwidth", MovedToAnalysisCommon.class)
|
||||
.put("classic", MovedToAnalysisCommon.class)
|
||||
.put("commongrams", MovedToAnalysisCommon.class)
|
||||
.put("commongramsquery", MovedToAnalysisCommon.class)
|
||||
.put("czechstem", MovedToAnalysisCommon.class)
|
||||
.put("decimaldigit", MovedToAnalysisCommon.class)
|
||||
.put("delimitedpayload", MovedToAnalysisCommon.class)
|
||||
.put("dictionarycompoundword", MovedToAnalysisCommon.class)
|
||||
.put("edgengram", MovedToAnalysisCommon.class)
|
||||
.put("elision", MovedToAnalysisCommon.class)
|
||||
.put("englishminimalstem", MovedToAnalysisCommon.class)
|
||||
.put("englishpossessive", MovedToAnalysisCommon.class)
|
||||
.put("finnishlightstem", MovedToAnalysisCommon.class)
|
||||
.put("fixedshingle", MovedToAnalysisCommon.class)
|
||||
.put("frenchlightstem", MovedToAnalysisCommon.class)
|
||||
.put("frenchminimalstem", MovedToAnalysisCommon.class)
|
||||
.put("galicianminimalstem", MovedToAnalysisCommon.class)
|
||||
.put("galicianstem", MovedToAnalysisCommon.class)
|
||||
.put("germanstem", MovedToAnalysisCommon.class)
|
||||
.put("germanlightstem", MovedToAnalysisCommon.class)
|
||||
.put("germanminimalstem", MovedToAnalysisCommon.class)
|
||||
.put("germannormalization", MovedToAnalysisCommon.class)
|
||||
.put("greeklowercase", MovedToAnalysisCommon.class)
|
||||
.put("greekstem", MovedToAnalysisCommon.class)
|
||||
.put("hindinormalization", MovedToAnalysisCommon.class)
|
||||
.put("hindistem", MovedToAnalysisCommon.class)
|
||||
.put("hungarianlightstem", MovedToAnalysisCommon.class)
|
||||
.put("hunspellstem", HunspellTokenFilterFactory.class)
|
||||
.put("hyphenationcompoundword", MovedToAnalysisCommon.class)
|
||||
.put("indicnormalization", MovedToAnalysisCommon.class)
|
||||
.put("irishlowercase", MovedToAnalysisCommon.class)
|
||||
.put("indonesianstem", MovedToAnalysisCommon.class)
|
||||
.put("italianlightstem", MovedToAnalysisCommon.class)
|
||||
.put("keepword", MovedToAnalysisCommon.class)
|
||||
.put("keywordmarker", MovedToAnalysisCommon.class)
|
||||
.put("kstem", MovedToAnalysisCommon.class)
|
||||
.put("latvianstem", MovedToAnalysisCommon.class)
|
||||
.put("length", MovedToAnalysisCommon.class)
|
||||
.put("limittokencount", MovedToAnalysisCommon.class)
|
||||
.put("lowercase", MovedToAnalysisCommon.class)
|
||||
.put("ngram", MovedToAnalysisCommon.class)
|
||||
.put("norwegianlightstem", MovedToAnalysisCommon.class)
|
||||
.put("norwegianminimalstem", MovedToAnalysisCommon.class)
|
||||
.put("patterncapturegroup", MovedToAnalysisCommon.class)
|
||||
.put("patternreplace", MovedToAnalysisCommon.class)
|
||||
.put("persiannormalization", MovedToAnalysisCommon.class)
|
||||
.put("porterstem", MovedToAnalysisCommon.class)
|
||||
.put("portuguesestem", MovedToAnalysisCommon.class)
|
||||
.put("portugueselightstem", MovedToAnalysisCommon.class)
|
||||
.put("portugueseminimalstem", MovedToAnalysisCommon.class)
|
||||
.put("reversestring", MovedToAnalysisCommon.class)
|
||||
.put("russianlightstem", MovedToAnalysisCommon.class)
|
||||
.put("scandinavianfolding", MovedToAnalysisCommon.class)
|
||||
.put("scandinaviannormalization", MovedToAnalysisCommon.class)
|
||||
.put("serbiannormalization", MovedToAnalysisCommon.class)
|
||||
.put("shingle", ShingleTokenFilterFactory.class)
|
||||
.put("minhash", MovedToAnalysisCommon.class)
|
||||
.put("snowballporter", MovedToAnalysisCommon.class)
|
||||
.put("soraninormalization", MovedToAnalysisCommon.class)
|
||||
.put("soranistem", MovedToAnalysisCommon.class)
|
||||
.put("spanishlightstem", MovedToAnalysisCommon.class)
|
||||
.put("stemmeroverride", MovedToAnalysisCommon.class)
|
||||
.put("stop", StopTokenFilterFactory.class)
|
||||
.put("swedishlightstem", MovedToAnalysisCommon.class)
|
||||
.put("synonym", MovedToAnalysisCommon.class)
|
||||
.put("synonymgraph", MovedToAnalysisCommon.class)
|
||||
.put("trim", MovedToAnalysisCommon.class)
|
||||
.put("truncate", MovedToAnalysisCommon.class)
|
||||
.put("turkishlowercase", MovedToAnalysisCommon.class)
|
||||
.put("type", MovedToAnalysisCommon.class)
|
||||
.put("uppercase", MovedToAnalysisCommon.class)
|
||||
.put("worddelimiter", MovedToAnalysisCommon.class)
|
||||
.put("worddelimitergraph", MovedToAnalysisCommon.class)
|
||||
.put("flattengraph", MovedToAnalysisCommon.class)
|
||||
|
||||
entry("apostrophe", MovedToAnalysisCommon.class),
|
||||
entry("arabicnormalization", MovedToAnalysisCommon.class),
|
||||
entry("arabicstem", MovedToAnalysisCommon.class),
|
||||
entry("asciifolding", MovedToAnalysisCommon.class),
|
||||
entry("bengalinormalization", MovedToAnalysisCommon.class),
|
||||
entry("bengalistem", MovedToAnalysisCommon.class),
|
||||
entry("brazilianstem", MovedToAnalysisCommon.class),
|
||||
entry("bulgarianstem", MovedToAnalysisCommon.class),
|
||||
entry("cjkbigram", MovedToAnalysisCommon.class),
|
||||
entry("cjkwidth", MovedToAnalysisCommon.class),
|
||||
entry("classic", MovedToAnalysisCommon.class),
|
||||
entry("commongrams", MovedToAnalysisCommon.class),
|
||||
entry("commongramsquery", MovedToAnalysisCommon.class),
|
||||
entry("czechstem", MovedToAnalysisCommon.class),
|
||||
entry("decimaldigit", MovedToAnalysisCommon.class),
|
||||
entry("delimitedpayload", MovedToAnalysisCommon.class),
|
||||
entry("dictionarycompoundword", MovedToAnalysisCommon.class),
|
||||
entry("edgengram", MovedToAnalysisCommon.class),
|
||||
entry("elision", MovedToAnalysisCommon.class),
|
||||
entry("englishminimalstem", MovedToAnalysisCommon.class),
|
||||
entry("englishpossessive", MovedToAnalysisCommon.class),
|
||||
entry("finnishlightstem", MovedToAnalysisCommon.class),
|
||||
entry("fixedshingle", MovedToAnalysisCommon.class),
|
||||
entry("frenchlightstem", MovedToAnalysisCommon.class),
|
||||
entry("frenchminimalstem", MovedToAnalysisCommon.class),
|
||||
entry("galicianminimalstem", MovedToAnalysisCommon.class),
|
||||
entry("galicianstem", MovedToAnalysisCommon.class),
|
||||
entry("germanstem", MovedToAnalysisCommon.class),
|
||||
entry("germanlightstem", MovedToAnalysisCommon.class),
|
||||
entry("germanminimalstem", MovedToAnalysisCommon.class),
|
||||
entry("germannormalization", MovedToAnalysisCommon.class),
|
||||
entry("greeklowercase", MovedToAnalysisCommon.class),
|
||||
entry("greekstem", MovedToAnalysisCommon.class),
|
||||
entry("hindinormalization", MovedToAnalysisCommon.class),
|
||||
entry("hindistem", MovedToAnalysisCommon.class),
|
||||
entry("hungarianlightstem", MovedToAnalysisCommon.class),
|
||||
entry("hunspellstem", HunspellTokenFilterFactory.class),
|
||||
entry("hyphenationcompoundword", MovedToAnalysisCommon.class),
|
||||
entry("indicnormalization", MovedToAnalysisCommon.class),
|
||||
entry("irishlowercase", MovedToAnalysisCommon.class),
|
||||
entry("indonesianstem", MovedToAnalysisCommon.class),
|
||||
entry("italianlightstem", MovedToAnalysisCommon.class),
|
||||
entry("keepword", MovedToAnalysisCommon.class),
|
||||
entry("keywordmarker", MovedToAnalysisCommon.class),
|
||||
entry("kstem", MovedToAnalysisCommon.class),
|
||||
entry("latvianstem", MovedToAnalysisCommon.class),
|
||||
entry("length", MovedToAnalysisCommon.class),
|
||||
entry("limittokencount", MovedToAnalysisCommon.class),
|
||||
entry("lowercase", MovedToAnalysisCommon.class),
|
||||
entry("ngram", MovedToAnalysisCommon.class),
|
||||
entry("norwegianlightstem", MovedToAnalysisCommon.class),
|
||||
entry("norwegianminimalstem", MovedToAnalysisCommon.class),
|
||||
entry("patterncapturegroup", MovedToAnalysisCommon.class),
|
||||
entry("patternreplace", MovedToAnalysisCommon.class),
|
||||
entry("persiannormalization", MovedToAnalysisCommon.class),
|
||||
entry("porterstem", MovedToAnalysisCommon.class),
|
||||
entry("portuguesestem", MovedToAnalysisCommon.class),
|
||||
entry("portugueselightstem", MovedToAnalysisCommon.class),
|
||||
entry("portugueseminimalstem", MovedToAnalysisCommon.class),
|
||||
entry("reversestring", MovedToAnalysisCommon.class),
|
||||
entry("russianlightstem", MovedToAnalysisCommon.class),
|
||||
entry("scandinavianfolding", MovedToAnalysisCommon.class),
|
||||
entry("scandinaviannormalization", MovedToAnalysisCommon.class),
|
||||
entry("serbiannormalization", MovedToAnalysisCommon.class),
|
||||
entry("shingle", ShingleTokenFilterFactory.class),
|
||||
entry("minhash", MovedToAnalysisCommon.class),
|
||||
entry("snowballporter", MovedToAnalysisCommon.class),
|
||||
entry("soraninormalization", MovedToAnalysisCommon.class),
|
||||
entry("soranistem", MovedToAnalysisCommon.class),
|
||||
entry("spanishlightstem", MovedToAnalysisCommon.class),
|
||||
entry("stemmeroverride", MovedToAnalysisCommon.class),
|
||||
entry("stop", StopTokenFilterFactory.class),
|
||||
entry("swedishlightstem", MovedToAnalysisCommon.class),
|
||||
entry("synonym", MovedToAnalysisCommon.class),
|
||||
entry("synonymgraph", MovedToAnalysisCommon.class),
|
||||
entry("trim", MovedToAnalysisCommon.class),
|
||||
entry("truncate", MovedToAnalysisCommon.class),
|
||||
entry("turkishlowercase", MovedToAnalysisCommon.class),
|
||||
entry("type", MovedToAnalysisCommon.class),
|
||||
entry("uppercase", MovedToAnalysisCommon.class),
|
||||
entry("worddelimiter", MovedToAnalysisCommon.class),
|
||||
entry("worddelimitergraph", MovedToAnalysisCommon.class),
|
||||
entry("flattengraph", MovedToAnalysisCommon.class),
|
||||
// TODO: these tokenfilters are not yet exposed: useful?
|
||||
// suggest stop
|
||||
.put("suggeststop", Void.class)
|
||||
entry("suggeststop", Void.class),
|
||||
// capitalizes tokens
|
||||
.put("capitalization", Void.class)
|
||||
entry("capitalization", Void.class),
|
||||
// like length filter (but codepoints)
|
||||
.put("codepointcount", Void.class)
|
||||
entry("codepointcount", Void.class),
|
||||
// puts hyphenated words back together
|
||||
.put("hyphenatedwords", Void.class)
|
||||
entry("hyphenatedwords", Void.class),
|
||||
// repeats anything marked as keyword
|
||||
.put("keywordrepeat", Void.class)
|
||||
entry("keywordrepeat", Void.class),
|
||||
// like limittokencount, but by offset
|
||||
.put("limittokenoffset", Void.class)
|
||||
entry("limittokenoffset", Void.class),
|
||||
// like limittokencount, but by position
|
||||
.put("limittokenposition", Void.class)
|
||||
entry("limittokenposition", Void.class),
|
||||
// ???
|
||||
.put("numericpayload", Void.class)
|
||||
entry("numericpayload", Void.class),
|
||||
// removes duplicates at the same position (this should be used by the existing factory)
|
||||
.put("removeduplicates", Void.class)
|
||||
entry("removeduplicates", Void.class),
|
||||
// ???
|
||||
.put("tokenoffsetpayload", Void.class)
|
||||
entry("tokenoffsetpayload", Void.class),
|
||||
// puts the type into the payload
|
||||
.put("typeaspayload", Void.class)
|
||||
entry("typeaspayload", Void.class),
|
||||
// puts the type as a synonym
|
||||
.put("typeassynonym", Void.class)
|
||||
entry("typeassynonym", Void.class),
|
||||
// fingerprint
|
||||
.put("fingerprint", Void.class)
|
||||
entry("fingerprint", Void.class),
|
||||
// for tee-sinks
|
||||
.put("daterecognizer", Void.class)
|
||||
entry("daterecognizer", Void.class),
|
||||
// for token filters that generate bad offsets, which are now rejected since Lucene 7
|
||||
.put("fixbrokenoffsets", Void.class)
|
||||
entry("fixbrokenoffsets", Void.class),
|
||||
// should we expose it, or maybe think about higher level integration of the
|
||||
// fake term frequency feature (LUCENE-7854)
|
||||
.put("delimitedtermfrequency", Void.class)
|
||||
entry("delimitedtermfrequency", Void.class),
|
||||
// LUCENE-8273: ProtectedTermFilterFactory allows analysis chains to skip
|
||||
// particular token filters based on the attributes of the current token.
|
||||
.put("protectedterm", Void.class)
|
||||
entry("protectedterm", Void.class),
|
||||
// LUCENE-8332
|
||||
.put("concatenategraph", Void.class)
|
||||
entry("concatenategraph", Void.class));
|
||||
|
||||
.immutableMap();
|
||||
|
||||
static final Map<String,Class<?>> KNOWN_CHARFILTERS = new MapBuilder<String,Class<?>>()
|
||||
// exposed in ES
|
||||
.put("htmlstrip", MovedToAnalysisCommon.class)
|
||||
.put("mapping", MovedToAnalysisCommon.class)
|
||||
.put("patternreplace", MovedToAnalysisCommon.class)
|
||||
|
||||
// TODO: these charfilters are not yet exposed: useful?
|
||||
// handling of zwnj for persian
|
||||
.put("persian", Void.class)
|
||||
.immutableMap();
|
||||
static final Map<String, Class<?>> KNOWN_CHARFILTERS = Map.of(
|
||||
"htmlstrip", MovedToAnalysisCommon.class,
|
||||
"mapping", MovedToAnalysisCommon.class,
|
||||
"patternreplace", MovedToAnalysisCommon.class,
|
||||
// TODO: these charfilters are not yet exposed: useful?
|
||||
// handling of zwnj for persian
|
||||
"persian", Void.class);
|
||||
|
||||
/**
|
||||
* The plugin being tested. Core uses an "empty" plugin so we don't have to throw null checks all over the place.
|
||||
|
|
|
@ -12,7 +12,6 @@ import org.elasticsearch.action.admin.cluster.snapshots.get.GetSnapshotsAction;
|
|||
import org.elasticsearch.action.admin.cluster.snapshots.status.SnapshotsStatusAction;
|
||||
import org.elasticsearch.action.admin.cluster.state.ClusterStateAction;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.collect.MapBuilder;
|
||||
import org.elasticsearch.xpack.core.indexlifecycle.action.GetLifecycleAction;
|
||||
import org.elasticsearch.xpack.core.indexlifecycle.action.GetStatusAction;
|
||||
import org.elasticsearch.xpack.core.security.action.token.InvalidateTokenAction;
|
||||
|
@ -28,6 +27,7 @@ import java.util.Set;
|
|||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.function.Predicate;
|
||||
|
||||
import static java.util.Map.entry;
|
||||
import static org.elasticsearch.xpack.core.security.support.Automatons.minusAndMinimize;
|
||||
import static org.elasticsearch.xpack.core.security.support.Automatons.patterns;
|
||||
|
||||
|
@ -93,33 +93,32 @@ public final class ClusterPrivilege extends Privilege {
|
|||
|
||||
public static final Predicate<String> ACTION_MATCHER = ClusterPrivilege.ALL.predicate();
|
||||
|
||||
private static final Map<String, ClusterPrivilege> VALUES = MapBuilder.<String, ClusterPrivilege>newMapBuilder()
|
||||
.put("none", NONE)
|
||||
.put("all", ALL)
|
||||
.put("monitor", MONITOR)
|
||||
.put("monitor_ml", MONITOR_ML)
|
||||
.put("monitor_data_frame_transforms", MONITOR_DATA_FRAME)
|
||||
.put("monitor_watcher", MONITOR_WATCHER)
|
||||
.put("monitor_rollup", MONITOR_ROLLUP)
|
||||
.put("manage", MANAGE)
|
||||
.put("manage_ml", MANAGE_ML)
|
||||
.put("manage_data_frame_transforms", MANAGE_DATA_FRAME)
|
||||
.put("manage_token", MANAGE_TOKEN)
|
||||
.put("manage_watcher", MANAGE_WATCHER)
|
||||
.put("manage_index_templates", MANAGE_IDX_TEMPLATES)
|
||||
.put("manage_ingest_pipelines", MANAGE_INGEST_PIPELINES)
|
||||
.put("transport_client", TRANSPORT_CLIENT)
|
||||
.put("manage_security", MANAGE_SECURITY)
|
||||
.put("manage_saml", MANAGE_SAML)
|
||||
.put("manage_oidc", MANAGE_OIDC)
|
||||
.put("manage_pipeline", MANAGE_PIPELINE)
|
||||
.put("manage_rollup", MANAGE_ROLLUP)
|
||||
.put("manage_ccr", MANAGE_CCR)
|
||||
.put("read_ccr", READ_CCR)
|
||||
.put("create_snapshot", CREATE_SNAPSHOT)
|
||||
.put("manage_ilm", MANAGE_ILM)
|
||||
.put("read_ilm", READ_ILM)
|
||||
.immutableMap();
|
||||
private static final Map<String, ClusterPrivilege> VALUES = Map.ofEntries(
|
||||
entry("none", NONE),
|
||||
entry("all", ALL),
|
||||
entry("monitor", MONITOR),
|
||||
entry("monitor_ml", MONITOR_ML),
|
||||
entry("monitor_data_frame_transforms", MONITOR_DATA_FRAME),
|
||||
entry("monitor_watcher", MONITOR_WATCHER),
|
||||
entry("monitor_rollup", MONITOR_ROLLUP),
|
||||
entry("manage", MANAGE),
|
||||
entry("manage_ml", MANAGE_ML),
|
||||
entry("manage_data_frame_transforms", MANAGE_DATA_FRAME),
|
||||
entry("manage_token", MANAGE_TOKEN),
|
||||
entry("manage_watcher", MANAGE_WATCHER),
|
||||
entry("manage_index_templates", MANAGE_IDX_TEMPLATES),
|
||||
entry("manage_ingest_pipelines", MANAGE_INGEST_PIPELINES),
|
||||
entry("transport_client", TRANSPORT_CLIENT),
|
||||
entry("manage_security", MANAGE_SECURITY),
|
||||
entry("manage_saml", MANAGE_SAML),
|
||||
entry("manage_oidc", MANAGE_OIDC),
|
||||
entry("manage_pipeline", MANAGE_PIPELINE),
|
||||
entry("manage_rollup", MANAGE_ROLLUP),
|
||||
entry("manage_ccr", MANAGE_CCR),
|
||||
entry("read_ccr", READ_CCR),
|
||||
entry("create_snapshot", CREATE_SNAPSHOT),
|
||||
entry("manage_ilm", MANAGE_ILM),
|
||||
entry("read_ilm", READ_ILM));
|
||||
|
||||
private static final ConcurrentHashMap<Set<String>, ClusterPrivilege> CACHE = new ConcurrentHashMap<>();
|
||||
|
||||
|
|
|
@ -21,7 +21,6 @@ import org.elasticsearch.action.admin.indices.mapping.put.PutMappingAction;
|
|||
import org.elasticsearch.action.admin.indices.settings.get.GetSettingsAction;
|
||||
import org.elasticsearch.action.admin.indices.validate.query.ValidateQueryAction;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.collect.MapBuilder;
|
||||
import org.elasticsearch.xpack.core.ccr.action.ForgetFollowerAction;
|
||||
import org.elasticsearch.xpack.core.ccr.action.PutFollowAction;
|
||||
import org.elasticsearch.xpack.core.ccr.action.UnfollowAction;
|
||||
|
@ -37,6 +36,7 @@ import java.util.Set;
|
|||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.function.Predicate;
|
||||
|
||||
import static java.util.Map.entry;
|
||||
import static org.elasticsearch.xpack.core.security.support.Automatons.patterns;
|
||||
import static org.elasticsearch.xpack.core.security.support.Automatons.unionAndMinimize;
|
||||
|
||||
|
@ -83,24 +83,23 @@ public final class IndexPrivilege extends Privilege {
|
|||
public static final IndexPrivilege MANAGE_LEADER_INDEX = new IndexPrivilege("manage_leader_index", MANAGE_LEADER_INDEX_AUTOMATON);
|
||||
public static final IndexPrivilege MANAGE_ILM = new IndexPrivilege("manage_ilm", MANAGE_ILM_AUTOMATON);
|
||||
|
||||
private static final Map<String, IndexPrivilege> VALUES = MapBuilder.<String, IndexPrivilege>newMapBuilder()
|
||||
.put("none", NONE)
|
||||
.put("all", ALL)
|
||||
.put("manage", MANAGE)
|
||||
.put("create_index", CREATE_INDEX)
|
||||
.put("monitor", MONITOR)
|
||||
.put("read", READ)
|
||||
.put("index", INDEX)
|
||||
.put("delete", DELETE)
|
||||
.put("write", WRITE)
|
||||
.put("create", CREATE)
|
||||
.put("delete_index", DELETE_INDEX)
|
||||
.put("view_index_metadata", VIEW_METADATA)
|
||||
.put("read_cross_cluster", READ_CROSS_CLUSTER)
|
||||
.put("manage_follow_index", MANAGE_FOLLOW_INDEX)
|
||||
.put("manage_leader_index", MANAGE_LEADER_INDEX)
|
||||
.put("manage_ilm", MANAGE_ILM)
|
||||
.immutableMap();
|
||||
private static final Map<String, IndexPrivilege> VALUES = Map.ofEntries(
|
||||
entry("none", NONE),
|
||||
entry("all", ALL),
|
||||
entry("manage", MANAGE),
|
||||
entry("create_index", CREATE_INDEX),
|
||||
entry("monitor", MONITOR),
|
||||
entry("read", READ),
|
||||
entry("index", INDEX),
|
||||
entry("delete", DELETE),
|
||||
entry("write", WRITE),
|
||||
entry("create", CREATE),
|
||||
entry("delete_index", DELETE_INDEX),
|
||||
entry("view_index_metadata", VIEW_METADATA),
|
||||
entry("read_cross_cluster", READ_CROSS_CLUSTER),
|
||||
entry("manage_follow_index", MANAGE_FOLLOW_INDEX),
|
||||
entry("manage_leader_index", MANAGE_LEADER_INDEX),
|
||||
entry("manage_ilm", MANAGE_ILM));
|
||||
|
||||
public static final Predicate<String> ACTION_MATCHER = ALL.predicate();
|
||||
public static final Predicate<String> CREATE_INDEX_MATCHER = CREATE_INDEX.predicate();
|
||||
|
|
|
@ -7,10 +7,9 @@ package org.elasticsearch.xpack.security.transport.filter;
|
|||
|
||||
|
||||
import io.netty.handler.ipfilter.IpFilterRuleType;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.apache.lucene.util.SetOnce;
|
||||
import org.elasticsearch.common.collect.MapBuilder;
|
||||
import org.elasticsearch.common.settings.ClusterSettings;
|
||||
import org.elasticsearch.common.settings.Setting;
|
||||
import org.elasticsearch.common.settings.Setting.Property;
|
||||
|
@ -78,10 +77,9 @@ public class IPFilter {
|
|||
public static final Setting<List<String>> HTTP_FILTER_DENY_SETTING = Setting.listSetting(setting("http.filter.deny"),
|
||||
HTTP_FILTER_DENY_FALLBACK, Function.identity(), Property.Dynamic, Property.NodeScope);
|
||||
|
||||
public static final Map<String, Object> DISABLED_USAGE_STATS = new MapBuilder<String, Object>()
|
||||
.put("http", false)
|
||||
.put("transport", false)
|
||||
.immutableMap();
|
||||
public static final Map<String, Object> DISABLED_USAGE_STATS = Map.of(
|
||||
"http", false,
|
||||
"transport", false);
|
||||
|
||||
public static final SecurityIpFilterRule DEFAULT_PROFILE_ACCEPT_ALL = new SecurityIpFilterRule(true, "default:accept_all") {
|
||||
|
||||
|
|
|
@ -16,7 +16,6 @@ import org.elasticsearch.action.support.PlainActionFuture;
|
|||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.bytes.BytesArray;
|
||||
import org.elasticsearch.common.collect.MapBuilder;
|
||||
import org.elasticsearch.common.settings.SecureString;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.unit.ByteSizeUnit;
|
||||
|
@ -67,11 +66,11 @@ import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDI
|
|||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
|
||||
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoTimeout;
|
||||
import static org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken.basicAuthHeaderValue;
|
||||
import static org.elasticsearch.xpack.security.support.SecurityIndexManager.SECURITY_INDEX_NAME;
|
||||
import static org.elasticsearch.xpack.security.support.SecurityIndexManager.INTERNAL_SECURITY_INDEX;
|
||||
import static org.elasticsearch.xpack.security.support.SecurityIndexManager.SECURITY_INDEX_NAME;
|
||||
import static org.hamcrest.Matchers.arrayContaining;
|
||||
import static org.hamcrest.Matchers.containsString;
|
||||
import static org.hamcrest.Matchers.contains;
|
||||
import static org.hamcrest.Matchers.containsString;
|
||||
import static org.hamcrest.Matchers.equalTo;
|
||||
import static org.hamcrest.Matchers.is;
|
||||
import static org.hamcrest.Matchers.notNullValue;
|
||||
|
@ -846,10 +845,9 @@ public class NativeRealmIntegTests extends NativeRealmIntegTestCase {
|
|||
.TEST_ROLE).get();
|
||||
securityClient().preparePutUser("executor", "s3krit".toCharArray(), hasher, "superuser").get();
|
||||
final String token = basicAuthHeaderValue("executor", new SecureString("s3krit".toCharArray()));
|
||||
final Client client = client().filterWithHeader(MapBuilder.<String, String>newMapBuilder()
|
||||
.put("Authorization", token)
|
||||
.put("es-security-runas-user", "joe")
|
||||
.immutableMap());
|
||||
final Client client = client().filterWithHeader(Map.of(
|
||||
"Authorization", token,
|
||||
"es-security-runas-user", "joe"));
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
final int numberOfProcessors = Runtime.getRuntime().availableProcessors();
|
||||
final int numberOfThreads = scaledRandomIntBetween(numberOfProcessors, numberOfProcessors * 3);
|
||||
|
|
|
@ -5,7 +5,6 @@
|
|||
*/
|
||||
package org.elasticsearch.xpack.watcher.actions;
|
||||
|
||||
import org.elasticsearch.common.collect.MapBuilder;
|
||||
import org.elasticsearch.xpack.watcher.actions.email.EmailAction;
|
||||
import org.elasticsearch.xpack.watcher.actions.index.IndexAction;
|
||||
import org.elasticsearch.xpack.watcher.actions.jira.JiraAction;
|
||||
|
@ -46,12 +45,8 @@ public final class ActionBuilders {
|
|||
return IndexAction.builder(index);
|
||||
}
|
||||
|
||||
public static JiraAction.Builder jiraAction(String account, MapBuilder<String, Object> fields) {
|
||||
return jiraAction(account, fields.immutableMap());
|
||||
}
|
||||
|
||||
public static JiraAction.Builder jiraAction(String account, Map<String, Object> fields) {
|
||||
return JiraAction.builder(account, fields);
|
||||
return JiraAction.builder(account, Map.copyOf(fields));
|
||||
}
|
||||
|
||||
public static WebhookAction.Builder webhookAction(HttpRequestTemplate.Builder httpRequest) {
|
||||
|
|
|
@ -6,7 +6,6 @@
|
|||
package org.elasticsearch.xpack.watcher.execution;
|
||||
|
||||
import com.google.common.collect.Iterables;
|
||||
|
||||
import org.apache.logging.log4j.LogManager;
|
||||
import org.apache.logging.log4j.Logger;
|
||||
import org.apache.logging.log4j.message.ParameterizedMessage;
|
||||
|
@ -23,7 +22,6 @@ import org.elasticsearch.action.update.UpdateRequest;
|
|||
import org.elasticsearch.client.Client;
|
||||
import org.elasticsearch.cluster.routing.Preference;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.collect.MapBuilder;
|
||||
import org.elasticsearch.common.collect.Tuple;
|
||||
import org.elasticsearch.common.metrics.MeanMetric;
|
||||
import org.elasticsearch.common.settings.Setting;
|
||||
|
@ -342,10 +340,9 @@ public class ExecutionService {
|
|||
// at the moment we store the status together with the watch,
|
||||
// so we just need to update the watch itself
|
||||
// we do not want to update the status.state field, as it might have been deactivated in-between
|
||||
Map<String, String> parameters = MapBuilder.<String, String>newMapBuilder()
|
||||
.put(Watch.INCLUDE_STATUS_KEY, "true")
|
||||
.put(WatchStatus.INCLUDE_STATE, "false")
|
||||
.immutableMap();
|
||||
Map<String, String> parameters = Map.of(
|
||||
Watch.INCLUDE_STATUS_KEY, "true",
|
||||
WatchStatus.INCLUDE_STATE, "false");
|
||||
ToXContent.MapParams params = new ToXContent.MapParams(parameters);
|
||||
XContentBuilder source = JsonXContent.contentBuilder().
|
||||
startObject()
|
||||
|
|
Loading…
Reference in New Issue