diff --git a/core/src/main/java/org/elasticsearch/action/index/IndexRequest.java b/core/src/main/java/org/elasticsearch/action/index/IndexRequest.java index b792241b56c..f8b0ce6c13f 100644 --- a/core/src/main/java/org/elasticsearch/action/index/IndexRequest.java +++ b/core/src/main/java/org/elasticsearch/action/index/IndexRequest.java @@ -20,18 +20,16 @@ package org.elasticsearch.action.index; import org.elasticsearch.ElasticsearchGenerationException; -import org.elasticsearch.Version; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.DocumentRequest; import org.elasticsearch.action.RoutingMissingException; import org.elasticsearch.action.TimestampParsingException; import org.elasticsearch.action.support.replication.ReplicationRequest; import org.elasticsearch.client.Requests; -import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.common.Nullable; -import org.elasticsearch.common.Strings; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.StreamInput; @@ -42,8 +40,6 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentType; -import org.elasticsearch.index.Index; -import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.mapper.internal.TimestampFieldMapper; @@ -613,7 +609,7 @@ public class IndexRequest extends ReplicationRequest implements Do // generate id if not already provided and id generation is allowed if (allowIdGeneration) { if (id == null) { - id(Strings.base64UUID()); + id(UUIDs.base64UUID()); } } diff --git a/core/src/main/java/org/elasticsearch/cluster/ClusterState.java b/core/src/main/java/org/elasticsearch/cluster/ClusterState.java index 7fa26df15b6..17c0b646760 100644 --- a/core/src/main/java/org/elasticsearch/cluster/ClusterState.java +++ b/core/src/main/java/org/elasticsearch/cluster/ClusterState.java @@ -41,6 +41,7 @@ import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.io.stream.BytesStreamOutput; @@ -693,7 +694,7 @@ public class ClusterState implements ToXContent, Diffable { public ClusterState build() { if (UNKNOWN_UUID.equals(uuid)) { - uuid = Strings.randomBase64UUID(); + uuid = UUIDs.randomBase64UUID(); } return new ClusterState(clusterName, version, uuid, metaData, routingTable, nodes, blocks, customs.build(), fromDiff); } diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java index 0e2f2ba7e70..934300e84ce 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java @@ -33,7 +33,7 @@ import org.elasticsearch.cluster.routing.allocation.decider.DiskThresholdDecider import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.ParseFieldMatcher; -import org.elasticsearch.common.Strings; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.collect.HppcMaps; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.io.stream.StreamInput; @@ -973,7 +973,7 @@ public class MetaData implements Iterable, Diffable, Fr public Builder generateClusterUuidIfNeeded() { if (clusterUUID.equals("_na_")) { - clusterUUID = Strings.randomBase64UUID(); + clusterUUID = UUIDs.randomBase64UUID(); } return this; } diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java index 6c2b3e5f728..455bfb61739 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java @@ -41,6 +41,7 @@ import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Priority; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.ValidationException; import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.compress.CompressedXContent; @@ -301,7 +302,7 @@ public class MetaDataCreateIndexService extends AbstractComponent { indexSettingsBuilder.put(SETTING_CREATION_DATE, new DateTime(DateTimeZone.UTC).getMillis()); } - indexSettingsBuilder.put(SETTING_INDEX_UUID, Strings.randomBase64UUID()); + indexSettingsBuilder.put(SETTING_INDEX_UUID, UUIDs.randomBase64UUID()); Settings actualIndexSettings = indexSettingsBuilder.build(); diff --git a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodeService.java b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodeService.java index 51c27321dd1..177c67f2986 100644 --- a/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodeService.java +++ b/core/src/main/java/org/elasticsearch/cluster/node/DiscoveryNodeService.java @@ -20,9 +20,8 @@ package org.elasticsearch.cluster.node; import org.elasticsearch.Version; -import org.elasticsearch.common.Booleans; import org.elasticsearch.common.Randomness; -import org.elasticsearch.common.Strings; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Setting; @@ -57,7 +56,7 @@ public class DiscoveryNodeService extends AbstractComponent { public static String generateNodeId(Settings settings) { Random random = Randomness.get(settings, NODE_ID_SEED_SETTING); - return Strings.randomBase64UUID(random); + return UUIDs.randomBase64UUID(random); } public DiscoveryNodeService addCustomAttributeProvider(CustomAttributesProvider customAttributesProvider) { diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/AllocationId.java b/core/src/main/java/org/elasticsearch/cluster/routing/AllocationId.java index a7bd942a20c..100752289f7 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/AllocationId.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/AllocationId.java @@ -23,7 +23,7 @@ import org.elasticsearch.common.Nullable; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseFieldMatcher; import org.elasticsearch.common.ParseFieldMatcherSupplier; -import org.elasticsearch.common.Strings; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.ObjectParser; @@ -96,7 +96,7 @@ public class AllocationId implements ToXContent { * Creates a new allocation id for initializing allocation. */ public static AllocationId newInitializing() { - return new AllocationId(Strings.randomBase64UUID(), null); + return new AllocationId(UUIDs.randomBase64UUID(), null); } /** @@ -121,7 +121,7 @@ public class AllocationId implements ToXContent { */ public static AllocationId newRelocation(AllocationId allocationId) { assert allocationId.getRelocationId() == null; - return new AllocationId(allocationId.getId(), Strings.randomBase64UUID()); + return new AllocationId(allocationId.getId(), UUIDs.randomBase64UUID()); } /** diff --git a/core/src/main/java/org/elasticsearch/common/MacAddressProvider.java b/core/src/main/java/org/elasticsearch/common/MacAddressProvider.java index feb4707eac8..52ebc283f3c 100644 --- a/core/src/main/java/org/elasticsearch/common/MacAddressProvider.java +++ b/core/src/main/java/org/elasticsearch/common/MacAddressProvider.java @@ -19,18 +19,12 @@ package org.elasticsearch.common; -import org.elasticsearch.common.logging.ESLogger; -import org.elasticsearch.common.logging.Loggers; - import java.net.NetworkInterface; import java.net.SocketException; import java.util.Enumeration; - public class MacAddressProvider { - private static final ESLogger logger = Loggers.getLogger(MacAddressProvider.class); - private static byte[] getMacAddress() throws SocketException { Enumeration en = NetworkInterface.getNetworkInterfaces(); if (en != null) { @@ -66,12 +60,10 @@ public class MacAddressProvider { try { address = getMacAddress(); } catch (Throwable t) { - logger.warn("Unable to get mac address, will use a dummy address", t); // address will be set below } if (!isValidAddress(address)) { - logger.warn("Unable to get a valid mac address, will use a dummy address"); address = constructDummyMulticastAddress(); } diff --git a/core/src/main/java/org/elasticsearch/common/Strings.java b/core/src/main/java/org/elasticsearch/common/Strings.java index 2a55dea9f13..bd1ba64c747 100644 --- a/core/src/main/java/org/elasticsearch/common/Strings.java +++ b/core/src/main/java/org/elasticsearch/common/Strings.java @@ -60,9 +60,6 @@ public class Strings { private static final String CURRENT_PATH = "."; - private static final RandomBasedUUIDGenerator RANDOM_UUID_GENERATOR = new RandomBasedUUIDGenerator(); - private static final UUIDGenerator TIME_UUID_GENERATOR = new TimeBasedUUIDGenerator(); - public static void spaceify(int spaces, String from, StringBuilder to) throws Exception { try (BufferedReader reader = new BufferedReader(new FastStringReader(from))) { String line; @@ -1060,24 +1057,6 @@ public class Strings { data.length == 1 && ("_all".equals(data[0]) || "*".equals(data[0])); } - /** Returns a Base64 encoded version of a Version 4.0 compatible UUID as defined here: http://www.ietf.org/rfc/rfc4122.txt, using a - * private {@code SecureRandom} instance */ - public static String randomBase64UUID() { - return RANDOM_UUID_GENERATOR.getBase64UUID(); - } - - /** Returns a Base64 encoded version of a Version 4.0 compatible UUID as defined here: http://www.ietf.org/rfc/rfc4122.txt, using the - * provided {@code Random} instance */ - public static String randomBase64UUID(Random random) { - return RANDOM_UUID_GENERATOR.getBase64UUID(random); - } - - /** Generates a time-based UUID (similar to Flake IDs), which is preferred when generating an ID to be indexed into a Lucene index as - * primary key. The id is opaque and the implementation is free to change at any time! */ - public static String base64UUID() { - return TIME_UUID_GENERATOR.getBase64UUID(); - } - /** * Return a {@link String} that is the json representation of the provided * {@link ToXContent}. diff --git a/core/src/main/java/org/elasticsearch/common/TimeBasedUUIDGenerator.java b/core/src/main/java/org/elasticsearch/common/TimeBasedUUIDGenerator.java index b45d17e20d5..d1a22a17cda 100644 --- a/core/src/main/java/org/elasticsearch/common/TimeBasedUUIDGenerator.java +++ b/core/src/main/java/org/elasticsearch/common/TimeBasedUUIDGenerator.java @@ -35,10 +35,10 @@ class TimeBasedUUIDGenerator implements UUIDGenerator { // Used to ensure clock moves forward: private long lastTimestamp; - private static final byte[] secureMungedAddress = MacAddressProvider.getSecureMungedAddress(); + private static final byte[] SECURE_MUNGED_ADDRESS = MacAddressProvider.getSecureMungedAddress(); static { - assert secureMungedAddress.length == 6; + assert SECURE_MUNGED_ADDRESS.length == 6; } /** Puts the lower numberOfLongBytes from l into the array, starting index pos. */ @@ -73,12 +73,12 @@ class TimeBasedUUIDGenerator implements UUIDGenerator { putLong(uuidBytes, timestamp, 0, 6); // MAC address adds 6 bytes: - System.arraycopy(secureMungedAddress, 0, uuidBytes, 6, secureMungedAddress.length); + System.arraycopy(SECURE_MUNGED_ADDRESS, 0, uuidBytes, 6, SECURE_MUNGED_ADDRESS.length); // Sequence number adds 3 bytes: putLong(uuidBytes, sequenceId, 12, 3); - assert 9 + secureMungedAddress.length == uuidBytes.length; + assert 9 + SECURE_MUNGED_ADDRESS.length == uuidBytes.length; byte[] encoded; try { diff --git a/core/src/main/java/org/elasticsearch/common/UUIDs.java b/core/src/main/java/org/elasticsearch/common/UUIDs.java new file mode 100644 index 00000000000..ca3c7cb222e --- /dev/null +++ b/core/src/main/java/org/elasticsearch/common/UUIDs.java @@ -0,0 +1,47 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.common; + +import java.util.Random; + +public class UUIDs { + + private static final RandomBasedUUIDGenerator RANDOM_UUID_GENERATOR = new RandomBasedUUIDGenerator(); + private static final UUIDGenerator TIME_UUID_GENERATOR = new TimeBasedUUIDGenerator(); + + /** Generates a time-based UUID (similar to Flake IDs), which is preferred when generating an ID to be indexed into a Lucene index as + * primary key. The id is opaque and the implementation is free to change at any time! */ + public static String base64UUID() { + return TIME_UUID_GENERATOR.getBase64UUID(); + } + + /** Returns a Base64 encoded version of a Version 4.0 compatible UUID as defined here: http://www.ietf.org/rfc/rfc4122.txt, using the + * provided {@code Random} instance */ + public static String randomBase64UUID(Random random) { + return RANDOM_UUID_GENERATOR.getBase64UUID(random); + } + + /** Returns a Base64 encoded version of a Version 4.0 compatible UUID as defined here: http://www.ietf.org/rfc/rfc4122.txt, using a + * private {@code SecureRandom} instance */ + public static String randomBase64UUID() { + return RANDOM_UUID_GENERATOR.getBase64UUID(); + } + +} diff --git a/core/src/main/java/org/elasticsearch/index/store/Store.java b/core/src/main/java/org/elasticsearch/index/store/Store.java index 39c4bcfb3c1..5bcfc422e81 100644 --- a/core/src/main/java/org/elasticsearch/index/store/Store.java +++ b/core/src/main/java/org/elasticsearch/index/store/Store.java @@ -47,7 +47,7 @@ import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.Version; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; -import org.elasticsearch.common.Strings; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.Streams; import org.elasticsearch.common.io.stream.BytesStreamOutput; @@ -1363,7 +1363,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref public void markStoreCorrupted(IOException exception) throws IOException { ensureOpen(); if (!isMarkedCorrupted()) { - String uuid = CORRUPTED + Strings.randomBase64UUID(); + String uuid = CORRUPTED + UUIDs.randomBase64UUID(); try (IndexOutput output = this.directory().createOutput(uuid, IOContext.DEFAULT)) { CodecUtil.writeHeader(output, CODEC, VERSION); BytesStreamOutput out = new BytesStreamOutput(); diff --git a/core/src/main/java/org/elasticsearch/index/translog/Translog.java b/core/src/main/java/org/elasticsearch/index/translog/Translog.java index 781bcc99c21..93f9db67b35 100644 --- a/core/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/core/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -26,7 +26,7 @@ import org.apache.lucene.util.Accountable; import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.RamUsageEstimator; import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.common.Strings; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.bytes.ReleasablePagedBytesReference; @@ -148,7 +148,7 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC super(config.getShardId(), config.getIndexSettings()); this.config = config; if (translogGeneration == null || translogGeneration.translogUUID == null) { // legacy case - translogUUID = Strings.randomBase64UUID(); + translogUUID = UUIDs.randomBase64UUID(); } else { translogUUID = translogGeneration.translogUUID; } diff --git a/core/src/main/java/org/elasticsearch/indices/flush/SyncedFlushService.java b/core/src/main/java/org/elasticsearch/indices/flush/SyncedFlushService.java index f572b3b5002..f3cb76199dc 100644 --- a/core/src/main/java/org/elasticsearch/indices/flush/SyncedFlushService.java +++ b/core/src/main/java/org/elasticsearch/indices/flush/SyncedFlushService.java @@ -31,7 +31,7 @@ import org.elasticsearch.cluster.routing.IndexRoutingTable; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.common.Strings; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.io.stream.StreamInput; @@ -212,7 +212,7 @@ public class SyncedFlushService extends AbstractComponent implements IndexEventL actionListener.onResponse(new ShardsSyncedFlushResult(shardId, totalShards, "[" + inflight + "] ongoing operations on primary")); } else { // 3. now send the sync request to all the shards - String syncId = Strings.base64UUID(); + String syncId = UUIDs.base64UUID(); sendSyncRequests(syncId, activeShards, state, commitIds, shardId, totalShards, actionListener); } } diff --git a/core/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java b/core/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java index 5d423552a56..ef0cab5c156 100644 --- a/core/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/core/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -27,6 +27,7 @@ import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.metadata.SnapshotId; import org.elasticsearch.common.ParseFieldMatcher; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.blobstore.BlobContainer; import org.elasticsearch.common.blobstore.BlobMetaData; import org.elasticsearch.common.blobstore.BlobPath; @@ -634,7 +635,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent { // nothing is going to be discovered, since no master will be elected sb.put(DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.getKey(), 0); if (sb.get("cluster.name") == null) { - sb.put("cluster.name", "tribe_" + Strings.randomBase64UUID()); // make sure it won't join other tribe nodes in the same JVM + sb.put("cluster.name", "tribe_" + UUIDs.randomBase64UUID()); // make sure it won't join other tribe nodes in the same JVM } sb.put(TransportMasterNodeReadAction.FORCE_LOCAL_SETTING.getKey(), true); return sb.build(); diff --git a/core/src/test/java/org/elasticsearch/action/admin/indices/shards/IndicesShardStoreResponseTests.java b/core/src/test/java/org/elasticsearch/action/admin/indices/shards/IndicesShardStoreResponseTests.java index e100bcae4ac..3c12d7d9b10 100644 --- a/core/src/test/java/org/elasticsearch/action/admin/indices/shards/IndicesShardStoreResponseTests.java +++ b/core/src/test/java/org/elasticsearch/action/admin/indices/shards/IndicesShardStoreResponseTests.java @@ -22,7 +22,7 @@ package org.elasticsearch.action.admin.indices.shards; import org.apache.lucene.util.CollectionUtil; import org.elasticsearch.Version; import org.elasticsearch.cluster.node.DiscoveryNode; -import org.elasticsearch.common.Strings; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.ImmutableOpenIntMap; import org.elasticsearch.common.collect.ImmutableOpenMap; @@ -57,8 +57,8 @@ public class IndicesShardStoreResponseTests extends ESTestCase { DiscoveryNode node2 = new DiscoveryNode("node2", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT); List storeStatusList = new ArrayList<>(); storeStatusList.add(new IndicesShardStoresResponse.StoreStatus(node1, 3, null, IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, null)); - storeStatusList.add(new IndicesShardStoresResponse.StoreStatus(node2, ShardStateMetaData.NO_VERSION, Strings.randomBase64UUID(), IndicesShardStoresResponse.StoreStatus.AllocationStatus.REPLICA, null)); - storeStatusList.add(new IndicesShardStoresResponse.StoreStatus(node1, ShardStateMetaData.NO_VERSION, Strings.randomBase64UUID(), IndicesShardStoresResponse.StoreStatus.AllocationStatus.UNUSED, new IOException("corrupted"))); + storeStatusList.add(new IndicesShardStoresResponse.StoreStatus(node2, ShardStateMetaData.NO_VERSION, UUIDs.randomBase64UUID(), IndicesShardStoresResponse.StoreStatus.AllocationStatus.REPLICA, null)); + storeStatusList.add(new IndicesShardStoresResponse.StoreStatus(node1, ShardStateMetaData.NO_VERSION, UUIDs.randomBase64UUID(), IndicesShardStoresResponse.StoreStatus.AllocationStatus.UNUSED, new IOException("corrupted"))); storeStatuses.put(0, storeStatusList); storeStatuses.put(1, storeStatusList); ImmutableOpenIntMap> storesMap = storeStatuses.build(); @@ -124,14 +124,14 @@ public class IndicesShardStoreResponseTests extends ESTestCase { public void testStoreStatusOrdering() throws Exception { DiscoveryNode node1 = new DiscoveryNode("node1", DummyTransportAddress.INSTANCE, emptyMap(), emptySet(), Version.CURRENT); List orderedStoreStatuses = new ArrayList<>(); - orderedStoreStatuses.add(new IndicesShardStoresResponse.StoreStatus(node1, ShardStateMetaData.NO_VERSION, Strings.randomBase64UUID(), IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, null)); - orderedStoreStatuses.add(new IndicesShardStoresResponse.StoreStatus(node1, ShardStateMetaData.NO_VERSION, Strings.randomBase64UUID(), IndicesShardStoresResponse.StoreStatus.AllocationStatus.REPLICA, null)); - orderedStoreStatuses.add(new IndicesShardStoresResponse.StoreStatus(node1, ShardStateMetaData.NO_VERSION, Strings.randomBase64UUID(), IndicesShardStoresResponse.StoreStatus.AllocationStatus.UNUSED, null)); + orderedStoreStatuses.add(new IndicesShardStoresResponse.StoreStatus(node1, ShardStateMetaData.NO_VERSION, UUIDs.randomBase64UUID(), IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, null)); + orderedStoreStatuses.add(new IndicesShardStoresResponse.StoreStatus(node1, ShardStateMetaData.NO_VERSION, UUIDs.randomBase64UUID(), IndicesShardStoresResponse.StoreStatus.AllocationStatus.REPLICA, null)); + orderedStoreStatuses.add(new IndicesShardStoresResponse.StoreStatus(node1, ShardStateMetaData.NO_VERSION, UUIDs.randomBase64UUID(), IndicesShardStoresResponse.StoreStatus.AllocationStatus.UNUSED, null)); orderedStoreStatuses.add(new IndicesShardStoresResponse.StoreStatus(node1, 2, null, IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, null)); orderedStoreStatuses.add(new IndicesShardStoresResponse.StoreStatus(node1, 1, null, IndicesShardStoresResponse.StoreStatus.AllocationStatus.PRIMARY, null)); orderedStoreStatuses.add(new IndicesShardStoresResponse.StoreStatus(node1, 1, null, IndicesShardStoresResponse.StoreStatus.AllocationStatus.REPLICA, null)); orderedStoreStatuses.add(new IndicesShardStoresResponse.StoreStatus(node1, 1, null, IndicesShardStoresResponse.StoreStatus.AllocationStatus.UNUSED, null)); - orderedStoreStatuses.add(new IndicesShardStoresResponse.StoreStatus(node1, ShardStateMetaData.NO_VERSION, Strings.randomBase64UUID(), IndicesShardStoresResponse.StoreStatus.AllocationStatus.REPLICA, new IOException("corrupted"))); + orderedStoreStatuses.add(new IndicesShardStoresResponse.StoreStatus(node1, ShardStateMetaData.NO_VERSION, UUIDs.randomBase64UUID(), IndicesShardStoresResponse.StoreStatus.AllocationStatus.REPLICA, new IOException("corrupted"))); orderedStoreStatuses.add(new IndicesShardStoresResponse.StoreStatus(node1, 3, null, IndicesShardStoresResponse.StoreStatus.AllocationStatus.REPLICA, new IOException("corrupted"))); List storeStatuses = new ArrayList<>(orderedStoreStatuses); diff --git a/core/src/test/java/org/elasticsearch/cluster/ClusterChangedEventTests.java b/core/src/test/java/org/elasticsearch/cluster/ClusterChangedEventTests.java index e7e73334d26..e488c1b1bcc 100644 --- a/core/src/test/java/org/elasticsearch/cluster/ClusterChangedEventTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/ClusterChangedEventTests.java @@ -26,7 +26,7 @@ import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.RoutingTable; -import org.elasticsearch.common.Strings; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.DummyTransportAddress; import org.elasticsearch.test.ESTestCase; @@ -49,7 +49,7 @@ public class ClusterChangedEventTests extends ESTestCase { private static final ClusterName TEST_CLUSTER_NAME = new ClusterName("test"); private static final int INDICES_CHANGE_NUM_TESTS = 5; private static final String NODE_ID_PREFIX = "node_"; - private static final String INITIAL_CLUSTER_ID = Strings.randomBase64UUID(); + private static final String INITIAL_CLUSTER_ID = UUIDs.randomBase64UUID(); // the initial indices which every cluster state test starts out with private static final List initialIndices = Arrays.asList("idx1", "idx2", "idx3"); // index settings @@ -249,12 +249,12 @@ public class ClusterChangedEventTests extends ESTestCase { final List addedIndices, final List deletedIndices, final int numNodesToRemove) { final ClusterState.Builder builder = ClusterState.builder(previousState); - builder.stateUUID(Strings.randomBase64UUID()); + builder.stateUUID(UUIDs.randomBase64UUID()); final MetaData.Builder metaBuilder = MetaData.builder(previousState.metaData()); if (changeClusterUUID || addedIndices.size() > 0 || deletedIndices.size() > 0) { // there is some change in metadata cluster state if (changeClusterUUID) { - metaBuilder.clusterUUID(Strings.randomBase64UUID()); + metaBuilder.clusterUUID(UUIDs.randomBase64UUID()); } for (String index : addedIndices) { metaBuilder.put(createIndexMetadata(index), true); diff --git a/core/src/test/java/org/elasticsearch/cluster/ClusterStateDiffIT.java b/core/src/test/java/org/elasticsearch/cluster/ClusterStateDiffIT.java index 84cb47f4b7a..8a634474ae1 100644 --- a/core/src/test/java/org/elasticsearch/cluster/ClusterStateDiffIT.java +++ b/core/src/test/java/org/elasticsearch/cluster/ClusterStateDiffIT.java @@ -38,7 +38,7 @@ import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.TestShardRouting; import org.elasticsearch.cluster.routing.UnassignedInfo; -import org.elasticsearch.common.Strings; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; @@ -518,7 +518,7 @@ public class ClusterStateDiffIT extends ESIntegTestCase { } break; case 2: - builder.settings(Settings.builder().put(part.getSettings()).put(IndexMetaData.SETTING_INDEX_UUID, Strings.randomBase64UUID())); + builder.settings(Settings.builder().put(part.getSettings()).put(IndexMetaData.SETTING_INDEX_UUID, UUIDs.randomBase64UUID())); break; default: throw new IllegalArgumentException("Shouldn't be here"); @@ -672,6 +672,6 @@ public class ClusterStateDiffIT extends ESIntegTestCase { * Generates a random name that starts with the given prefix */ private String randomName(String prefix) { - return prefix + Strings.randomBase64UUID(random()); + return prefix + UUIDs.randomBase64UUID(random()); } } diff --git a/core/src/test/java/org/elasticsearch/cluster/SimpleClusterStateIT.java b/core/src/test/java/org/elasticsearch/cluster/SimpleClusterStateIT.java index 2de99d6296c..5bb656499b3 100644 --- a/core/src/test/java/org/elasticsearch/cluster/SimpleClusterStateIT.java +++ b/core/src/test/java/org/elasticsearch/cluster/SimpleClusterStateIT.java @@ -26,7 +26,7 @@ import org.elasticsearch.client.Client; import org.elasticsearch.client.Requests; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MappingMetaData; -import org.elasticsearch.common.Strings; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; @@ -133,7 +133,7 @@ public class SimpleClusterStateIT extends ESIntegTestCase { int counter = 0; int numberOfFields = 0; while (true) { - mapping.startObject(Strings.randomBase64UUID()).field("type", "text").endObject(); + mapping.startObject(UUIDs.randomBase64UUID()).field("type", "text").endObject(); counter += 10; // each field is about 10 bytes, assuming compression in place numberOfFields++; if (counter > estimatedBytesSize) { diff --git a/core/src/test/java/org/elasticsearch/common/util/IndexFolderUpgraderTests.java b/core/src/test/java/org/elasticsearch/common/util/IndexFolderUpgraderTests.java index b158b961d9a..26d6af1cd5f 100644 --- a/core/src/test/java/org/elasticsearch/common/util/IndexFolderUpgraderTests.java +++ b/core/src/test/java/org/elasticsearch/common/util/IndexFolderUpgraderTests.java @@ -26,7 +26,7 @@ import org.elasticsearch.Version; import org.elasticsearch.bwcompat.OldIndexBackwardsCompatibilityIT; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.AllocationId; -import org.elasticsearch.common.Strings; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.io.FileSystemUtils; import org.elasticsearch.common.settings.Settings; @@ -70,7 +70,7 @@ public class IndexFolderUpgraderTests extends ESTestCase { .put(NodeEnvironment.ADD_NODE_ID_TO_CUSTOM_PATH.getKey(), randomBoolean()) .put(Environment.PATH_SHARED_DATA_SETTING.getKey(), customPath.toAbsolutePath().toString()).build(); try (NodeEnvironment nodeEnv = newNodeEnvironment(nodeSettings)) { - final Index index = new Index(randomAsciiOfLength(10), Strings.randomBase64UUID()); + final Index index = new Index(randomAsciiOfLength(10), UUIDs.randomBase64UUID()); Settings settings = Settings.builder() .put(nodeSettings) .put(IndexMetaData.SETTING_INDEX_UUID, index.getUUID()) @@ -99,7 +99,7 @@ public class IndexFolderUpgraderTests extends ESTestCase { .put(NodeEnvironment.ADD_NODE_ID_TO_CUSTOM_PATH.getKey(), randomBoolean()) .put(Environment.PATH_SHARED_DATA_SETTING.getKey(), customPath.toAbsolutePath().toString()).build(); try (NodeEnvironment nodeEnv = newNodeEnvironment(nodeSettings)) { - final Index index = new Index(randomAsciiOfLength(10), Strings.randomBase64UUID()); + final Index index = new Index(randomAsciiOfLength(10), UUIDs.randomBase64UUID()); Settings settings = Settings.builder() .put(nodeSettings) .put(IndexMetaData.SETTING_INDEX_UUID, index.getUUID()) @@ -138,7 +138,7 @@ public class IndexFolderUpgraderTests extends ESTestCase { final Settings nodeSettings = Settings.builder() .put(NodeEnvironment.ADD_NODE_ID_TO_CUSTOM_PATH.getKey(), randomBoolean()).build(); try (NodeEnvironment nodeEnv = newNodeEnvironment(nodeSettings)) { - final Index index = new Index(randomAsciiOfLength(10), Strings.randomBase64UUID()); + final Index index = new Index(randomAsciiOfLength(10), UUIDs.randomBase64UUID()); Settings settings = Settings.builder() .put(nodeSettings) .put(IndexMetaData.SETTING_INDEX_UUID, index.getUUID()) @@ -163,7 +163,7 @@ public class IndexFolderUpgraderTests extends ESTestCase { try (NodeEnvironment nodeEnv = newNodeEnvironment(nodeSettings)) { Map> indexSettingsMap = new HashMap<>(); for (int i = 0; i < randomIntBetween(2, 5); i++) { - final Index index = new Index(randomAsciiOfLength(10), Strings.randomBase64UUID()); + final Index index = new Index(randomAsciiOfLength(10), UUIDs.randomBase64UUID()); Settings settings = Settings.builder() .put(nodeSettings) .put(IndexMetaData.SETTING_INDEX_UUID, index.getUUID()) @@ -247,7 +247,7 @@ public class IndexFolderUpgraderTests extends ESTestCase { } public void testNeedsUpgrade() throws IOException { - final Index index = new Index("foo", Strings.randomBase64UUID()); + final Index index = new Index("foo", UUIDs.randomBase64UUID()); IndexMetaData indexState = IndexMetaData.builder(index.getName()) .settings(Settings.builder() .put(IndexMetaData.SETTING_INDEX_UUID, index.getUUID()) diff --git a/core/src/test/java/org/elasticsearch/gateway/PrimaryShardAllocatorTests.java b/core/src/test/java/org/elasticsearch/gateway/PrimaryShardAllocatorTests.java index 1c63cda4174..b62d2fe3342 100644 --- a/core/src/test/java/org/elasticsearch/gateway/PrimaryShardAllocatorTests.java +++ b/core/src/test/java/org/elasticsearch/gateway/PrimaryShardAllocatorTests.java @@ -35,7 +35,7 @@ import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders; import org.elasticsearch.common.Nullable; -import org.elasticsearch.common.Strings; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.index.shard.ShardId; @@ -184,8 +184,8 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { * Tests that when there was a node that previously had the primary, it will be allocated to that same node again. */ public void testPreferAllocatingPreviousPrimary() { - String primaryAllocId = Strings.randomBase64UUID(); - String replicaAllocId = Strings.randomBase64UUID(); + String primaryAllocId = UUIDs.randomBase64UUID(); + String replicaAllocId = UUIDs.randomBase64UUID(); RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), false, randomFrom(Version.V_2_0_0, Version.CURRENT), primaryAllocId, replicaAllocId); boolean node1HasPrimaryShard = randomBoolean(); testAllocator.addData(node1, ShardStateMetaData.NO_VERSION, node1HasPrimaryShard ? primaryAllocId : replicaAllocId, node1HasPrimaryShard); diff --git a/core/src/test/java/org/elasticsearch/index/IndexTests.java b/core/src/test/java/org/elasticsearch/index/IndexTests.java index 6ce38c6acba..daf5b4a9ec0 100644 --- a/core/src/test/java/org/elasticsearch/index/IndexTests.java +++ b/core/src/test/java/org/elasticsearch/index/IndexTests.java @@ -20,7 +20,7 @@ package org.elasticsearch.index; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.common.Strings; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.test.ESTestCase; import static org.apache.lucene.util.TestUtil.randomSimpleString; @@ -33,7 +33,7 @@ public class IndexTests extends ESTestCase { assertEquals("[name]", new Index("name", ClusterState.UNKNOWN_UUID).toString()); Index random = new Index(randomSimpleString(random(), 1, 100), - usually() ? Strings.randomBase64UUID(random()) : ClusterState.UNKNOWN_UUID); + usually() ? UUIDs.randomBase64UUID(random()) : ClusterState.UNKNOWN_UUID); assertThat(random.toString(), containsString(random.getName())); if (ClusterState.UNKNOWN_UUID.equals(random.getUUID())) { assertThat(random.toString(), not(containsString(random.getUUID()))); diff --git a/core/src/test/java/org/elasticsearch/index/store/StoreTests.java b/core/src/test/java/org/elasticsearch/index/store/StoreTests.java index fbb671e36bd..8f64390dbd3 100644 --- a/core/src/test/java/org/elasticsearch/index/store/StoreTests.java +++ b/core/src/test/java/org/elasticsearch/index/store/StoreTests.java @@ -20,10 +20,6 @@ package org.elasticsearch.index.store; import org.apache.lucene.analysis.MockAnalyzer; import org.apache.lucene.codecs.CodecUtil; -import org.apache.lucene.codecs.FilterCodec; -import org.apache.lucene.codecs.SegmentInfoFormat; -import org.apache.lucene.codecs.lucene50.Lucene50SegmentInfoFormat; -import org.apache.lucene.codecs.lucene54.Lucene54Codec; import org.apache.lucene.document.Document; import org.apache.lucene.document.Field; import org.apache.lucene.document.SortedDocValuesField; @@ -40,7 +36,6 @@ import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.index.KeepOnlyLastCommitDeletionPolicy; import org.apache.lucene.index.NoDeletionPolicy; import org.apache.lucene.index.NoMergePolicy; -import org.apache.lucene.index.SegmentInfo; import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.index.SnapshotDeletionPolicy; import org.apache.lucene.index.Term; @@ -59,7 +54,7 @@ import org.apache.lucene.util.TestUtil; import org.apache.lucene.util.Version; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.cluster.metadata.IndexMetaData; -import org.elasticsearch.common.Strings; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.io.stream.InputStreamStreamInput; import org.elasticsearch.common.io.stream.OutputStreamStreamOutput; import org.elasticsearch.common.lucene.Lucene; @@ -91,11 +86,8 @@ import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Random; -import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; -import java.util.zip.Adler32; -import static java.util.Collections.emptyMap; import static java.util.Collections.unmodifiableMap; import static org.elasticsearch.test.VersionUtils.randomVersion; import static org.hamcrest.Matchers.empty; @@ -1080,7 +1072,7 @@ public class StoreTests extends ESTestCase { Store store = new Store(shardId, INDEX_SETTINGS, directoryService, new DummyShardLock(shardId)); CorruptIndexException exception = new CorruptIndexException("foo", "bar"); - String uuid = Store.CORRUPTED + Strings.randomBase64UUID(); + String uuid = Store.CORRUPTED + UUIDs.randomBase64UUID(); try (IndexOutput output = dir.createOutput(uuid, IOContext.DEFAULT)) { CodecUtil.writeHeader(output, Store.CODEC, Store.VERSION_STACK_TRACE); output.writeString(ExceptionsHelper.detailedMessage(exception, true, 0)); diff --git a/core/src/test/java/org/elasticsearch/indices/flush/SyncedFlushSingleNodeTests.java b/core/src/test/java/org/elasticsearch/indices/flush/SyncedFlushSingleNodeTests.java index da1bb7ae303..5932434438c 100644 --- a/core/src/test/java/org/elasticsearch/indices/flush/SyncedFlushSingleNodeTests.java +++ b/core/src/test/java/org/elasticsearch/indices/flush/SyncedFlushSingleNodeTests.java @@ -22,7 +22,7 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.common.Strings; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.engine.Engine; @@ -54,7 +54,7 @@ public class SyncedFlushSingleNodeTests extends ESSingleNodeTestCase { Map commitIds = SyncedFlushUtil.sendPreSyncRequests(flushService, activeShards, state, shardId); assertEquals("exactly one commit id", 1, commitIds.size()); client().prepareIndex("test", "test", "2").setSource("{}").get(); - String syncId = Strings.base64UUID(); + String syncId = UUIDs.base64UUID(); SyncedFlushUtil.LatchedListener listener = new SyncedFlushUtil.LatchedListener<>(); flushService.sendSyncRequests(syncId, activeShards, state, commitIds, shardId, shardRoutingTable.size(), listener); listener.latch.await(); @@ -174,7 +174,7 @@ public class SyncedFlushSingleNodeTests extends ESSingleNodeTestCase { client().prepareIndex("test", "test", "2").setSource("{}").get(); } client().admin().indices().prepareFlush("test").setForce(true).get(); - String syncId = Strings.base64UUID(); + String syncId = UUIDs.base64UUID(); final SyncedFlushUtil.LatchedListener listener = new SyncedFlushUtil.LatchedListener(); flushService.sendSyncRequests(syncId, activeShards, state, commitIds, shardId, shardRoutingTable.size(), listener); listener.latch.await(); @@ -204,7 +204,7 @@ public class SyncedFlushSingleNodeTests extends ESSingleNodeTestCase { Map commitIds = SyncedFlushUtil.sendPreSyncRequests(flushService, activeShards, state, shardId); assertEquals("exactly one commit id", 1, commitIds.size()); commitIds.clear(); // wipe it... - String syncId = Strings.base64UUID(); + String syncId = UUIDs.base64UUID(); SyncedFlushUtil.LatchedListener listener = new SyncedFlushUtil.LatchedListener(); flushService.sendSyncRequests(syncId, activeShards, state, commitIds, shardId, shardRoutingTable.size(), listener); listener.latch.await(); diff --git a/core/src/test/java/org/elasticsearch/search/searchafter/SearchAfterIT.java b/core/src/test/java/org/elasticsearch/search/searchafter/SearchAfterIT.java index a115567059d..13c99944959 100644 --- a/core/src/test/java/org/elasticsearch/search/searchafter/SearchAfterIT.java +++ b/core/src/test/java/org/elasticsearch/search/searchafter/SearchAfterIT.java @@ -24,7 +24,7 @@ import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.common.Strings; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.text.Text; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.search.SearchContextException; @@ -193,7 +193,7 @@ public class SearchAfterIT extends ESIntegTestCase { break; } } - values.add(new Text(Strings.randomBase64UUID())); + values.add(new Text(UUIDs.randomBase64UUID())); documents.add(values); } int reqSize = randomInt(NUM_DOCS-1); diff --git a/core/src/test/java/org/elasticsearch/tribe/TribeIT.java b/core/src/test/java/org/elasticsearch/tribe/TribeIT.java index 014366d492c..52827491fe4 100644 --- a/core/src/test/java/org/elasticsearch/tribe/TribeIT.java +++ b/core/src/test/java/org/elasticsearch/tribe/TribeIT.java @@ -31,7 +31,7 @@ import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.common.Priority; -import org.elasticsearch.common.Strings; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.network.NetworkModule; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; @@ -95,7 +95,7 @@ public class TribeIT extends ESIntegTestCase { }; cluster2 = new InternalTestCluster(InternalTestCluster.configuredNodeMode(), randomLong(), createTempDir(), 2, 2, - Strings.randomBase64UUID(random()), nodeConfigurationSource, 0, false, SECOND_CLUSTER_NODE_PREFIX, Collections.emptyList(), Function.identity()); + UUIDs.randomBase64UUID(random()), nodeConfigurationSource, 0, false, SECOND_CLUSTER_NODE_PREFIX, Collections.emptyList(), Function.identity()); cluster2.beforeTest(random(), 0.1); cluster2.ensureAtLeastNumDataNodes(2);