From ff6a8851e64b09f9fd21a960ed19eb6eacf06e2f Mon Sep 17 00:00:00 2001 From: Ashish Date: Thu, 3 Aug 2023 20:02:43 +0530 Subject: [PATCH 01/19] Fix flaky test testStatsOnShardUnassigned in RemoteStoreStatsIT (#9057) --------- Signed-off-by: Ashish Singh --- .../opensearch/remotestore/RemoteStoreStatsIT.java | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreStatsIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreStatsIT.java index 1c7f14701b3e7..bd546a01b0b88 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreStatsIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreStatsIT.java @@ -9,6 +9,7 @@ package org.opensearch.remotestore; import org.junit.Before; +import org.opensearch.action.admin.cluster.health.ClusterHealthResponse; import org.opensearch.action.admin.cluster.remotestore.restore.RestoreRemoteStoreRequest; import org.opensearch.action.admin.cluster.remotestore.stats.RemoteStoreStats; import org.opensearch.action.admin.cluster.remotestore.stats.RemoteStoreStatsRequestBuilder; @@ -22,6 +23,7 @@ import org.opensearch.common.settings.Settings; import org.opensearch.index.IndexSettings; import org.opensearch.index.remote.RemoteSegmentTransferTracker; +import org.opensearch.test.InternalTestCluster; import org.opensearch.test.OpenSearchIntegTestCase; import java.io.IOException; @@ -443,12 +445,19 @@ public void testStatsOnShardUnassigned() throws IOException { createIndex(INDEX_NAME, remoteStoreIndexSettings(2, 1)); ensureGreen(INDEX_NAME); indexDocs(); - int dataNodeCountBeforeStop = client().admin().cluster().prepareHealth().get().getNumberOfDataNodes(); - internalCluster().stopRandomDataNode(); + ClusterHealthResponse clusterHealthResponse = client().admin().cluster().prepareHealth().get(); + int dataNodeCountBeforeStop = clusterHealthResponse.getNumberOfDataNodes(); + int nodeCount = clusterHealthResponse.getNumberOfNodes(); + String nodeToBeStopped = randomBoolean() ? primaryNodeName(INDEX_NAME) : replicaNodeName(INDEX_NAME); + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(nodeToBeStopped)); + ensureYellowAndNoInitializingShards(INDEX_NAME); + ensureStableCluster(nodeCount - 1); RemoteStoreStatsResponse response = client().admin().cluster().prepareRemoteStoreStats(INDEX_NAME, "0").get(); int dataNodeCountAfterStop = client().admin().cluster().prepareHealth().get().getNumberOfDataNodes(); assertEquals(dataNodeCountBeforeStop, response.getTotalShards()); assertEquals(dataNodeCountAfterStop, response.getSuccessfulShards()); + // Indexing docs to ensure that the primary has started + indexSingleDoc(INDEX_NAME); } public void testStatsOnRemoteStoreRestore() throws IOException { From c3acf47b4d643c3a3ab86dc3b07fe722ac6e4982 Mon Sep 17 00:00:00 2001 From: Marc Handalian Date: Thu, 3 Aug 2023 09:47:30 -0700 Subject: [PATCH 02/19] Fix test testDropPrimaryDuringReplication and clean up ReplicationCheckpoint validation (#8889) * Fix test testDropPrimaryDuringReplication and clean up ReplicationCheckpoint validation. This test is now occasionally failing with replicas having 0 documents. This occurs in a couple of ways: 1. After dropping the old primary the new primary is not publishing a checkpoint to replicas unless it indexes docs from translog after flipping to primary mode. If there is nothing to index, it will not publish a checkpoint, but the other replica could have never sync'd with the original primary and be left out of date. - This PR fixes this by force publishing a checkpoint after the new primary flips to primary mode. 2. The replica receives a checkpoint post failover and cancels its sync with the former primary that is still active, recognizing a primary term bump. However this cancellation is async and immediately starting a new replication event could fail as its still replicating. - This PR fixes this by attempting to process the latest received checkpoint on failure, if the shard is not failed and still behind. This PR also introduces a few changes to ensure the accuracy of the ReplicationCheckpoint tracked on primary & replicas. - Ensure the checkpoint stored in SegmentReplicationTarget is the checkpoint passed from the primary and not locally computed. This ensures checks for primary term are accurate and not using a locally compued operationPrimaryTerm. - Introduces a refresh listener for both primary & replica to update the ReplicationCheckpoint and store it in replicationTracker post refresh rather than redundantly computing when accessed. - Removes unnecessary onCheckpointPublished method used to start replication timers manually. This will happen automatically on primaries once its local cp is updated. Signed-off-by: Marc Handalian * Handle NoSuchFileException when attempting to delete decref'd files. To avoid divergent logic with remote store, we always incref/decref the segmentinfos.files(true) which includes the segments_n file. Decref to 0 will attempt to delete the file from the store and its possible this _n file does not yet exist. This change will ignore if we get a noSuchFile while attempting to delete. Signed-off-by: Marc Handalian * Add more unit tests. Signed-off-by: Marc Handalian * Clean up IndexShardTests.testCheckpointReffreshListenerWithNull Signed-off-by: Marc Handalian * Remove unnecessary catch for NoSuchFileException. Signed-off-by: Marc Handalian * Add another test for non segrep. Signed-off-by: Marc Handalian * PR Feedback. Signed-off-by: Marc Handalian * re-compute replication checkpoint on primary promotion. Signed-off-by: Marc Handalian --------- Signed-off-by: Marc Handalian --- .../replication/SegmentReplicationIT.java | 10 +- .../index/engine/NRTReplicationEngine.java | 15 +++ .../opensearch/index/shard/IndexShard.java | 69 +++++++---- .../shard/RemoteStoreRefreshListener.java | 5 +- .../replication/SegmentReplicationTarget.java | 20 +++- .../SegmentReplicationTargetService.java | 107 +++++++++++------- ...SegmentReplicationCheckpointPublisher.java | 1 - .../engine/NRTReplicationEngineTests.java | 72 ++++++++++++ .../index/shard/IndexShardTests.java | 51 +-------- .../SegmentReplicationIndexShardTests.java | 79 ++++++++++++- ...licationWithNodeToNodeIndexShardTests.java | 16 ++- .../SegmentReplicationTargetServiceTests.java | 90 +++++++++------ .../SegmentReplicationTargetTests.java | 14 +-- .../recovery/ReplicationCollectionTests.java | 2 + .../index/shard/IndexShardTestCase.java | 60 +++++++++- 15 files changed, 435 insertions(+), 176 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationIT.java index 08186bf3f9362..72b6a0296e3bb 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationIT.java @@ -44,6 +44,8 @@ import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.ShardRoutingState; import org.opensearch.cluster.routing.allocation.command.CancelAllocationCommand; +import org.opensearch.common.collect.Tuple; +import org.opensearch.common.concurrent.GatedCloseable; import org.opensearch.core.common.io.stream.NamedWriteableRegistry; import org.opensearch.common.lucene.index.OpenSearchDirectoryReader; import org.opensearch.common.settings.Settings; @@ -60,6 +62,7 @@ import org.opensearch.index.shard.IndexShard; import org.opensearch.core.index.shard.ShardId; import org.opensearch.indices.recovery.FileChunkRequest; +import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.search.SearchService; import org.opensearch.search.builder.PointInTimeBuilder; @@ -983,8 +986,11 @@ public void testScrollCreatedOnReplica() throws Exception { ) ); final IndexShard replicaShard = getIndexShard(replica, INDEX_NAME); - final SegmentInfos segmentInfos = replicaShard.getLatestSegmentInfosAndCheckpoint().v1().get(); - final Collection snapshottedSegments = segmentInfos.files(false); + final Tuple, ReplicationCheckpoint> tuple = replicaShard.getLatestSegmentInfosAndCheckpoint(); + final Collection snapshottedSegments; + try (final GatedCloseable closeable = tuple.v1()) { + snapshottedSegments = closeable.get().files(false); + } // opens a scrolled query before a flush is called. // this is for testing scroll segment consistency between refresh and flush SearchResponse searchResponse = client(replica).prepareSearch() diff --git a/server/src/main/java/org/opensearch/index/engine/NRTReplicationEngine.java b/server/src/main/java/org/opensearch/index/engine/NRTReplicationEngine.java index e852658d7b3ba..6b09b8d86dc6c 100644 --- a/server/src/main/java/org/opensearch/index/engine/NRTReplicationEngine.java +++ b/server/src/main/java/org/opensearch/index/engine/NRTReplicationEngine.java @@ -34,6 +34,7 @@ import java.io.Closeable; import java.io.IOException; import java.util.Arrays; +import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Objects; @@ -445,6 +446,20 @@ protected SegmentInfos getLatestSegmentInfos() { return readerManager.getSegmentInfos(); } + @Override + public synchronized GatedCloseable getSegmentInfosSnapshot() { + // get reference to latest infos + final SegmentInfos latestSegmentInfos = getLatestSegmentInfos(); + // incref all files + try { + final Collection files = latestSegmentInfos.files(false); + store.incRefFileDeleter(files); + return new GatedCloseable<>(latestSegmentInfos, () -> store.decRefFileDeleter(files)); + } catch (IOException e) { + throw new EngineException(shardId, e.getMessage(), e); + } + } + protected LocalCheckpointTracker getLocalCheckpointTracker() { return localCheckpointTracker; } diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index 2b85193275a13..ace6ed56c007c 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -677,8 +677,19 @@ public void updateShardState( // this Shard's engine was read only, we need to update its engine before restoring local history from xlog. assert newRouting.primary() && currentRouting.primary() == false; resetEngineToGlobalCheckpoint(); + // It is possible an engine can open with a SegmentInfos on a higher gen but the reader does not refresh to + // trigger our refresh listener. + // Force update the checkpoint post engine reset. + updateReplicationCheckpoint(); } + replicationTracker.activatePrimaryMode(getLocalCheckpoint()); + if (indexSettings.isSegRepEnabled()) { + // force publish a checkpoint once in primary mode so that replicas not caught up to previous primary + // are brought up to date. + checkpointPublisher.publish(this, getLatestReplicationCheckpoint()); + } + ensurePeerRecoveryRetentionLeasesExist(); /* * If this shard was serving as a replica shard when another shard was promoted to primary then @@ -1551,15 +1562,7 @@ public GatedCloseable acquireSafeIndexCommit() throws EngineExcepti * @return EMPTY checkpoint before the engine is opened and null for non-segrep enabled indices */ public ReplicationCheckpoint getLatestReplicationCheckpoint() { - final Tuple, ReplicationCheckpoint> infosAndCheckpoint = getLatestSegmentInfosAndCheckpoint(); - if (infosAndCheckpoint == null) { - return null; - } - try (final GatedCloseable ignored = infosAndCheckpoint.v1()) { - return infosAndCheckpoint.v2(); - } catch (IOException e) { - throw new OpenSearchException("Error Closing SegmentInfos Snapshot", e); - } + return replicationTracker.getLatestReplicationCheckpoint(); } /** @@ -1573,13 +1576,11 @@ public ReplicationCheckpoint getLatestReplicationCheckpoint() { * */ public Tuple, ReplicationCheckpoint> getLatestSegmentInfosAndCheckpoint() { - if (indexSettings.isSegRepEnabled() == false) { - return null; - } + assert indexSettings.isSegRepEnabled(); Tuple, ReplicationCheckpoint> nullSegmentInfosEmptyCheckpoint = new Tuple<>( new GatedCloseable<>(null, () -> {}), - ReplicationCheckpoint.empty(shardId, getDefaultCodecName()) + getLatestReplicationCheckpoint() ); if (getEngineOrNull() == null) { @@ -1598,11 +1599,7 @@ public Tuple, ReplicationCheckpoint> getLatestSegme getOperationPrimaryTerm(), segmentInfos.getGeneration(), segmentInfos.getVersion(), - // TODO: Update replicas to compute length from SegmentInfos. Replicas do not yet incref segments with - // getSegmentInfosSnapshot, so computing length from SegmentInfos can cause issues. - shardRouting.primary() - ? store.getSegmentMetadataMap(segmentInfos).values().stream().mapToLong(StoreFileMetadata::length).sum() - : store.stats(StoreStats.UNKNOWN_RESERVED_BYTES).getSizeInBytes(), + store.getSegmentMetadataMap(segmentInfos).values().stream().mapToLong(StoreFileMetadata::length).sum(), getEngine().config().getCodec().getName() ) ); @@ -1858,10 +1855,6 @@ public void resetToWriteableEngine() throws IOException, InterruptedException, T indexShardOperationPermits.blockOperations(30, TimeUnit.MINUTES, () -> { resetEngineToGlobalCheckpoint(); }); } - public void onCheckpointPublished(ReplicationCheckpoint checkpoint) { - replicationTracker.setLatestReplicationCheckpoint(checkpoint); - } - /** * Wrapper for a non-closing reader * @@ -2342,6 +2335,11 @@ private void innerOpenEngineAndTranslog(LongSupplier globalCheckpointSupplier, b final Engine newEngine = engineFactory.newReadWriteEngine(config); onNewEngine(newEngine); currentEngineReference.set(newEngine); + + if (indexSettings.isSegRepEnabled()) { + // set initial replication checkpoints into tracker. + updateReplicationCheckpoint(); + } // We set active because we are now writing operations to the engine; this way, // we can flush if we go idle after some time and become inactive. active.set(true); @@ -3667,6 +3665,9 @@ private EngineConfig newEngineConfig(LongSupplier globalCheckpointSupplier) thro internalRefreshListener.clear(); internalRefreshListener.add(new RefreshMetricUpdater(refreshMetric)); + if (indexSettings.isSegRepEnabled()) { + internalRefreshListener.add(new ReplicationCheckpointUpdater()); + } if (this.checkpointPublisher != null && shardRouting.primary() && indexSettings.isSegRepLocalEnabled()) { internalRefreshListener.add(new CheckpointRefreshListener(this, this.checkpointPublisher)); } @@ -4471,6 +4472,30 @@ public void afterRefresh(boolean didRefresh) throws IOException { } } + /** + * Refresh listener to update the Shard's ReplicationCheckpoint post refresh. + */ + private class ReplicationCheckpointUpdater implements ReferenceManager.RefreshListener { + @Override + public void beforeRefresh() throws IOException {} + + @Override + public void afterRefresh(boolean didRefresh) throws IOException { + if (didRefresh) { + updateReplicationCheckpoint(); + } + } + } + + private void updateReplicationCheckpoint() { + final Tuple, ReplicationCheckpoint> tuple = getLatestSegmentInfosAndCheckpoint(); + try (final GatedCloseable ignored = tuple.v1()) { + replicationTracker.setLatestReplicationCheckpoint(tuple.v2()); + } catch (IOException e) { + throw new OpenSearchException("Error Closing SegmentInfos Snapshot", e); + } + } + private EngineConfig.TombstoneDocSupplier tombstoneDocSupplier() { final RootObjectMapper.Builder noopRootMapper = new RootObjectMapper.Builder("__noop"); final DocumentMapper noopDocumentMapper = mapperService != null diff --git a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java index 8dd0c8b9d4405..d56054dd1c42b 100644 --- a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java +++ b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java @@ -185,7 +185,6 @@ private synchronized boolean syncSegments() { return true; } ReplicationCheckpoint checkpoint = indexShard.getLatestReplicationCheckpoint(); - indexShard.onCheckpointPublished(checkpoint); beforeSegmentsSync(); long refreshTimeMs = segmentTracker.getLocalRefreshTimeMs(), refreshClockTimeMs = segmentTracker.getLocalRefreshClockTimeMs(); long refreshSeqNo = segmentTracker.getLocalRefreshSeqNo(); @@ -207,6 +206,10 @@ private synchronized boolean syncSegments() { try (GatedCloseable segmentInfosGatedCloseable = indexShard.getSegmentInfosSnapshot()) { SegmentInfos segmentInfos = segmentInfosGatedCloseable.get(); + assert segmentInfos.getGeneration() == checkpoint.getSegmentsGen() : "SegmentInfos generation: " + + segmentInfos.getGeneration() + + " does not match metadata generation: " + + checkpoint.getSegmentsGen(); // Capture replication checkpoint before uploading the segments as upload can take some time and checkpoint can // move. long lastRefreshedCheckpoint = ((InternalEngine) indexShard.getEngine()).lastRefreshedCheckpoint(); diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java index c22701dfc94ce..3a84163bb979d 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java @@ -51,9 +51,14 @@ public class SegmentReplicationTarget extends ReplicationTarget { public final static String REPLICATION_PREFIX = "replication."; - public SegmentReplicationTarget(IndexShard indexShard, SegmentReplicationSource source, ReplicationListener listener) { + public SegmentReplicationTarget( + IndexShard indexShard, + ReplicationCheckpoint checkpoint, + SegmentReplicationSource source, + ReplicationListener listener + ) { super("replication_target", indexShard, new ReplicationLuceneIndex(), listener); - this.checkpoint = indexShard.getLatestReplicationCheckpoint(); + this.checkpoint = checkpoint; this.source = source; this.state = new SegmentReplicationState( indexShard.routingEntry(), @@ -90,12 +95,19 @@ public SegmentReplicationState state() { } public SegmentReplicationTarget retryCopy() { - return new SegmentReplicationTarget(indexShard, source, listener); + return new SegmentReplicationTarget(indexShard, checkpoint, source, listener); } @Override public String description() { - return String.format(Locale.ROOT, "Id:[%d] Shard:[%s] Source:[%s]", getId(), shardId(), source.getDescription()); + return String.format( + Locale.ROOT, + "Id:[%d] Checkpoint [%s] Shard:[%s] Source:[%s]", + getId(), + getCheckpoint(), + shardId(), + source.getDescription() + ); } @Override diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java index b41c9e09add45..84d6a722e572e 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java @@ -234,7 +234,7 @@ public synchronized void onNewCheckpoint(final ReplicationCheckpoint receivedChe logger.trace( () -> new ParameterizedMessage( "Ignoring new replication checkpoint - shard is currently replicating to checkpoint {}", - replicaShard.getLatestReplicationCheckpoint() + ongoingReplicationTarget.getCheckpoint() ) ); return; @@ -242,7 +242,7 @@ public synchronized void onNewCheckpoint(final ReplicationCheckpoint receivedChe } final Thread thread = Thread.currentThread(); if (replicaShard.shouldProcessCheckpoint(receivedCheckpoint)) { - startReplication(replicaShard, new SegmentReplicationListener() { + startReplication(replicaShard, receivedCheckpoint, new SegmentReplicationListener() { @Override public void onReplicationDone(SegmentReplicationState state) { logger.trace( @@ -280,6 +280,8 @@ public void onReplicationFailure( ); if (sendShardFailure == true) { failShard(e, replicaShard); + } else { + processLatestReceivedCheckpoint(replicaShard, thread); } } }); @@ -396,8 +398,24 @@ protected void updateLatestReceivedCheckpoint(ReplicationCheckpoint receivedChec } } - public SegmentReplicationTarget startReplication(final IndexShard indexShard, final SegmentReplicationListener listener) { - final SegmentReplicationTarget target = new SegmentReplicationTarget(indexShard, sourceFactory.get(indexShard), listener); + /** + * Start a round of replication and sync to at least the given checkpoint. + * @param indexShard - {@link IndexShard} replica shard + * @param checkpoint - {@link ReplicationCheckpoint} checkpoint to sync to + * @param listener - {@link ReplicationListener} + * @return {@link SegmentReplicationTarget} target event orchestrating the event. + */ + public SegmentReplicationTarget startReplication( + final IndexShard indexShard, + final ReplicationCheckpoint checkpoint, + final SegmentReplicationListener listener + ) { + final SegmentReplicationTarget target = new SegmentReplicationTarget( + indexShard, + checkpoint, + sourceFactory.get(indexShard), + listener + ); startReplication(target); return target; } @@ -529,50 +547,59 @@ private void forceReplication(ForceSyncRequest request, ActionListener new ParameterizedMessage( + "[shardId {}] [replication id {}] Force replication Sync complete to {}, timing data: {}", + shardId, + state.getReplicationId(), + indexShard.getLatestReplicationCheckpoint(), + state.getTimingData() + ) + ); + // Promote engine type for primary target + if (indexShard.recoveryState().getPrimary() == true) { + indexShard.resetToWriteableEngine(); + } else { + // Update the replica's checkpoint on primary's replication tracker. + updateVisibleCheckpoint(state.getReplicationId(), indexShard); + } + listener.onResponse(TransportResponse.Empty.INSTANCE); + } catch (Exception e) { + logger.error("Error while marking replication completed", e); + listener.onFailure(e); + } + } + + @Override + public void onReplicationFailure( + SegmentReplicationState state, + ReplicationFailedException e, + boolean sendShardFailure + ) { + logger.error( () -> new ParameterizedMessage( - "[shardId {}] [replication id {}] Force replication Sync complete to {}, timing data: {}", - shardId, + "[shardId {}] [replication id {}] Force replication Sync failed, timing data: {}", + indexShard.shardId().getId(), state.getReplicationId(), - indexShard.getLatestReplicationCheckpoint(), state.getTimingData() - ) + ), + e ); - // Promote engine type for primary target - if (indexShard.recoveryState().getPrimary() == true) { - indexShard.resetToWriteableEngine(); - } else { - // Update the replica's checkpoint on primary's replication tracker. - updateVisibleCheckpoint(state.getReplicationId(), indexShard); + if (sendShardFailure) { + failShard(e, indexShard); } - listener.onResponse(TransportResponse.Empty.INSTANCE); - } catch (Exception e) { - logger.error("Error while marking replication completed", e); listener.onFailure(e); } } - - @Override - public void onReplicationFailure(SegmentReplicationState state, ReplicationFailedException e, boolean sendShardFailure) { - logger.error( - () -> new ParameterizedMessage( - "[shardId {}] [replication id {}] Replication failed, timing data: {}", - indexShard.shardId().getId(), - state.getReplicationId(), - state.getTimingData() - ), - e - ); - if (sendShardFailure) { - failShard(e, indexShard); - } - listener.onFailure(e); - } - }); + ); } } diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/SegmentReplicationCheckpointPublisher.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/SegmentReplicationCheckpointPublisher.java index b4bcdc92e539a..f5cb32b741862 100644 --- a/server/src/main/java/org/opensearch/indices/replication/checkpoint/SegmentReplicationCheckpointPublisher.java +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/SegmentReplicationCheckpointPublisher.java @@ -34,7 +34,6 @@ public SegmentReplicationCheckpointPublisher(PublishAction publishAction) { public void publish(IndexShard indexShard, ReplicationCheckpoint checkpoint) { publishAction.publish(indexShard, checkpoint); - indexShard.onCheckpointPublished(checkpoint); } /** diff --git a/server/src/test/java/org/opensearch/index/engine/NRTReplicationEngineTests.java b/server/src/test/java/org/opensearch/index/engine/NRTReplicationEngineTests.java index 64fe42493c686..4c87df48f583f 100644 --- a/server/src/test/java/org/opensearch/index/engine/NRTReplicationEngineTests.java +++ b/server/src/test/java/org/opensearch/index/engine/NRTReplicationEngineTests.java @@ -12,7 +12,9 @@ import org.apache.lucene.index.NoMergePolicy; import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.search.ReferenceManager; +import org.apache.lucene.store.IOContext; import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.UUIDs; import org.opensearch.common.concurrent.GatedCloseable; import org.opensearch.common.lucene.Lucene; import org.opensearch.common.lucene.index.OpenSearchDirectoryReader; @@ -28,6 +30,8 @@ import java.io.IOException; import java.nio.file.Path; +import java.util.Arrays; +import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Set; @@ -367,4 +371,72 @@ private NRTReplicationEngine buildNrtReplicaEngine(AtomicLong globalCheckpoint, private NRTReplicationEngine buildNrtReplicaEngine(AtomicLong globalCheckpoint, Store store) throws IOException { return buildNrtReplicaEngine(globalCheckpoint, store, defaultSettings); } + + public void testGetSegmentInfosSnapshotPreservesFilesUntilRelease() throws Exception { + final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); + + // TODO: Remove this divergent commit logic and copy Segments_N from primary with node-node. + // randomly toggle commit / no commit. + IndexSettings settings = REMOTE_STORE_INDEX_SETTINGS; + final boolean shouldCommit = randomBoolean(); + if (shouldCommit) { + settings = INDEX_SETTINGS; + } + try ( + final Store nrtEngineStore = createStore(REMOTE_STORE_INDEX_SETTINGS, newDirectory()); + final NRTReplicationEngine nrtEngine = buildNrtReplicaEngine(globalCheckpoint, nrtEngineStore, settings) + ) { + // only index 2 docs here, this will create segments _0 and _1 and after forcemerge into _2. + final int docCount = 2; + List operations = generateHistoryOnReplica(docCount, randomBoolean(), randomBoolean(), randomBoolean()); + for (Engine.Operation op : operations) { + applyOperation(engine, op); + applyOperation(nrtEngine, op); + // refresh to create a lot of segments. + engine.refresh("test"); + } + assertEquals(2, engine.segmentsStats(false, false).getCount()); + // wipe the nrt directory initially so we can sync with primary. + Lucene.cleanLuceneIndex(nrtEngineStore.directory()); + assertFalse( + Arrays.stream(nrtEngineStore.directory().listAll()) + .anyMatch(file -> file.equals("write.lock") == false && file.equals("extra0") == false) + ); + for (String file : engine.getLatestSegmentInfos().files(true)) { + nrtEngineStore.directory().copyFrom(store.directory(), file, file, IOContext.DEFAULT); + } + nrtEngine.updateSegments(engine.getLatestSegmentInfos()); + assertEquals(engine.getLatestSegmentInfos(), nrtEngine.getLatestSegmentInfos()); + final GatedCloseable snapshot = nrtEngine.getSegmentInfosSnapshot(); + final Collection replicaSnapshotFiles = snapshot.get().files(false); + List replicaFiles = List.of(nrtEngine.store.directory().listAll()); + + // merge primary down to 1 segment + engine.forceMerge(true, 1, false, false, false, UUIDs.randomBase64UUID()); + // we expect a 3rd segment to be created after merge. + assertEquals(3, engine.segmentsStats(false, false).getCount()); + final Collection latestPrimaryFiles = engine.getLatestSegmentInfos().files(false); + + // copy new segments in and load reader. + for (String file : latestPrimaryFiles) { + if (replicaFiles.contains(file) == false) { + nrtEngineStore.directory().copyFrom(store.directory(), file, file, IOContext.DEFAULT); + } + } + nrtEngine.updateSegments(engine.getLatestSegmentInfos()); + + replicaFiles = List.of(nrtEngine.store.directory().listAll()); + assertTrue(replicaFiles.containsAll(replicaSnapshotFiles)); + + // close snapshot, files should be cleaned up + snapshot.close(); + + replicaFiles = List.of(nrtEngine.store.directory().listAll()); + assertFalse(replicaFiles.containsAll(replicaSnapshotFiles)); + + // Ensure we still have all the active files. Note - we exclude the infos file here if we aren't committing + // the nrt reader will still reference segments_n-1 after being loaded until a local commit occurs. + assertTrue(replicaFiles.containsAll(nrtEngine.getLatestSegmentInfos().files(shouldCommit))); + } + } } diff --git a/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java index f1ae4b16d9bad..8ffeb5d689f55 100644 --- a/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java @@ -3753,7 +3753,7 @@ public void testReadSnapshotConcurrently() throws IOException, InterruptedExcept */ public void testCheckpointRefreshListener() throws IOException { final SegmentReplicationCheckpointPublisher mock = mock(SegmentReplicationCheckpointPublisher.class); - IndexShard shard = newStartedShard(p -> newShard(mock), true); + IndexShard shard = newStartedShard(p -> newShard(true, mock), true); List refreshListeners = shard.getEngine().config().getInternalRefreshListener(); assertTrue(refreshListeners.stream().anyMatch(e -> e instanceof CheckpointRefreshListener)); closeShards(shard); @@ -3763,58 +3763,13 @@ public void testCheckpointRefreshListener() throws IOException { * here we are passing null in place of SegmentReplicationCheckpointPublisher and testing on index shard if CheckpointRefreshListener is not added to the InternalrefreshListerners List */ public void testCheckpointRefreshListenerWithNull() throws IOException { - IndexShard shard = newStartedShard(p -> newShard(null), true); + final SegmentReplicationCheckpointPublisher publisher = null; + IndexShard shard = newStartedShard(p -> newShard(true, publisher), true); List refreshListeners = shard.getEngine().config().getInternalRefreshListener(); assertFalse(refreshListeners.stream().anyMatch(e -> e instanceof CheckpointRefreshListener)); closeShards(shard); } - /** - * creates a new initializing shard. The shard will be put in its proper path under the - * current node id the shard is assigned to. - * @param checkpointPublisher Segment Replication Checkpoint Publisher to publish checkpoint - */ - private IndexShard newShard(SegmentReplicationCheckpointPublisher checkpointPublisher) throws IOException { - final ShardId shardId = new ShardId("index", "_na_", 0); - final ShardRouting shardRouting = TestShardRouting.newShardRouting( - shardId, - randomAlphaOfLength(10), - true, - ShardRoutingState.INITIALIZING, - RecoverySource.EmptyStoreRecoverySource.INSTANCE - ); - final NodeEnvironment.NodePath nodePath = new NodeEnvironment.NodePath(createTempDir()); - ShardPath shardPath = new ShardPath(false, nodePath.resolve(shardId), nodePath.resolve(shardId), shardId); - - Settings indexSettings = Settings.builder() - .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) - .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) - .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) - .put(IndexMetadata.SETTING_REPLICATION_TYPE, "SEGMENT") - .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), between(0, 1000)) - .put(Settings.EMPTY) - .build(); - IndexMetadata metadata = IndexMetadata.builder(shardRouting.getIndexName()) - .settings(indexSettings) - .primaryTerm(0, primaryTerm) - .putMapping("{ \"properties\": {} }") - .build(); - return newShard( - shardRouting, - shardPath, - metadata, - null, - null, - new InternalEngineFactory(), - new EngineConfigFactory(new IndexSettings(metadata, metadata.getSettings())), - () -> {}, - RetentionLeaseSyncer.EMPTY, - EMPTY_EVENT_LISTENER, - checkpointPublisher, - null - ); - } - public void testIndexCheckOnStartup() throws Exception { final IndexShard indexShard = newStartedShard(true); diff --git a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java index 9c02f430e4e6d..58ae4b404e69c 100644 --- a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java @@ -19,6 +19,7 @@ import org.opensearch.action.index.IndexRequest; import org.opensearch.action.support.PlainActionFuture; import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.routing.IndexShardRoutingTable; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.ShardRoutingHelper; import org.opensearch.common.collect.Tuple; @@ -63,6 +64,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.concurrent.CountDownLatch; @@ -423,7 +425,70 @@ public void testShardIdleWithNoReplicas() throws Exception { /** * here we are starting a new primary shard in PrimaryMode and testing if the shard publishes checkpoint after refresh. */ - public void testPublishCheckpointOnPrimaryMode() throws IOException { + public void testPublishCheckpointOnPrimaryMode() throws IOException, InterruptedException { + final SegmentReplicationCheckpointPublisher mock = mock(SegmentReplicationCheckpointPublisher.class); + IndexShard shard = newStartedShard(p -> newShard(false, mock, settings), false); + + final ShardRouting shardRouting = shard.routingEntry(); + promoteReplica( + shard, + Collections.singleton(shardRouting.allocationId().getId()), + new IndexShardRoutingTable.Builder(shardRouting.shardId()).addShard(shardRouting).build() + ); + + final CountDownLatch latch = new CountDownLatch(1); + shard.acquirePrimaryOperationPermit(new ActionListener() { + @Override + public void onResponse(Releasable releasable) { + releasable.close(); + latch.countDown(); + } + + @Override + public void onFailure(Exception e) { + throw new RuntimeException(e); + } + }, ThreadPool.Names.GENERIC, ""); + + latch.await(); + // verify checkpoint is published + verify(mock, times(1)).publish(any(), any()); + closeShards(shard); + } + + public void testPublishCheckpointOnPrimaryMode_segrep_off() throws IOException, InterruptedException { + final SegmentReplicationCheckpointPublisher mock = mock(SegmentReplicationCheckpointPublisher.class); + final Settings settings = Settings.builder().put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.DOCUMENT).build(); + IndexShard shard = newStartedShard(p -> newShard(false, mock, settings), false); + + final ShardRouting shardRouting = shard.routingEntry(); + promoteReplica( + shard, + Collections.singleton(shardRouting.allocationId().getId()), + new IndexShardRoutingTable.Builder(shardRouting.shardId()).addShard(shardRouting).build() + ); + + final CountDownLatch latch = new CountDownLatch(1); + shard.acquirePrimaryOperationPermit(new ActionListener() { + @Override + public void onResponse(Releasable releasable) { + releasable.close(); + latch.countDown(); + } + + @Override + public void onFailure(Exception e) { + throw new RuntimeException(e); + } + }, ThreadPool.Names.GENERIC, ""); + + latch.await(); + // verify checkpoint is published + verify(mock, times(0)).publish(any(), any()); + closeShards(shard); + } + + public void testPublishCheckpointPostFailover() throws IOException { final SegmentReplicationCheckpointPublisher mock = mock(SegmentReplicationCheckpointPublisher.class); IndexShard shard = newStartedShard(true); CheckpointRefreshListener refreshListener = new CheckpointRefreshListener(shard, mock); @@ -482,7 +547,7 @@ public void testRejectCheckpointOnShardRoutingPrimary() throws IOException { spy.onNewCheckpoint(new ReplicationCheckpoint(primaryShard.shardId(), 0L, 0L, 0L, Codec.getDefault().getName()), spyShard); // Verify that checkpoint is not processed as shard routing is primary. - verify(spy, times(0)).startReplication(any(), any()); + verify(spy, times(0)).startReplication(any(), any(), any()); closeShards(primaryShard); } @@ -656,7 +721,7 @@ public void cancel() { } }; when(sourceFactory.get(any())).thenReturn(source); - startReplicationAndAssertCancellation(replica, targetService); + startReplicationAndAssertCancellation(replica, primary, targetService); shards.removeReplica(replica); closeShards(replica); @@ -736,11 +801,15 @@ protected void resolveCheckpointInfoResponseListener(ActionListener() { @@ -422,7 +422,11 @@ public void testTemporaryFilesNotCleanup() throws Exception { runnablePostGetFiles ); when(sourceFactory.get(any())).thenReturn(segmentReplicationSource); - targetService.startReplication(replica, getTargetListener(primaryShard, replica, primaryMetadata, countDownLatch)); + targetService.startReplication( + replica, + primaryShard.getLatestReplicationCheckpoint(), + getTargetListener(primaryShard, replica, primaryMetadata, countDownLatch) + ); countDownLatch.await(30, TimeUnit.SECONDS); assertEquals("Replication failed", 0, countDownLatch.getCount()); shards.assertAllEqual(numDocs); diff --git a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetServiceTests.java b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetServiceTests.java index efb8dda201e87..4a04a64196918 100644 --- a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetServiceTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetServiceTests.java @@ -84,6 +84,7 @@ public class SegmentReplicationTargetServiceTests extends IndexShardTestCase { private IndicesService indicesService; private SegmentReplicationState state; + private ReplicationCheckpoint initialCheckpoint; private static final long TRANSPORT_TIMEOUT = 30000;// 30sec @@ -130,7 +131,7 @@ public void setUp() throws Exception { when(clusterState.nodes()).thenReturn(DiscoveryNodes.builder().add(localNode).build()); sut = prepareForReplication(primaryShard, replicaShard, transportService, indicesService, clusterService); - ReplicationCheckpoint initialCheckpoint = replicaShard.getLatestReplicationCheckpoint(); + initialCheckpoint = primaryShard.getLatestReplicationCheckpoint(); aheadCheckpoint = new ReplicationCheckpoint( initialCheckpoint.getShardId(), initialCheckpoint.getPrimaryTerm(), @@ -165,19 +166,23 @@ public void tearDown() throws Exception { public void testsSuccessfulReplication_listenerCompletes() throws InterruptedException { CountDownLatch latch = new CountDownLatch(1); - sut.startReplication(replicaShard, new SegmentReplicationTargetService.SegmentReplicationListener() { - @Override - public void onReplicationDone(SegmentReplicationState state) { - assertEquals(SegmentReplicationState.Stage.DONE, state.getStage()); - latch.countDown(); - } + sut.startReplication( + replicaShard, + primaryShard.getLatestReplicationCheckpoint(), + new SegmentReplicationTargetService.SegmentReplicationListener() { + @Override + public void onReplicationDone(SegmentReplicationState state) { + assertEquals(SegmentReplicationState.Stage.DONE, state.getStage()); + latch.countDown(); + } - @Override - public void onReplicationFailure(SegmentReplicationState state, ReplicationFailedException e, boolean sendShardFailure) { - logger.error("Unexpected error", e); - Assert.fail("Test should succeed"); + @Override + public void onReplicationFailure(SegmentReplicationState state, ReplicationFailedException e, boolean sendShardFailure) { + logger.error("Unexpected error", e); + Assert.fail("Test should succeed"); + } } - }); + ); latch.await(2, TimeUnit.SECONDS); assertEquals(0, latch.getCount()); } @@ -209,6 +214,7 @@ public void getSegmentFiles( }; final SegmentReplicationTarget target = new SegmentReplicationTarget( replicaShard, + primaryShard.getLatestReplicationCheckpoint(), source, new SegmentReplicationTargetService.SegmentReplicationListener() { @Override @@ -233,7 +239,7 @@ public void onReplicationFailure(SegmentReplicationState state, ReplicationFaile public void testAlreadyOnNewCheckpoint() { SegmentReplicationTargetService spy = spy(sut); spy.onNewCheckpoint(replicaShard.getLatestReplicationCheckpoint(), replicaShard); - verify(spy, times(0)).startReplication(any(), any()); + verify(spy, times(0)).startReplication(any(), any(), any()); } public void testShardAlreadyReplicating() { @@ -271,24 +277,22 @@ public void getSegmentFiles( } }; final SegmentReplicationTarget target = spy( - new SegmentReplicationTarget(replicaShard, source, mock(SegmentReplicationTargetService.SegmentReplicationListener.class)) + new SegmentReplicationTarget( + replicaShard, + primaryShard.getLatestReplicationCheckpoint(), + source, + mock(SegmentReplicationTargetService.SegmentReplicationListener.class) + ) ); + + final SegmentReplicationTargetService spy = spy(sut); + doReturn(false).when(spy).processLatestReceivedCheckpoint(eq(replicaShard), any()); // Start first round of segment replication. - sut.startReplication(target); + spy.startReplication(target); // Start second round of segment replication, this should fail to start as first round is still in-progress - sut.startReplication(replicaShard, new SegmentReplicationTargetService.SegmentReplicationListener() { - @Override - public void onReplicationDone(SegmentReplicationState state) { - Assert.fail("Should not succeed"); - } - - @Override - public void onReplicationFailure(SegmentReplicationState state, ReplicationFailedException e, boolean sendShardFailure) { - assertEquals("Shard " + replicaShard.shardId() + " is already replicating", e.getMessage()); - assertFalse(sendShardFailure); - } - }); + spy.onNewCheckpoint(newPrimaryCheckpoint, replicaShard); + verify(spy, times(1)).processLatestReceivedCheckpoint(eq(replicaShard), any()); blockGetCheckpointMetadata.countDown(); } @@ -337,8 +341,21 @@ public void cancel() { } }; + final ReplicationCheckpoint updatedCheckpoint = new ReplicationCheckpoint( + initialCheckpoint.getShardId(), + initialCheckpoint.getPrimaryTerm(), + initialCheckpoint.getSegmentsGen(), + initialCheckpoint.getSegmentInfosVersion() + 1, + primaryShard.getDefaultCodecName() + ); + final SegmentReplicationTarget targetSpy = spy( - new SegmentReplicationTarget(replicaShard, source, mock(SegmentReplicationTargetService.SegmentReplicationListener.class)) + new SegmentReplicationTarget( + replicaShard, + updatedCheckpoint, + source, + mock(SegmentReplicationTargetService.SegmentReplicationListener.class) + ) ); // start replication. This adds the target to on-ongoing replication collection @@ -352,20 +369,20 @@ public void cancel() { // ensure the old target is cancelled. and new iteration kicks off. verify(targetSpy, times(1)).cancel("Cancelling stuck target after new primary"); - verify(serviceSpy, times(1)).startReplication(eq(replicaShard), any()); + verify(serviceSpy, times(1)).startReplication(eq(replicaShard), any(), any()); } public void testNewCheckpointBehindCurrentCheckpoint() { SegmentReplicationTargetService spy = spy(sut); spy.onNewCheckpoint(checkpoint, replicaShard); - verify(spy, times(0)).startReplication(any(), any()); + verify(spy, times(0)).startReplication(any(), any(), any()); } public void testShardNotStarted() throws IOException { SegmentReplicationTargetService spy = spy(sut); IndexShard shard = newShard(false); spy.onNewCheckpoint(checkpoint, shard); - verify(spy, times(0)).startReplication(any(), any()); + verify(spy, times(0)).startReplication(any(), any(), any()); closeShards(shard); } @@ -381,7 +398,7 @@ public void testRejectCheckpointOnShardPrimaryMode() throws IOException { spy.onNewCheckpoint(aheadCheckpoint, spyShard); // Verify that checkpoint is not processed as shard is in PrimaryMode. - verify(spy, times(0)).startReplication(any(), any()); + verify(spy, times(0)).startReplication(any(), any(), any()); closeShards(primaryShard); } @@ -406,10 +423,10 @@ public void testStartReplicationListenerSuccess() throws InterruptedException { SegmentReplicationTargetService spy = spy(sut); CountDownLatch latch = new CountDownLatch(1); doAnswer(i -> { - ((SegmentReplicationTargetService.SegmentReplicationListener) i.getArgument(1)).onReplicationDone(state); + ((SegmentReplicationTargetService.SegmentReplicationListener) i.getArgument(2)).onReplicationDone(state); latch.countDown(); return null; - }).when(spy).startReplication(any(), any()); + }).when(spy).startReplication(any(), any(), any()); doNothing().when(spy).updateVisibleCheckpoint(eq(0L), any()); spy.afterIndexShardStarted(replicaShard); @@ -422,14 +439,14 @@ public void testStartReplicationListenerFailure() throws InterruptedException { SegmentReplicationTargetService spy = spy(sut); CountDownLatch latch = new CountDownLatch(1); doAnswer(i -> { - ((SegmentReplicationTargetService.SegmentReplicationListener) i.getArgument(1)).onReplicationFailure( + ((SegmentReplicationTargetService.SegmentReplicationListener) i.getArgument(2)).onReplicationFailure( state, new ReplicationFailedException(replicaShard, null), false ); latch.countDown(); return null; - }).when(spy).startReplication(any(), any()); + }).when(spy).startReplication(any(), any(), any()); doNothing().when(spy).updateVisibleCheckpoint(eq(0L), any()); spy.afterIndexShardStarted(replicaShard); @@ -570,6 +587,7 @@ public void testForceSegmentSyncHandlerWithFailure_AlreadyClosedException_swallo public void testTargetCancelledBeforeStartInvoked() { final SegmentReplicationTarget target = new SegmentReplicationTarget( replicaShard, + primaryShard.getLatestReplicationCheckpoint(), mock(SegmentReplicationSource.class), new SegmentReplicationTargetService.SegmentReplicationListener() { @Override diff --git a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetTests.java b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetTests.java index 176954b6d6b3d..5b996fd774baf 100644 --- a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetTests.java @@ -141,7 +141,7 @@ public void getSegmentFiles( SegmentReplicationTargetService.SegmentReplicationListener segRepListener = mock( SegmentReplicationTargetService.SegmentReplicationListener.class ); - segrepTarget = new SegmentReplicationTarget(spyIndexShard, segrepSource, segRepListener); + segrepTarget = new SegmentReplicationTarget(spyIndexShard, repCheckpoint, segrepSource, segRepListener); segrepTarget.startReplication(new ActionListener() { @Override @@ -189,7 +189,7 @@ public void getSegmentFiles( SegmentReplicationTargetService.SegmentReplicationListener segRepListener = mock( SegmentReplicationTargetService.SegmentReplicationListener.class ); - segrepTarget = new SegmentReplicationTarget(spyIndexShard, segrepSource, segRepListener); + segrepTarget = new SegmentReplicationTarget(spyIndexShard, repCheckpoint, segrepSource, segRepListener); segrepTarget.startReplication(new ActionListener() { @Override @@ -232,7 +232,7 @@ public void getSegmentFiles( SegmentReplicationTargetService.SegmentReplicationListener segRepListener = mock( SegmentReplicationTargetService.SegmentReplicationListener.class ); - segrepTarget = new SegmentReplicationTarget(spyIndexShard, segrepSource, segRepListener); + segrepTarget = new SegmentReplicationTarget(spyIndexShard, repCheckpoint, segrepSource, segRepListener); segrepTarget.startReplication(new ActionListener() { @Override @@ -275,7 +275,7 @@ public void getSegmentFiles( SegmentReplicationTargetService.SegmentReplicationListener segRepListener = mock( SegmentReplicationTargetService.SegmentReplicationListener.class ); - segrepTarget = new SegmentReplicationTarget(spyIndexShard, segrepSource, segRepListener); + segrepTarget = new SegmentReplicationTarget(spyIndexShard, repCheckpoint, segrepSource, segRepListener); doThrow(exception).when(spyIndexShard).finalizeReplication(any()); @@ -320,7 +320,7 @@ public void getSegmentFiles( SegmentReplicationTargetService.SegmentReplicationListener segRepListener = mock( SegmentReplicationTargetService.SegmentReplicationListener.class ); - segrepTarget = new SegmentReplicationTarget(spyIndexShard, segrepSource, segRepListener); + segrepTarget = new SegmentReplicationTarget(spyIndexShard, repCheckpoint, segrepSource, segRepListener); doThrow(exception).when(spyIndexShard).finalizeReplication(any()); @@ -364,7 +364,7 @@ public void getSegmentFiles( SegmentReplicationTargetService.SegmentReplicationListener segRepListener = mock( SegmentReplicationTargetService.SegmentReplicationListener.class ); - segrepTarget = new SegmentReplicationTarget(spyIndexShard, segrepSource, segRepListener); + segrepTarget = new SegmentReplicationTarget(spyIndexShard, repCheckpoint, segrepSource, segRepListener); when(spyIndexShard.getSegmentMetadataMap()).thenReturn(SI_SNAPSHOT_DIFFERENT); segrepTarget.startReplication(new ActionListener() { @Override @@ -417,7 +417,7 @@ public void getSegmentFiles( SegmentReplicationTargetService.SegmentReplicationListener.class ); - segrepTarget = new SegmentReplicationTarget(spyIndexShard, segrepSource, segRepListener); + segrepTarget = new SegmentReplicationTarget(spyIndexShard, repCheckpoint, segrepSource, segRepListener); when(spyIndexShard.getSegmentMetadataMap()).thenReturn(storeMetadataSnapshots.get(0).asMap()); segrepTarget.startReplication(new ActionListener() { @Override diff --git a/server/src/test/java/org/opensearch/recovery/ReplicationCollectionTests.java b/server/src/test/java/org/opensearch/recovery/ReplicationCollectionTests.java index 776173f73ce5c..9c38c5848e297 100644 --- a/server/src/test/java/org/opensearch/recovery/ReplicationCollectionTests.java +++ b/server/src/test/java/org/opensearch/recovery/ReplicationCollectionTests.java @@ -120,11 +120,13 @@ public void testStartMultipleReplicationsForSingleShard() throws Exception { shards.recoverReplica(shard); final SegmentReplicationTarget target1 = new SegmentReplicationTarget( shard, + shards.getPrimary().getLatestReplicationCheckpoint(), mock(SegmentReplicationSource.class), mock(ReplicationListener.class) ); final SegmentReplicationTarget target2 = new SegmentReplicationTarget( shard, + shards.getPrimary().getLatestReplicationCheckpoint(), mock(SegmentReplicationSource.class), mock(ReplicationListener.class) ); diff --git a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java index 29ecc6b376ad0..7ef460ebbfb50 100644 --- a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java @@ -94,6 +94,7 @@ import org.opensearch.index.engine.EngineFactory; import org.opensearch.index.engine.EngineTestCase; import org.opensearch.index.engine.InternalEngineFactory; +import org.opensearch.index.engine.NRTReplicationEngineFactory; import org.opensearch.index.mapper.MapperService; import org.opensearch.index.mapper.SourceToParse; import org.opensearch.index.remote.RemoteRefreshSegmentPressureService; @@ -141,6 +142,7 @@ import org.opensearch.indices.replication.common.ReplicationFailedException; import org.opensearch.indices.replication.common.ReplicationListener; import org.opensearch.indices.replication.common.ReplicationState; +import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.repositories.IndexId; import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.Repository; @@ -531,6 +533,58 @@ protected IndexShard newShard( ); } + protected IndexShard newShard(boolean primary, SegmentReplicationCheckpointPublisher checkpointPublisher) throws IOException { + final Settings settings = Settings.builder().put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT).build(); + return newShard(primary, checkpointPublisher, settings); + } + + /** + * creates a new initializing shard. The shard will be put in its proper path under the + * current node id the shard is assigned to. + * @param checkpointPublisher Segment Replication Checkpoint Publisher to publish checkpoint + */ + protected IndexShard newShard(boolean primary, SegmentReplicationCheckpointPublisher checkpointPublisher, Settings settings) + throws IOException { + final ShardId shardId = new ShardId("index", "_na_", 0); + final ShardRouting shardRouting = TestShardRouting.newShardRouting( + shardId, + randomAlphaOfLength(10), + primary, + ShardRoutingState.INITIALIZING, + primary ? RecoverySource.EmptyStoreRecoverySource.INSTANCE : RecoverySource.PeerRecoverySource.INSTANCE + ); + final NodeEnvironment.NodePath nodePath = new NodeEnvironment.NodePath(createTempDir()); + ShardPath shardPath = new ShardPath(false, nodePath.resolve(shardId), nodePath.resolve(shardId), shardId); + + Settings indexSettings = Settings.builder() + .put(settings) + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), between(0, 1000)) + .put(Settings.EMPTY) + .build(); + IndexMetadata metadata = IndexMetadata.builder(shardRouting.getIndexName()) + .settings(indexSettings) + .primaryTerm(0, primaryTerm) + .putMapping("{ \"properties\": {} }") + .build(); + return newShard( + shardRouting, + shardPath, + metadata, + null, + null, + new NRTReplicationEngineFactory(), + new EngineConfigFactory(new IndexSettings(metadata, metadata.getSettings())), + () -> {}, + RetentionLeaseSyncer.EMPTY, + EMPTY_EVENT_LISTENER, + checkpointPublisher, + null + ); + } + /** * creates a new initializing shard. * @param routing shard routing to use @@ -1527,10 +1581,7 @@ public void getCheckpointMetadata( ActionListener listener ) { try { - final CopyState copyState = new CopyState( - ReplicationCheckpoint.empty(primaryShard.shardId, primaryShard.getLatestReplicationCheckpoint().getCodec()), - primaryShard - ); + final CopyState copyState = new CopyState(primaryShard.getLatestReplicationCheckpoint(), primaryShard); listener.onResponse( new CheckpointInfoResponse(copyState.getCheckpoint(), copyState.getMetadataMap(), copyState.getInfosBytes()) ); @@ -1585,6 +1636,7 @@ protected final List replicateSegments(IndexShard prim final SegmentReplicationTargetService targetService = prepareForReplication(primaryShard, replica); final SegmentReplicationTarget target = targetService.startReplication( replica, + primaryShard.getLatestReplicationCheckpoint(), getTargetListener(primaryShard, replica, primaryMetadata, countDownLatch) ); ids.add(target); From 56a19eaef346034d8b49fe4cac83111ef093b674 Mon Sep 17 00:00:00 2001 From: Sorabh Date: Thu, 3 Aug 2023 11:00:02 -0700 Subject: [PATCH 03/19] Change to determine if concurrent segment search should be used by the request during SearchContext creation (#9059) * Change to determine if concurrent segment search should be used by the request during SearchContext creation. It caches the e evaluated output for all future invocation. It also provide executor to IndexSearcher based on this evaluation Signed-off-by: Sorabh Hamirwasia * Add test case to enable/disable concurrent search cluster setting and verify context object state Signed-off-by: Sorabh Hamirwasia --------- Signed-off-by: Sorabh Hamirwasia --- .../search/DefaultSearchContext.java | 37 ++++++---- .../internal/FilteredSearchContext.java | 5 ++ .../opensearch/search/SearchServiceTests.java | 69 +++++++++++++++++++ 3 files changed, 98 insertions(+), 13 deletions(-) diff --git a/server/src/main/java/org/opensearch/search/DefaultSearchContext.java b/server/src/main/java/org/opensearch/search/DefaultSearchContext.java index f377a5e315e1b..d10173184f1c6 100644 --- a/server/src/main/java/org/opensearch/search/DefaultSearchContext.java +++ b/server/src/main/java/org/opensearch/search/DefaultSearchContext.java @@ -183,6 +183,7 @@ final class DefaultSearchContext extends SearchContext { private final QueryShardContext queryShardContext; private final FetchPhase fetchPhase; private final Function requestToAggReduceContextBuilder; + private final boolean useConcurrentSearch; DefaultSearchContext( ReaderContext readerContext, @@ -213,13 +214,14 @@ final class DefaultSearchContext extends SearchContext { this.indexShard = readerContext.indexShard(); this.clusterService = clusterService; this.engineSearcher = readerContext.acquireSearcher("search"); + this.useConcurrentSearch = useConcurrentSearch(executor); this.searcher = new ContextIndexSearcher( engineSearcher.getIndexReader(), engineSearcher.getSimilarity(), engineSearcher.getQueryCache(), engineSearcher.getQueryCachingPolicy(), lowLevelCancellation, - executor, + useConcurrentSearch ? executor : null, this ); this.relativeTimeSupplier = relativeTimeSupplier; @@ -878,18 +880,7 @@ public Profilers getProfilers() { */ @Override public boolean isConcurrentSegmentSearchEnabled() { - if (FeatureFlags.isEnabled(FeatureFlags.CONCURRENT_SEGMENT_SEARCH) - && (clusterService != null) - && (searcher().getExecutor() != null)) { - return indexService.getIndexSettings() - .getSettings() - .getAsBoolean( - IndexSettings.INDEX_CONCURRENT_SEGMENT_SEARCH_SETTING.getKey(), - clusterService.getClusterSettings().get(CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING) - ); - } else { - return false; - } + return useConcurrentSearch; } public void setProfilers(Profilers profilers) { @@ -932,4 +923,24 @@ public void setBucketCollectorProcessor(BucketCollectorProcessor bucketCollector public BucketCollectorProcessor bucketCollectorProcessor() { return bucketCollectorProcessor; } + + /** + * Evaluate based on cluster and index settings if concurrent segment search should be used for this request context + * @return true: use concurrent search + * false: otherwise + */ + private boolean useConcurrentSearch(Executor concurrentSearchExecutor) { + if (FeatureFlags.isEnabled(FeatureFlags.CONCURRENT_SEGMENT_SEARCH) + && (clusterService != null) + && (concurrentSearchExecutor != null)) { + return indexService.getIndexSettings() + .getSettings() + .getAsBoolean( + IndexSettings.INDEX_CONCURRENT_SEGMENT_SEARCH_SETTING.getKey(), + clusterService.getClusterSettings().get(CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING) + ); + } else { + return false; + } + } } diff --git a/server/src/main/java/org/opensearch/search/internal/FilteredSearchContext.java b/server/src/main/java/org/opensearch/search/internal/FilteredSearchContext.java index bb990e69e7722..02e6568369e16 100644 --- a/server/src/main/java/org/opensearch/search/internal/FilteredSearchContext.java +++ b/server/src/main/java/org/opensearch/search/internal/FilteredSearchContext.java @@ -559,4 +559,9 @@ public void setBucketCollectorProcessor(BucketCollectorProcessor bucketCollector public BucketCollectorProcessor bucketCollectorProcessor() { return in.bucketCollectorProcessor(); } + + @Override + public boolean isConcurrentSegmentSearchEnabled() { + return in.isConcurrentSegmentSearchEnabled(); + } } diff --git a/server/src/test/java/org/opensearch/search/SearchServiceTests.java b/server/src/test/java/org/opensearch/search/SearchServiceTests.java index 876a2d15cad7e..8643941970fe0 100644 --- a/server/src/test/java/org/opensearch/search/SearchServiceTests.java +++ b/server/src/test/java/org/opensearch/search/SearchServiceTests.java @@ -1275,8 +1275,77 @@ public void testConcurrentSegmentSearchSearchContext() throws IOException { .getSetting(index, IndexSettings.INDEX_CONCURRENT_SEGMENT_SEARCH_SETTING.getKey()) ); assertEquals(concurrentSearchEnabled, searchContext.isConcurrentSegmentSearchEnabled()); + // verify executor nullability with concurrent search enabled/disabled + if (concurrentSearchEnabled) { + assertNotNull(searchContext.searcher().getExecutor()); + } else { + assertNull(searchContext.searcher().getExecutor()); + } + } + } + // Cleanup + client().admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings(Settings.builder().putNull(SearchService.CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING.getKey())) + .get(); + } + + /** + * Test that the Search Context for concurrent segment search enabled is set correctly at the time of construction. + * The same is used throughout the context object lifetime even if cluster setting changes before the request completion. + */ + public void testConcurrentSegmentSearchIsSetOnceDuringContextCreation() throws IOException { + String index = randomAlphaOfLengthBetween(5, 10).toLowerCase(Locale.ROOT); + IndexService indexService = createIndex(index); + final SearchService service = getInstanceFromNode(SearchService.class); + ShardId shardId = new ShardId(indexService.index(), 0); + long nowInMillis = System.currentTimeMillis(); + String clusterAlias = randomBoolean() ? null : randomAlphaOfLengthBetween(3, 10); + SearchRequest searchRequest = new SearchRequest(); + searchRequest.allowPartialSearchResults(randomBoolean()); + ShardSearchRequest request = new ShardSearchRequest( + OriginalIndices.NONE, + searchRequest, + shardId, + indexService.numberOfShards(), + AliasFilter.EMPTY, + 1f, + nowInMillis, + clusterAlias, + Strings.EMPTY_ARRAY + ); + + Boolean[] concurrentSearchStates = new Boolean[] { true, false }; + for (Boolean concurrentSearchSetting : concurrentSearchStates) { + // update concurrent search cluster setting and create search context + client().admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings( + Settings.builder().put(SearchService.CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING.getKey(), concurrentSearchSetting) + ) + .get(); + try (DefaultSearchContext searchContext = service.createSearchContext(request, new TimeValue(System.currentTimeMillis()))) { + // verify concurrent search state in context + assertEquals(concurrentSearchSetting, searchContext.isConcurrentSegmentSearchEnabled()); + // verify executor state in searcher + assertEquals(concurrentSearchSetting, (searchContext.searcher().getExecutor() != null)); + + // update cluster setting to flip the concurrent segment search state + client().admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings( + Settings.builder().put(SearchService.CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING.getKey(), !concurrentSearchSetting) + ) + .get(); + + // verify that concurrent segment search is still set to same expected value for the context + assertEquals(concurrentSearchSetting, searchContext.isConcurrentSegmentSearchEnabled()); } } + // Cleanup client().admin() .cluster() From 6e51c475c30ca1a0f0875f9e8574242e4bbd6c9d Mon Sep 17 00:00:00 2001 From: Andrew Ross Date: Thu, 3 Aug 2023 13:07:09 -0700 Subject: [PATCH 04/19] Remove empty if/try-finally in base integ test class (#9098) The [commit to comment out `forceFailure()`][1] was done about 8 years ago so I don't think we'll be bringing it back. Also this makes the code base a slightly friendlier place :) [1]: https://github.com/opensearch-project/OpenSearch/commit/e5a699fa05a6e4ffcb919e8a2a6da8f57e1c2ce8 Signed-off-by: Andrew Ross --- .../test/OpenSearchIntegTestCase.java | 52 ++++++++----------- 1 file changed, 21 insertions(+), 31 deletions(-) diff --git a/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java b/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java index 45d69703456fc..422f6d8dfbe7d 100644 --- a/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java +++ b/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java @@ -590,44 +590,34 @@ private static void clearClusters() throws Exception { } private void afterInternal(boolean afterClass) throws Exception { - boolean success = false; + final Scope currentClusterScope = getCurrentClusterScope(); + if (isInternalCluster()) { + internalCluster().clearDisruptionScheme(); + } try { - final Scope currentClusterScope = getCurrentClusterScope(); - if (isInternalCluster()) { - internalCluster().clearDisruptionScheme(); - } - try { - if (cluster() != null) { - if (currentClusterScope != Scope.TEST) { - Metadata metadata = client().admin().cluster().prepareState().execute().actionGet().getState().getMetadata(); + if (cluster() != null) { + if (currentClusterScope != Scope.TEST) { + Metadata metadata = client().admin().cluster().prepareState().execute().actionGet().getState().getMetadata(); - final Set persistentKeys = new HashSet<>(metadata.persistentSettings().keySet()); - assertThat("test leaves persistent cluster metadata behind", persistentKeys, empty()); + final Set persistentKeys = new HashSet<>(metadata.persistentSettings().keySet()); + assertThat("test leaves persistent cluster metadata behind", persistentKeys, empty()); - final Set transientKeys = new HashSet<>(metadata.transientSettings().keySet()); - assertThat("test leaves transient cluster metadata behind", transientKeys, empty()); - } - ensureClusterSizeConsistency(); - ensureClusterStateConsistency(); - ensureClusterStateCanBeReadByNodeTool(); - beforeIndexDeletion(); - cluster().wipe(excludeTemplates()); // wipe after to make sure we fail in the test that didn't ack the delete - if (afterClass || currentClusterScope == Scope.TEST) { - cluster().close(); - } - cluster().assertAfterTest(); + final Set transientKeys = new HashSet<>(metadata.transientSettings().keySet()); + assertThat("test leaves transient cluster metadata behind", transientKeys, empty()); } - } finally { - if (currentClusterScope == Scope.TEST) { - clearClusters(); // it is ok to leave persistent / transient cluster state behind if scope is TEST + ensureClusterSizeConsistency(); + ensureClusterStateConsistency(); + ensureClusterStateCanBeReadByNodeTool(); + beforeIndexDeletion(); + cluster().wipe(excludeTemplates()); // wipe after to make sure we fail in the test that didn't ack the delete + if (afterClass || currentClusterScope == Scope.TEST) { + cluster().close(); } + cluster().assertAfterTest(); } - success = true; } finally { - if (!success) { - // if we failed here that means that something broke horribly so we should clear all clusters - // TODO: just let the exception happen, WTF is all this horseshit - // afterTestRule.forceFailure(); + if (currentClusterScope == Scope.TEST) { + clearClusters(); // it is ok to leave persistent / transient cluster state behind if scope is TEST } } } From 9357b6192e3c107844b4cbe9bacd3502a472e58d Mon Sep 17 00:00:00 2001 From: Suraj Singh Date: Thu, 3 Aug 2023 14:42:15 -0700 Subject: [PATCH 05/19] [Remote Store] Add missing unit test in main (#9101) Signed-off-by: Suraj Singh --- .../RemoteSegmentStoreDirectoryTests.java | 29 +++++++++++++++++++ 1 file changed, 29 insertions(+) diff --git a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java index 3b2e33388925a..f2d39b2ac7bee 100644 --- a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java @@ -758,6 +758,35 @@ public void testUploadMetadataNonEmpty() throws IOException { } } + public void testUploadMetadataMissingSegment() throws IOException { + populateMetadata(); + remoteSegmentStoreDirectory.init(); + + Directory storeDirectory = mock(Directory.class); + IndexOutput indexOutput = mock(IndexOutput.class); + + String generation = RemoteStoreUtils.invertLong(segmentInfos.getGeneration()); + long primaryTermLong = indexShard.getLatestReplicationCheckpoint().getPrimaryTerm(); + String primaryTerm = RemoteStoreUtils.invertLong(primaryTermLong); + when(storeDirectory.createOutput(startsWith("metadata__" + primaryTerm + "__" + generation), eq(IOContext.DEFAULT))).thenReturn( + indexOutput + ); + + Collection segmentFiles = List.of("_123.si"); + assertThrows( + NoSuchFileException.class, + () -> remoteSegmentStoreDirectory.uploadMetadata( + segmentFiles, + segmentInfos, + storeDirectory, + 12L, + indexShard.getLatestReplicationCheckpoint() + ) + ); + verify(indexOutput).close(); + verify(storeDirectory).deleteFile(startsWith("metadata__" + primaryTerm + "__" + generation)); + } + public void testUploadMetadataNoSegmentCommitInfos() throws IOException { SegmentInfos segInfos = indexShard.store().readLastCommittedSegmentsInfo(); int numSegCommitInfos = segInfos.size(); From 24595c9fbace3516e886137058f3a8b992937652 Mon Sep 17 00:00:00 2001 From: Neetika Singhal Date: Thu, 3 Aug 2023 17:31:08 -0700 Subject: [PATCH 06/19] Make MultiBucketConsumerService thread safe to use across slices during search (#9047) Signed-off-by: Neetika Singhal --- CHANGELOG.md | 2 +- .../MultiBucketConsumerService.java | 54 ++++++++++++-- .../MultiBucketConsumerTests.java | 72 +++++++++++++++++++ 3 files changed, 121 insertions(+), 7 deletions(-) create mode 100644 server/src/test/java/org/opensearch/search/aggregations/MultiBucketConsumerTests.java diff --git a/CHANGELOG.md b/CHANGELOG.md index e29bbd2da4db5..3b81cd3a60deb 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -110,7 +110,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Create separate SourceLookup instance per segment slice in SignificantTextAggregatorFactory ([#8807](https://github.com/opensearch-project/OpenSearch/pull/8807)) - Add support for aggregation profiler with concurrent aggregation ([#8801](https://github.com/opensearch-project/OpenSearch/pull/8801)) - [Remove] Deprecated Fractional ByteSizeValue support #9005 ([#9005](https://github.com/opensearch-project/OpenSearch/pull/9005)) - +- Make MultiBucketConsumerService thread safe to use across slices during search ([#9047](https://github.com/opensearch-project/OpenSearch/pull/9047)) ### Deprecated ### Removed diff --git a/server/src/main/java/org/opensearch/search/aggregations/MultiBucketConsumerService.java b/server/src/main/java/org/opensearch/search/aggregations/MultiBucketConsumerService.java index f1416fddebfa2..825e37b7cd952 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/MultiBucketConsumerService.java +++ b/server/src/main/java/org/opensearch/search/aggregations/MultiBucketConsumerService.java @@ -33,6 +33,7 @@ import org.opensearch.cluster.service.ClusterService; import org.opensearch.core.common.breaker.CircuitBreaker; +import org.opensearch.core.common.breaker.CircuitBreakingException; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.common.settings.Setting; @@ -42,6 +43,7 @@ import org.opensearch.search.aggregations.bucket.BucketsAggregator; import java.io.IOException; +import java.util.concurrent.atomic.LongAdder; import java.util.function.IntConsumer; /** @@ -127,13 +129,36 @@ public static class MultiBucketConsumer implements IntConsumer { private final int limit; private final CircuitBreaker breaker; - // aggregations execute in a single thread so no atomic here + // aggregations execute in a single thread for both sequential + // and concurrent search, so no atomic here private int count; - private int callCount = 0; + + // will be updated by multiple threads in concurrent search + // hence making it as LongAdder + private final LongAdder callCount; + private volatile boolean circuitBreakerTripped; + private final int availProcessors; public MultiBucketConsumer(int limit, CircuitBreaker breaker) { this.limit = limit; this.breaker = breaker; + callCount = new LongAdder(); + availProcessors = Runtime.getRuntime().availableProcessors(); + } + + // only visible for testing + protected MultiBucketConsumer( + int limit, + CircuitBreaker breaker, + LongAdder callCount, + boolean circuitBreakerTripped, + int availProcessors + ) { + this.limit = limit; + this.breaker = breaker; + this.callCount = callCount; + this.circuitBreakerTripped = circuitBreakerTripped; + this.availProcessors = availProcessors; } @Override @@ -153,10 +178,27 @@ public void accept(int value) { ); } } - // check parent circuit breaker every 1024 calls - callCount++; - if ((callCount & 0x3FF) == 0) { - breaker.addEstimateBytesAndMaybeBreak(0, "allocated_buckets"); + callCount.increment(); + // tripping the circuit breaker for other threads in case of concurrent search + // if the circuit breaker has tripped for one of the threads already, more info + // can be found on: https://github.com/opensearch-project/OpenSearch/issues/7785 + if (circuitBreakerTripped) { + throw new CircuitBreakingException( + "Circuit breaker for this consumer has already been tripped by previous invocations. " + + "This can happen in case of concurrent segment search when multiple threads are " + + "executing the request and one of the thread has already tripped the circuit breaker", + breaker.getDurability() + ); + } + // check parent circuit breaker every 1024 to (1024 + available processors) calls + long sum = callCount.sum(); + if ((sum >= 1024) && (sum & 0x3FF) <= availProcessors) { + try { + breaker.addEstimateBytesAndMaybeBreak(0, "allocated_buckets"); + } catch (CircuitBreakingException e) { + circuitBreakerTripped = true; + throw e; + } } } diff --git a/server/src/test/java/org/opensearch/search/aggregations/MultiBucketConsumerTests.java b/server/src/test/java/org/opensearch/search/aggregations/MultiBucketConsumerTests.java new file mode 100644 index 0000000000000..eda705f95bf9b --- /dev/null +++ b/server/src/test/java/org/opensearch/search/aggregations/MultiBucketConsumerTests.java @@ -0,0 +1,72 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.search.aggregations; + +import org.mockito.Mockito; +import org.opensearch.core.common.breaker.CircuitBreaker; +import org.opensearch.core.common.breaker.CircuitBreakingException; +import org.opensearch.test.OpenSearchTestCase; + +import java.util.concurrent.atomic.LongAdder; + +import static org.opensearch.search.aggregations.MultiBucketConsumerService.DEFAULT_MAX_BUCKETS; + +public class MultiBucketConsumerTests extends OpenSearchTestCase { + + public void testMultiConsumerAcceptWhenCBTripped() { + CircuitBreaker breaker = Mockito.mock(CircuitBreaker.class); + MultiBucketConsumerService.MultiBucketConsumer multiBucketConsumer = new MultiBucketConsumerService.MultiBucketConsumer( + DEFAULT_MAX_BUCKETS, + breaker, + new LongAdder(), + true, + 1 + ); + // exception is thrown upfront since the circuit breaker has already tripped + expectThrows(CircuitBreakingException.class, () -> multiBucketConsumer.accept(0)); + Mockito.verify(breaker, Mockito.times(0)).addEstimateBytesAndMaybeBreak(0, "allocated_buckets"); + } + + public void testMultiConsumerAcceptToTripCB() { + CircuitBreaker breaker = Mockito.mock(CircuitBreaker.class); + LongAdder callCount = new LongAdder(); + for (int i = 0; i < 1024; i++) { + callCount.increment(); + } + MultiBucketConsumerService.MultiBucketConsumer multiBucketConsumer = new MultiBucketConsumerService.MultiBucketConsumer( + DEFAULT_MAX_BUCKETS, + breaker, + callCount, + false, + 2 + ); + // circuit breaker check is performed as the value of call count would be 1025 which is still in range + Mockito.when(breaker.addEstimateBytesAndMaybeBreak(0, "allocated_buckets")).thenThrow(CircuitBreakingException.class); + expectThrows(CircuitBreakingException.class, () -> multiBucketConsumer.accept(0)); + Mockito.verify(breaker, Mockito.times(1)).addEstimateBytesAndMaybeBreak(0, "allocated_buckets"); + } + + public void testMultiConsumerAccept() { + CircuitBreaker breaker = Mockito.mock(CircuitBreaker.class); + LongAdder callCount = new LongAdder(); + for (int i = 0; i < 1100; i++) { + callCount.increment(); + } + MultiBucketConsumerService.MultiBucketConsumer multiBucketConsumer = new MultiBucketConsumerService.MultiBucketConsumer( + DEFAULT_MAX_BUCKETS, + breaker, + callCount, + false, + 1 + ); + // no exception is thrown as the call count value is not in the expected range and CB is not checked + multiBucketConsumer.accept(0); + Mockito.verify(breaker, Mockito.times(0)).addEstimateBytesAndMaybeBreak(0, "allocated_buckets"); + } +} From 5bb7fa3ead94051f9fe3e8772f8f093d422a356d Mon Sep 17 00:00:00 2001 From: Nick Knize Date: Fri, 4 Aug 2023 14:55:34 -0500 Subject: [PATCH 07/19] [Refactor] CollectionUtils, Iterators, and String methods to core lib (#9120) This commit refactors the CollectionUtils, and Iterators utility classes to the core and common libraries, respectively. Several other base Strings utility methods are also refactored to the core Strings utility library. This is done to incrementally move the remaining Strings methods in server to the core String utility. Signed-off-by: Nicholas Walter Knize --- .../opensearch/client/RequestConverters.java | 2 +- .../client/ClusterRequestConvertersTests.java | 2 +- .../client/IndicesRequestConvertersTests.java | 2 +- .../opensearch/common/collect/Iterators.java | 0 .../common/collect/IteratorsTests.java | 2 + .../org/opensearch/core/common/Strings.java | 225 +++++++++++++----- .../core}/common/util/CollectionUtils.java | 6 +- .../opensearch/core/common/StringsTests.java | 38 +++ .../geo/algorithm/PolygonGenerator.java | 2 +- .../ingest/common/ScriptProcessor.java | 2 +- .../CustomReflectionObjectHandler.java | 2 +- .../painless/lookup/PainlessClass.java | 2 +- .../painless/lookup/PainlessLookup.java | 2 +- .../AbstractAzureComputeServiceTestCase.java | 2 +- .../RemoveCorruptedShardDataCommandIT.java | 2 +- .../index/store/CorruptedFileIT.java | 2 +- .../recovery/RecoveryWhileUnderLoadIT.java | 2 +- .../bucket/terms/BaseStringTermsTestCase.java | 2 +- .../metrics/HDRPercentileRanksIT.java | 2 +- .../metrics/HDRPercentilesIT.java | 2 +- .../metrics/TDigestPercentileRanksIT.java | 2 +- .../metrics/TDigestPercentilesIT.java | 2 +- .../suggest/CompletionSuggestSearchIT.java | 2 +- .../snapshots/SearchableSnapshotIT.java | 2 +- .../search/grouping/CollapseTopFieldDocs.java | 2 +- .../health/TransportClusterHealthAction.java | 2 +- .../TransportSnapshotsStatusAction.java | 2 +- .../indices/close/CloseIndexRequest.java | 2 +- .../indices/close/CloseIndexResponse.java | 2 +- .../datastream/DeleteDataStreamAction.java | 2 +- .../indices/delete/DeleteIndexRequest.java | 2 +- .../mapping/put/PutMappingRequest.java | 6 +- .../admin/indices/open/OpenIndexRequest.java | 2 +- .../readonly/AddIndexBlockRequest.java | 2 +- .../readonly/AddIndexBlockResponse.java | 2 +- .../get/TransportGetSettingsAction.java | 2 +- .../search/SearchPhaseExecutionException.java | 2 +- .../support/tasks/BaseTasksRequest.java | 2 +- .../metadata/IndexNameExpressionResolver.java | 2 +- .../routing/RotationShardShuffler.java | 2 +- .../java/org/opensearch/common/Strings.java | 113 --------- .../opensearch/common/logging/Loggers.java | 2 +- .../org/opensearch/common/regex/Regex.java | 4 +- .../identity/tokens/RestTokenExtractor.java | 2 +- .../org/opensearch/index/IndexingSlowLog.java | 2 +- .../index/analysis/CustomAnalyzer.java | 2 +- .../analysis/ReloadableCustomAnalyzer.java | 2 +- .../index/engine/CompletionStatsCache.java | 2 +- .../index/fielddata/ShardFieldData.java | 2 +- .../index/mapper/BinaryFieldMapper.java | 2 +- .../index/mapper/IndexFieldMapper.java | 2 +- .../index/mapper/SourceFieldMapper.java | 2 +- .../index/query/GeoValidationMethod.java | 2 +- .../org/opensearch/index/query/Operator.java | 2 +- .../index/query/SimpleQueryStringBuilder.java | 2 +- .../reindex/ClientScrollableHitSource.java | 2 +- .../index/search/stats/ShardSearchStats.java | 2 +- .../opensearch/indices/IndicesService.java | 2 +- .../java/org/opensearch/rest/RestRequest.java | 13 +- .../opensearch/rest/action/RestActions.java | 2 +- .../rest/action/cat/RestNodeAttrsAction.java | 2 +- .../rest/action/cat/RestNodesAction.java | 2 +- .../rest/action/cat/RestTasksAction.java | 2 +- .../org/opensearch/search/SearchService.java | 2 +- .../metrics/InternalScriptedMetric.java | 2 +- .../metrics/ScriptedMetricAggregator.java | 2 +- .../aggregations/support/ValuesSource.java | 2 +- .../support/values/ScriptBytesValues.java | 2 +- .../fetch/subphase/ScriptFieldsPhase.java | 2 +- .../highlight/FastVectorHighlighter.java | 2 +- .../java/org/opensearch/tasks/TaskId.java | 2 +- .../org/opensearch/watcher/FileWatcher.java | 2 +- ...dateSettingsRequestSerializationTests.java | 2 +- .../allocation/AddIncrementallyTests.java | 2 +- .../org/opensearch/common/StringsTests.java | 45 +--- .../common/collect/EvictingQueueTests.java | 2 +- .../common/util/CollectionUtilsTests.java | 3 +- .../common/xcontent/BaseXContentTestCase.java | 2 +- .../query/MoreLikeThisQueryBuilderTests.java | 2 +- ...archAllocationWithConstraintsTestCase.java | 2 +- .../coordination/LinearizabilityChecker.java | 2 +- .../test/OpenSearchIntegTestCase.java | 2 +- .../opensearch/test/OpenSearchTestCase.java | 2 +- 83 files changed, 302 insertions(+), 297 deletions(-) rename {server => libs/common}/src/main/java/org/opensearch/common/collect/Iterators.java (100%) rename {server => libs/common}/src/test/java/org/opensearch/common/collect/IteratorsTests.java (98%) rename {server/src/main/java/org/opensearch => libs/core/src/main/java/org/opensearch/core}/common/util/CollectionUtils.java (98%) diff --git a/client/rest-high-level/src/main/java/org/opensearch/client/RequestConverters.java b/client/rest-high-level/src/main/java/org/opensearch/client/RequestConverters.java index d23a5976fada6..c48acfc5b6ece 100644 --- a/client/rest-high-level/src/main/java/org/opensearch/client/RequestConverters.java +++ b/client/rest-high-level/src/main/java/org/opensearch/client/RequestConverters.java @@ -76,7 +76,7 @@ import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.common.lucene.uid.Versions; import org.opensearch.common.unit.TimeValue; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.common.xcontent.XContentHelper; import org.opensearch.common.xcontent.XContentType; import org.opensearch.core.common.Strings; diff --git a/client/rest-high-level/src/test/java/org/opensearch/client/ClusterRequestConvertersTests.java b/client/rest-high-level/src/test/java/org/opensearch/client/ClusterRequestConvertersTests.java index f201599632969..5ac3969a6e887 100644 --- a/client/rest-high-level/src/test/java/org/opensearch/client/ClusterRequestConvertersTests.java +++ b/client/rest-high-level/src/test/java/org/opensearch/client/ClusterRequestConvertersTests.java @@ -40,7 +40,7 @@ import org.opensearch.client.cluster.RemoteInfoRequest; import org.opensearch.cluster.health.ClusterHealthStatus; import org.opensearch.common.Priority; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.test.OpenSearchTestCase; import org.apache.hc.client5.http.classic.methods.HttpGet; import org.apache.hc.client5.http.classic.methods.HttpPut; diff --git a/client/rest-high-level/src/test/java/org/opensearch/client/IndicesRequestConvertersTests.java b/client/rest-high-level/src/test/java/org/opensearch/client/IndicesRequestConvertersTests.java index c672ed6be110d..75c5a71303af4 100644 --- a/client/rest-high-level/src/test/java/org/opensearch/client/IndicesRequestConvertersTests.java +++ b/client/rest-high-level/src/test/java/org/opensearch/client/IndicesRequestConvertersTests.java @@ -74,7 +74,7 @@ import org.opensearch.common.CheckedFunction; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.common.xcontent.XContentType; import org.opensearch.core.common.Strings; import org.opensearch.test.OpenSearchTestCase; diff --git a/server/src/main/java/org/opensearch/common/collect/Iterators.java b/libs/common/src/main/java/org/opensearch/common/collect/Iterators.java similarity index 100% rename from server/src/main/java/org/opensearch/common/collect/Iterators.java rename to libs/common/src/main/java/org/opensearch/common/collect/Iterators.java diff --git a/server/src/test/java/org/opensearch/common/collect/IteratorsTests.java b/libs/common/src/test/java/org/opensearch/common/collect/IteratorsTests.java similarity index 98% rename from server/src/test/java/org/opensearch/common/collect/IteratorsTests.java rename to libs/common/src/test/java/org/opensearch/common/collect/IteratorsTests.java index 6ad272542dbb1..2578926454c0b 100644 --- a/server/src/test/java/org/opensearch/common/collect/IteratorsTests.java +++ b/libs/common/src/test/java/org/opensearch/common/collect/IteratorsTests.java @@ -83,6 +83,7 @@ public void testRandomSingleton() { int numberOfIterators = randomIntBetween(1, 1000); int singletonIndex = randomIntBetween(0, numberOfIterators - 1); int value = randomInt(); + @SuppressWarnings("rawtypes") Iterator[] iterators = new Iterator[numberOfIterators]; for (int i = 0; i < numberOfIterators; i++) { iterators[i] = i != singletonIndex ? empty() : singletonIterator(value); @@ -92,6 +93,7 @@ public void testRandomSingleton() { public void testRandomIterators() { int numberOfIterators = randomIntBetween(1, 1000); + @SuppressWarnings("rawtypes") Iterator[] iterators = new Iterator[numberOfIterators]; List values = new ArrayList<>(); for (int i = 0; i < numberOfIterators; i++) { diff --git a/libs/core/src/main/java/org/opensearch/core/common/Strings.java b/libs/core/src/main/java/org/opensearch/core/common/Strings.java index ffb7711f84492..2e610f6047fd4 100644 --- a/libs/core/src/main/java/org/opensearch/core/common/Strings.java +++ b/libs/core/src/main/java/org/opensearch/core/common/Strings.java @@ -9,6 +9,7 @@ package org.opensearch.core.common; import org.opensearch.common.Nullable; +import org.opensearch.core.common.util.CollectionUtils; import java.io.BufferedReader; import java.io.StringReader; @@ -35,18 +36,6 @@ public class Strings { public static final String UNKNOWN_UUID_VALUE = "_na_"; public static final String[] EMPTY_ARRAY = new String[0]; - /** - * Split the specified string by commas to an array. - * - * @param s the string to split - * @return the array of split values - * @see String#split(String) - */ - public static String[] splitStringByCommaToArray(final String s) { - if (s == null || s.isEmpty()) return Strings.EMPTY_ARRAY; - else return s.split(","); - } - /** * Convenience method to return a Collection as a delimited (e.g. CSV) * String. E.g. useful for toString() implementations. @@ -341,47 +330,6 @@ public static String[] toStringArray(final Collection collection) { return collection.toArray(new String[0]); } - /** - * Take a String which is a delimited list and convert it to a String array. - *

A single delimiter can consists of more than one character: It will still - * be considered as single delimiter string, rather than as bunch of potential - * delimiter characters - in contrast to tokenizeToStringArray. - * - * @param str the input String - * @param delimiter the delimiter between elements (this is a single delimiter, - * rather than a bunch individual delimiter characters) - * @param charsToDelete a set of characters to delete. Useful for deleting unwanted - * line breaks: e.g. "\r\n\f" will delete all new lines and line feeds in a String. - * @return an array of the tokens in the list - * @see #tokenizeToStringArray - */ - public static String[] delimitedListToStringArray(String str, String delimiter, String charsToDelete) { - if (str == null) { - return Strings.EMPTY_ARRAY; - } - if (delimiter == null) { - return new String[] { str }; - } - List result = new ArrayList<>(); - if ("".equals(delimiter)) { - for (int i = 0; i < str.length(); i++) { - result.add(deleteAny(str.substring(i, i + 1), charsToDelete)); - } - } else { - int pos = 0; - int delPos; - while ((delPos = str.indexOf(delimiter, pos)) != -1) { - result.add(deleteAny(str.substring(pos, delPos), charsToDelete)); - pos = delPos + delimiter.length(); - } - if (str.length() > 0 && pos <= str.length()) { - // Add rest of String, but not in case of empty input. - result.add(deleteAny(str.substring(pos), charsToDelete)); - } - } - return toStringArray(result); - } - /** * Tokenize the specified string by commas to a set, trimming whitespace and ignoring empty tokens. * @@ -393,6 +341,41 @@ public static Set tokenizeByCommaToSet(final String s) { return tokenizeToCollection(s, ",", HashSet::new); } + /** + * Split the specified string by commas to an array. + * + * @param s the string to split + * @return the array of split values + * @see String#split(String) + */ + public static String[] splitStringByCommaToArray(final String s) { + if (s == null || s.isEmpty()) return Strings.EMPTY_ARRAY; + else return s.split(","); + } + + /** + * Split a String at the first occurrence of the delimiter. + * Does not include the delimiter in the result. + * + * @param toSplit the string to split + * @param delimiter to split the string up with + * @return a two element array with index 0 being before the delimiter, and + * index 1 being after the delimiter (neither element includes the delimiter); + * or null if the delimiter wasn't found in the given input String + */ + public static String[] split(String toSplit, String delimiter) { + if (hasLength(toSplit) == false || hasLength(delimiter) == false) { + return null; + } + int offset = toSplit.indexOf(delimiter); + if (offset < 0) { + return null; + } + String beforeDelimiter = toSplit.substring(0, offset); + String afterDelimiter = toSplit.substring(offset + delimiter.length()); + return new String[] { beforeDelimiter, afterDelimiter }; + } + /** * Tokenize the given String into a String array via a StringTokenizer. * Trims tokens and omits empty tokens. @@ -446,6 +429,47 @@ private static > T tokenizeToCollection( return tokens; } + /** + * Take a String which is a delimited list and convert it to a String array. + *

A single delimiter can consists of more than one character: It will still + * be considered as single delimiter string, rather than as bunch of potential + * delimiter characters - in contrast to tokenizeToStringArray. + * + * @param str the input String + * @param delimiter the delimiter between elements (this is a single delimiter, + * rather than a bunch individual delimiter characters) + * @param charsToDelete a set of characters to delete. Useful for deleting unwanted + * line breaks: e.g. "\r\n\f" will delete all new lines and line feeds in a String. + * @return an array of the tokens in the list + * @see #tokenizeToStringArray + */ + public static String[] delimitedListToStringArray(String str, String delimiter, String charsToDelete) { + if (str == null) { + return Strings.EMPTY_ARRAY; + } + if (delimiter == null) { + return new String[] { str }; + } + List result = new ArrayList<>(); + if ("".equals(delimiter)) { + for (int i = 0; i < str.length(); i++) { + result.add(deleteAny(str.substring(i, i + 1), charsToDelete)); + } + } else { + int pos = 0; + int delPos; + while ((delPos = str.indexOf(delimiter, pos)) != -1) { + result.add(deleteAny(str.substring(pos, delPos), charsToDelete)); + pos = delPos + delimiter.length(); + } + if (str.length() > 0 && pos <= str.length()) { + // Add rest of String, but not in case of empty input. + result.add(deleteAny(str.substring(pos), charsToDelete)); + } + } + return toStringArray(result); + } + /** * Take a String which is a delimited list and convert it to a String array. *

A single delimiter can consists of more than one character: It will still @@ -486,10 +510,6 @@ public static Set commaDelimitedListToSet(String str) { return set; } - public static boolean isNullOrEmpty(@Nullable String s) { - return s == null || s.isEmpty(); - } - /** * Capitalize a String, changing the first letter to * upper case as per {@link Character#toUpperCase(char)}. @@ -538,4 +558,95 @@ public static String format1Decimals(double value, String suffix) { } } } + + /** + * Return substring(beginIndex, endIndex) that is impervious to string length. + */ + public static String substring(String s, int beginIndex, int endIndex) { + if (s == null) { + return s; + } + + int realEndIndex = s.length() > 0 ? s.length() - 1 : 0; + + if (endIndex > realEndIndex) { + return s.substring(beginIndex); + } else { + return s.substring(beginIndex, endIndex); + } + } + + /** + * If an array only consists of zero or one element, which is "*" or "_all" return an empty array + * which is usually used as everything + */ + public static boolean isAllOrWildcard(String[] data) { + return CollectionUtils.isEmpty(data) || data.length == 1 && isAllOrWildcard(data[0]); + } + + /** + * Returns `true` if the string is `_all` or `*`. + */ + public static boolean isAllOrWildcard(String data) { + return "_all".equals(data) || "*".equals(data); + } + + /** + * Truncates string to a length less than length. Backtracks to throw out + * high surrogates. + */ + public static String cleanTruncate(String s, int length) { + if (s == null) { + return s; + } + /* + * Its pretty silly for you to truncate to 0 length but just in case + * someone does this shouldn't break. + */ + if (length == 0) { + return ""; + } + if (length >= s.length()) { + return s; + } + if (Character.isHighSurrogate(s.charAt(length - 1))) { + length--; + } + return s.substring(0, length); + } + + public static boolean isNullOrEmpty(@Nullable String s) { + return s == null || s.isEmpty(); + } + + public static String padStart(String s, int minimumLength, char c) { + if (s == null) { + throw new NullPointerException("s"); + } + if (s.length() >= minimumLength) { + return s; + } else { + StringBuilder sb = new StringBuilder(minimumLength); + for (int i = s.length(); i < minimumLength; i++) { + sb.append(c); + } + + sb.append(s); + return sb.toString(); + } + } + + public static String toLowercaseAscii(String in) { + StringBuilder out = new StringBuilder(); + Iterator iter = in.codePoints().iterator(); + while (iter.hasNext()) { + int codepoint = iter.next(); + if (codepoint > 128) { + out.appendCodePoint(codepoint); + } else { + out.appendCodePoint(Character.toLowerCase(codepoint)); + } + } + return out.toString(); + } } diff --git a/server/src/main/java/org/opensearch/common/util/CollectionUtils.java b/libs/core/src/main/java/org/opensearch/core/common/util/CollectionUtils.java similarity index 98% rename from server/src/main/java/org/opensearch/common/util/CollectionUtils.java rename to libs/core/src/main/java/org/opensearch/core/common/util/CollectionUtils.java index 2037b2e46488f..5cb8b900814b9 100644 --- a/server/src/main/java/org/opensearch/common/util/CollectionUtils.java +++ b/libs/core/src/main/java/org/opensearch/core/common/util/CollectionUtils.java @@ -30,7 +30,7 @@ * GitHub history for details. */ -package org.opensearch.common.util; +package org.opensearch.core.common.util; import org.opensearch.common.collect.Iterators; import org.opensearch.core.common.Strings; @@ -217,6 +217,7 @@ public int size() { } } + @SuppressWarnings("unchecked") public static ArrayList iterableAsArrayList(Iterable elements) { if (elements == null) { throw new NullPointerException("elements"); @@ -232,6 +233,7 @@ public static ArrayList iterableAsArrayList(Iterable element } } + @SuppressWarnings("unchecked") public static ArrayList arrayAsArrayList(E... elements) { if (elements == null) { throw new NullPointerException("elements"); @@ -239,6 +241,7 @@ public static ArrayList arrayAsArrayList(E... elements) { return new ArrayList<>(Arrays.asList(elements)); } + @SuppressWarnings("unchecked") public static ArrayList asArrayList(E first, E... other) { if (other == null) { throw new NullPointerException("other"); @@ -249,6 +252,7 @@ public static ArrayList asArrayList(E first, E... other) { return list; } + @SuppressWarnings("unchecked") public static ArrayList asArrayList(E first, E second, E... other) { if (other == null) { throw new NullPointerException("other"); diff --git a/libs/core/src/test/java/org/opensearch/core/common/StringsTests.java b/libs/core/src/test/java/org/opensearch/core/common/StringsTests.java index 532251e02e685..ca0bd788c1991 100644 --- a/libs/core/src/test/java/org/opensearch/core/common/StringsTests.java +++ b/libs/core/src/test/java/org/opensearch/core/common/StringsTests.java @@ -11,8 +11,46 @@ import org.opensearch.common.util.set.Sets; import org.opensearch.test.OpenSearchTestCase; +import static org.hamcrest.Matchers.is; + /** tests for Strings utility class */ public class StringsTests extends OpenSearchTestCase { + public void testIsAllOrWildCardString() { + assertThat(Strings.isAllOrWildcard("_all"), is(true)); + assertThat(Strings.isAllOrWildcard("*"), is(true)); + assertThat(Strings.isAllOrWildcard("foo"), is(false)); + assertThat(Strings.isAllOrWildcard(""), is(false)); + assertThat(Strings.isAllOrWildcard((String) null), is(false)); + } + + public void testSubstring() { + assertEquals(null, Strings.substring(null, 0, 1000)); + assertEquals("foo", Strings.substring("foo", 0, 1000)); + assertEquals("foo", Strings.substring("foo", 0, 3)); + assertEquals("oo", Strings.substring("foo", 1, 3)); + assertEquals("oo", Strings.substring("foo", 1, 100)); + assertEquals("f", Strings.substring("foo", 0, 1)); + } + + public void testCleanTruncate() { + assertEquals(null, Strings.cleanTruncate(null, 10)); + assertEquals("foo", Strings.cleanTruncate("foo", 10)); + assertEquals("foo", Strings.cleanTruncate("foo", 3)); + // Throws out high surrogates + assertEquals("foo", Strings.cleanTruncate("foo\uD83D\uDEAB", 4)); + // But will keep the whole character + assertEquals("foo\uD83D\uDEAB", Strings.cleanTruncate("foo\uD83D\uDEAB", 5)); + /* + * Doesn't take care around combining marks. This example has its + * meaning changed because that last codepoint is supposed to combine + * backwards into the find "o" and be represented as the "o" with a + * circle around it with a slash through it. As in "no 'o's allowed + * here. + */ + assertEquals("o", org.opensearch.core.common.Strings.cleanTruncate("o\uD83D\uDEAB", 1)); + assertEquals("", org.opensearch.core.common.Strings.cleanTruncate("foo", 0)); + } + public void testSplitStringToSet() { assertEquals(Strings.tokenizeByCommaToSet(null), Sets.newHashSet()); assertEquals(Strings.tokenizeByCommaToSet(""), Sets.newHashSet()); diff --git a/modules/geo/src/main/java/org/opensearch/geo/algorithm/PolygonGenerator.java b/modules/geo/src/main/java/org/opensearch/geo/algorithm/PolygonGenerator.java index 246ece4342cff..da1d97260ec96 100644 --- a/modules/geo/src/main/java/org/opensearch/geo/algorithm/PolygonGenerator.java +++ b/modules/geo/src/main/java/org/opensearch/geo/algorithm/PolygonGenerator.java @@ -10,7 +10,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import java.awt.geom.Point2D; import java.util.ArrayList; diff --git a/modules/ingest-common/src/main/java/org/opensearch/ingest/common/ScriptProcessor.java b/modules/ingest-common/src/main/java/org/opensearch/ingest/common/ScriptProcessor.java index 5f61091495cd5..b66d0b709a824 100644 --- a/modules/ingest-common/src/main/java/org/opensearch/ingest/common/ScriptProcessor.java +++ b/modules/ingest-common/src/main/java/org/opensearch/ingest/common/ScriptProcessor.java @@ -34,7 +34,7 @@ import org.opensearch.common.Nullable; import org.opensearch.core.common.bytes.BytesReference; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.common.xcontent.LoggingDeprecationHandler; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.core.xcontent.XContentBuilder; diff --git a/modules/lang-mustache/src/main/java/org/opensearch/script/mustache/CustomReflectionObjectHandler.java b/modules/lang-mustache/src/main/java/org/opensearch/script/mustache/CustomReflectionObjectHandler.java index 57451a027c5d7..0936dee0c3016 100644 --- a/modules/lang-mustache/src/main/java/org/opensearch/script/mustache/CustomReflectionObjectHandler.java +++ b/modules/lang-mustache/src/main/java/org/opensearch/script/mustache/CustomReflectionObjectHandler.java @@ -33,7 +33,7 @@ package org.opensearch.script.mustache; import com.github.mustachejava.reflect.ReflectionObjectHandler; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.common.util.iterable.Iterables; import java.lang.reflect.Array; diff --git a/modules/lang-painless/src/main/java/org/opensearch/painless/lookup/PainlessClass.java b/modules/lang-painless/src/main/java/org/opensearch/painless/lookup/PainlessClass.java index efa2d51524557..fdf7df94252b6 100644 --- a/modules/lang-painless/src/main/java/org/opensearch/painless/lookup/PainlessClass.java +++ b/modules/lang-painless/src/main/java/org/opensearch/painless/lookup/PainlessClass.java @@ -32,7 +32,7 @@ package org.opensearch.painless.lookup; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import java.lang.invoke.MethodHandle; import java.util.Map; diff --git a/modules/lang-painless/src/main/java/org/opensearch/painless/lookup/PainlessLookup.java b/modules/lang-painless/src/main/java/org/opensearch/painless/lookup/PainlessLookup.java index 1249a9cffecb2..9a3b8bf9e2eee 100644 --- a/modules/lang-painless/src/main/java/org/opensearch/painless/lookup/PainlessLookup.java +++ b/modules/lang-painless/src/main/java/org/opensearch/painless/lookup/PainlessLookup.java @@ -32,7 +32,7 @@ package org.opensearch.painless.lookup; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import java.lang.invoke.MethodHandle; import java.util.Map; diff --git a/plugins/discovery-azure-classic/src/internalClusterTest/java/org/opensearch/cloud/azure/classic/AbstractAzureComputeServiceTestCase.java b/plugins/discovery-azure-classic/src/internalClusterTest/java/org/opensearch/cloud/azure/classic/AbstractAzureComputeServiceTestCase.java index d5d6aae23f344..9a6327a5c88ba 100644 --- a/plugins/discovery-azure-classic/src/internalClusterTest/java/org/opensearch/cloud/azure/classic/AbstractAzureComputeServiceTestCase.java +++ b/plugins/discovery-azure-classic/src/internalClusterTest/java/org/opensearch/cloud/azure/classic/AbstractAzureComputeServiceTestCase.java @@ -63,7 +63,7 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; -import static org.opensearch.common.util.CollectionUtils.newSingletonArrayList; +import static org.opensearch.core.common.util.CollectionUtils.newSingletonArrayList; import static org.opensearch.discovery.DiscoveryModule.DISCOVERY_SEED_PROVIDERS_SETTING; public abstract class AbstractAzureComputeServiceTestCase extends OpenSearchIntegTestCase { diff --git a/server/src/internalClusterTest/java/org/opensearch/index/shard/RemoveCorruptedShardDataCommandIT.java b/server/src/internalClusterTest/java/org/opensearch/index/shard/RemoveCorruptedShardDataCommandIT.java index df2c8c62ca392..986155b99217e 100644 --- a/server/src/internalClusterTest/java/org/opensearch/index/shard/RemoveCorruptedShardDataCommandIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/index/shard/RemoveCorruptedShardDataCommandIT.java @@ -105,7 +105,7 @@ import java.util.stream.StreamSupport; import static org.opensearch.action.admin.cluster.node.stats.NodesStatsRequest.Metric.FS; -import static org.opensearch.common.util.CollectionUtils.iterableAsArrayList; +import static org.opensearch.core.common.util.CollectionUtils.iterableAsArrayList; import static org.opensearch.index.query.QueryBuilders.matchAllQuery; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertHitCount; diff --git a/server/src/internalClusterTest/java/org/opensearch/index/store/CorruptedFileIT.java b/server/src/internalClusterTest/java/org/opensearch/index/store/CorruptedFileIT.java index 89031f68aba97..52ee6fd10ced7 100644 --- a/server/src/internalClusterTest/java/org/opensearch/index/store/CorruptedFileIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/index/store/CorruptedFileIT.java @@ -112,7 +112,7 @@ import java.util.stream.Collectors; import static org.opensearch.action.admin.cluster.node.stats.NodesStatsRequest.Metric.FS; -import static org.opensearch.common.util.CollectionUtils.iterableAsArrayList; +import static org.opensearch.core.common.util.CollectionUtils.iterableAsArrayList; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAllSuccessful; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertHitCount; diff --git a/server/src/internalClusterTest/java/org/opensearch/recovery/RecoveryWhileUnderLoadIT.java b/server/src/internalClusterTest/java/org/opensearch/recovery/RecoveryWhileUnderLoadIT.java index a675eb7c77344..5b72eaca0b4e5 100644 --- a/server/src/internalClusterTest/java/org/opensearch/recovery/RecoveryWhileUnderLoadIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/recovery/RecoveryWhileUnderLoadIT.java @@ -46,7 +46,7 @@ import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.index.IndexService; import org.opensearch.index.IndexSettings; import org.opensearch.index.shard.DocsStats; diff --git a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/terms/BaseStringTermsTestCase.java b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/terms/BaseStringTermsTestCase.java index 7775618ba5b13..f7195c7cb4d2c 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/terms/BaseStringTermsTestCase.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/terms/BaseStringTermsTestCase.java @@ -11,7 +11,7 @@ import org.junit.After; import org.junit.Before; import org.opensearch.action.index.IndexRequestBuilder; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.index.fielddata.ScriptDocValues; import org.opensearch.plugins.Plugin; import org.opensearch.search.aggregations.AggregationTestScriptsPlugin; diff --git a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/metrics/HDRPercentileRanksIT.java b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/metrics/HDRPercentileRanksIT.java index 20fc6aaee20c9..6a62dabd98bce 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/metrics/HDRPercentileRanksIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/metrics/HDRPercentileRanksIT.java @@ -53,7 +53,7 @@ import java.util.Map; import static java.util.Collections.emptyMap; -import static org.opensearch.common.util.CollectionUtils.iterableAsArrayList; +import static org.opensearch.core.common.util.CollectionUtils.iterableAsArrayList; import static org.opensearch.index.query.QueryBuilders.matchAllQuery; import static org.opensearch.index.query.QueryBuilders.termQuery; import static org.opensearch.search.aggregations.AggregationBuilders.filter; diff --git a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/metrics/HDRPercentilesIT.java b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/metrics/HDRPercentilesIT.java index 2660dbe0a88ed..ad3fd6517d1b1 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/metrics/HDRPercentilesIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/metrics/HDRPercentilesIT.java @@ -34,7 +34,7 @@ import org.apache.logging.log4j.LogManager; import org.opensearch.action.search.SearchResponse; import org.opensearch.common.settings.Settings; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.plugins.Plugin; import org.opensearch.script.Script; import org.opensearch.script.ScriptType; diff --git a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/metrics/TDigestPercentileRanksIT.java b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/metrics/TDigestPercentileRanksIT.java index adf027222d7d9..602cfe344b496 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/metrics/TDigestPercentileRanksIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/metrics/TDigestPercentileRanksIT.java @@ -34,7 +34,7 @@ import org.apache.logging.log4j.LogManager; import org.opensearch.action.search.SearchResponse; import org.opensearch.common.settings.Settings; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.plugins.Plugin; import org.opensearch.script.Script; import org.opensearch.script.ScriptType; diff --git a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/metrics/TDigestPercentilesIT.java b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/metrics/TDigestPercentilesIT.java index fda15f9b90ea2..2c05ed0bac44a 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/metrics/TDigestPercentilesIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/metrics/TDigestPercentilesIT.java @@ -34,7 +34,7 @@ import org.apache.logging.log4j.LogManager; import org.opensearch.action.search.SearchResponse; import org.opensearch.common.settings.Settings; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.plugins.Plugin; import org.opensearch.script.Script; import org.opensearch.script.ScriptType; diff --git a/server/src/internalClusterTest/java/org/opensearch/search/suggest/CompletionSuggestSearchIT.java b/server/src/internalClusterTest/java/org/opensearch/search/suggest/CompletionSuggestSearchIT.java index 8eb957733944d..764d40eb7bc8e 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/suggest/CompletionSuggestSearchIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/suggest/CompletionSuggestSearchIT.java @@ -78,7 +78,7 @@ import static org.opensearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_SHARDS; -import static org.opensearch.common.util.CollectionUtils.iterableAsArrayList; +import static org.opensearch.core.common.util.CollectionUtils.iterableAsArrayList; import static org.opensearch.common.xcontent.XContentFactory.jsonBuilder; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAllSuccessful; diff --git a/server/src/internalClusterTest/java/org/opensearch/snapshots/SearchableSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/snapshots/SearchableSnapshotIT.java index efd072fce511d..2a9f8cf67a607 100644 --- a/server/src/internalClusterTest/java/org/opensearch/snapshots/SearchableSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/snapshots/SearchableSnapshotIT.java @@ -56,7 +56,7 @@ import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; import static org.opensearch.action.admin.cluster.node.stats.NodesStatsRequest.Metric.FS; -import static org.opensearch.common.util.CollectionUtils.iterableAsArrayList; +import static org.opensearch.core.common.util.CollectionUtils.iterableAsArrayList; @ThreadLeakFilters(filters = CleanerDaemonThreadLeakFilter.class) public final class SearchableSnapshotIT extends AbstractSnapshotIntegTestCase { diff --git a/server/src/main/java/org/apache/lucene/search/grouping/CollapseTopFieldDocs.java b/server/src/main/java/org/apache/lucene/search/grouping/CollapseTopFieldDocs.java index fe26c313d72b2..e93e5cdcc3f7b 100644 --- a/server/src/main/java/org/apache/lucene/search/grouping/CollapseTopFieldDocs.java +++ b/server/src/main/java/org/apache/lucene/search/grouping/CollapseTopFieldDocs.java @@ -39,7 +39,7 @@ import org.apache.lucene.search.TopFieldDocs; import org.apache.lucene.search.TotalHits; import org.apache.lucene.util.PriorityQueue; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import java.util.ArrayList; import java.util.HashSet; diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/health/TransportClusterHealthAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/health/TransportClusterHealthAction.java index d84179fa6bc0a..09f6b7cd06cda 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/health/TransportClusterHealthAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/health/TransportClusterHealthAction.java @@ -60,7 +60,7 @@ import org.opensearch.common.inject.Inject; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.common.unit.TimeValue; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.core.common.Strings; import org.opensearch.discovery.ClusterManagerNotDiscoveredException; import org.opensearch.discovery.Discovery; diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java index 78952805e5b49..b49c18082d0dc 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/TransportSnapshotsStatusAction.java @@ -48,7 +48,7 @@ import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.inject.Inject; import org.opensearch.core.common.io.stream.StreamInput; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.common.util.set.Sets; import org.opensearch.core.common.Strings; import org.opensearch.core.index.shard.ShardId; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/close/CloseIndexRequest.java b/server/src/main/java/org/opensearch/action/admin/indices/close/CloseIndexRequest.java index 8c6dde80d8d97..0f98550343a13 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/close/CloseIndexRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/close/CloseIndexRequest.java @@ -39,7 +39,7 @@ import org.opensearch.action.support.master.AcknowledgedRequest; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/close/CloseIndexResponse.java b/server/src/main/java/org/opensearch/action/admin/indices/close/CloseIndexResponse.java index 8e58ee92db80a..a3c599a54a10c 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/close/CloseIndexResponse.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/close/CloseIndexResponse.java @@ -39,7 +39,7 @@ import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.common.xcontent.XContentType; import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.XContentBuilder; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/datastream/DeleteDataStreamAction.java b/server/src/main/java/org/opensearch/action/admin/indices/datastream/DeleteDataStreamAction.java index e5ae7cd582481..5f5be3c166dbc 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/datastream/DeleteDataStreamAction.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/datastream/DeleteDataStreamAction.java @@ -59,7 +59,7 @@ import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.common.regex.Regex; import org.opensearch.common.unit.TimeValue; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.core.common.Strings; import org.opensearch.core.index.Index; import org.opensearch.snapshots.SnapshotInProgressException; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/delete/DeleteIndexRequest.java b/server/src/main/java/org/opensearch/action/admin/indices/delete/DeleteIndexRequest.java index 35fb6a8748941..006da6b3cbb09 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/delete/DeleteIndexRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/delete/DeleteIndexRequest.java @@ -38,7 +38,7 @@ import org.opensearch.action.support.master.AcknowledgedRequest; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/mapping/put/PutMappingRequest.java b/server/src/main/java/org/opensearch/action/admin/indices/mapping/put/PutMappingRequest.java index 2588dfd718e71..373331eb1554b 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/mapping/put/PutMappingRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/mapping/put/PutMappingRequest.java @@ -43,7 +43,7 @@ import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.common.xcontent.XContentFactory; import org.opensearch.common.xcontent.XContentHelper; import org.opensearch.common.xcontent.XContentType; @@ -251,7 +251,7 @@ public static XContentBuilder simpleMapping(String... source) { builder.startObject(fieldName); String[] s1 = Strings.splitStringByCommaToArray(source[i]); for (String s : s1) { - String[] s2 = org.opensearch.common.Strings.split(s, "="); + String[] s2 = Strings.split(s, "="); if (s2.length != 2) { throw new IllegalArgumentException("malformed " + s); } @@ -271,7 +271,7 @@ public static XContentBuilder simpleMapping(String... source) { builder.startObject(fieldName); String[] s1 = Strings.splitStringByCommaToArray(source[i]); for (String s : s1) { - String[] s2 = org.opensearch.common.Strings.split(s, "="); + String[] s2 = Strings.split(s, "="); if (s2.length != 2) { throw new IllegalArgumentException("malformed " + s); } diff --git a/server/src/main/java/org/opensearch/action/admin/indices/open/OpenIndexRequest.java b/server/src/main/java/org/opensearch/action/admin/indices/open/OpenIndexRequest.java index d42f3699765e7..16451e311e7d3 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/open/OpenIndexRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/open/OpenIndexRequest.java @@ -39,7 +39,7 @@ import org.opensearch.action.support.master.AcknowledgedRequest; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import java.io.IOException; import java.util.Arrays; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/readonly/AddIndexBlockRequest.java b/server/src/main/java/org/opensearch/action/admin/indices/readonly/AddIndexBlockRequest.java index ca5a17f0a1520..b5097f96fe52b 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/readonly/AddIndexBlockRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/readonly/AddIndexBlockRequest.java @@ -39,7 +39,7 @@ import org.opensearch.cluster.metadata.IndexMetadata.APIBlock; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import java.io.IOException; import java.util.Objects; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/readonly/AddIndexBlockResponse.java b/server/src/main/java/org/opensearch/action/admin/indices/readonly/AddIndexBlockResponse.java index 39268d752c8c5..f90bdaca510ae 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/readonly/AddIndexBlockResponse.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/readonly/AddIndexBlockResponse.java @@ -39,7 +39,7 @@ import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.common.xcontent.XContentType; import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.XContentBuilder; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/settings/get/TransportGetSettingsAction.java b/server/src/main/java/org/opensearch/action/admin/indices/settings/get/TransportGetSettingsAction.java index 22cbcf804f9d2..d3bf275823278 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/settings/get/TransportGetSettingsAction.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/settings/get/TransportGetSettingsAction.java @@ -47,7 +47,7 @@ import org.opensearch.common.settings.IndexScopedSettings; import org.opensearch.common.settings.Settings; import org.opensearch.common.settings.SettingsFilter; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.core.index.Index; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportService; diff --git a/server/src/main/java/org/opensearch/action/search/SearchPhaseExecutionException.java b/server/src/main/java/org/opensearch/action/search/SearchPhaseExecutionException.java index b3ed42824e91a..998ba720c4523 100644 --- a/server/src/main/java/org/opensearch/action/search/SearchPhaseExecutionException.java +++ b/server/src/main/java/org/opensearch/action/search/SearchPhaseExecutionException.java @@ -37,7 +37,7 @@ import org.opensearch.core.action.ShardOperationFailedException; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.rest.RestStatus; diff --git a/server/src/main/java/org/opensearch/action/support/tasks/BaseTasksRequest.java b/server/src/main/java/org/opensearch/action/support/tasks/BaseTasksRequest.java index 43ece159247bc..f8d96d57b341b 100644 --- a/server/src/main/java/org/opensearch/action/support/tasks/BaseTasksRequest.java +++ b/server/src/main/java/org/opensearch/action/support/tasks/BaseTasksRequest.java @@ -38,7 +38,7 @@ import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.common.regex.Regex; import org.opensearch.common.unit.TimeValue; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.core.common.Strings; import org.opensearch.tasks.Task; import org.opensearch.tasks.TaskId; diff --git a/server/src/main/java/org/opensearch/cluster/metadata/IndexNameExpressionResolver.java b/server/src/main/java/org/opensearch/cluster/metadata/IndexNameExpressionResolver.java index 52a76e9dd90f1..c7ef959d6a080 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/IndexNameExpressionResolver.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/IndexNameExpressionResolver.java @@ -44,7 +44,7 @@ import org.opensearch.common.time.DateFormatter; import org.opensearch.common.time.DateMathParser; import org.opensearch.common.time.DateUtils; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.common.util.concurrent.ThreadContext; import org.opensearch.common.util.set.Sets; import org.opensearch.core.common.Strings; diff --git a/server/src/main/java/org/opensearch/cluster/routing/RotationShardShuffler.java b/server/src/main/java/org/opensearch/cluster/routing/RotationShardShuffler.java index c49ad09b85344..a15d96e9adc99 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/RotationShardShuffler.java +++ b/server/src/main/java/org/opensearch/cluster/routing/RotationShardShuffler.java @@ -32,7 +32,7 @@ package org.opensearch.cluster.routing; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; diff --git a/server/src/main/java/org/opensearch/common/Strings.java b/server/src/main/java/org/opensearch/common/Strings.java index e9f4d32ed2664..e3404d6ffb297 100644 --- a/server/src/main/java/org/opensearch/common/Strings.java +++ b/server/src/main/java/org/opensearch/common/Strings.java @@ -36,14 +36,12 @@ import org.opensearch.ExceptionsHelper; import org.opensearch.OpenSearchException; import org.opensearch.core.common.bytes.BytesReference; -import org.opensearch.common.util.CollectionUtils; import org.opensearch.core.xcontent.MediaType; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.XContentBuilder; import java.io.IOException; import java.util.Arrays; -import java.util.Iterator; import java.util.Set; import static java.util.Collections.unmodifiableSet; @@ -131,30 +129,6 @@ public static boolean validFileNameExcludingAstrix(String fileName) { return true; } - /** - * Split a String at the first occurrence of the delimiter. - * Does not include the delimiter in the result. - * - * @param toSplit the string to split - * @param delimiter to split the string up with - * @return a two element array with index 0 being before the delimiter, and - * index 1 being after the delimiter (neither element includes the delimiter); - * or null if the delimiter wasn't found in the given input String - */ - public static String[] split(String toSplit, String delimiter) { - if (org.opensearch.core.common.Strings.hasLength(toSplit) == false - || org.opensearch.core.common.Strings.hasLength(delimiter) == false) { - return null; - } - int offset = toSplit.indexOf(delimiter); - if (offset < 0) { - return null; - } - String beforeDelimiter = toSplit.substring(0, offset); - String afterDelimiter = toSplit.substring(offset + delimiter.length()); - return new String[] { beforeDelimiter, afterDelimiter }; - } - private Strings() {} public static byte[] toUTF8Bytes(CharSequence charSequence) { @@ -166,38 +140,6 @@ public static byte[] toUTF8Bytes(CharSequence charSequence, BytesRefBuilder spar return Arrays.copyOf(spare.bytes(), spare.length()); } - /** - * Return substring(beginIndex, endIndex) that is impervious to string length. - */ - public static String substring(String s, int beginIndex, int endIndex) { - if (s == null) { - return s; - } - - int realEndIndex = s.length() > 0 ? s.length() - 1 : 0; - - if (endIndex > realEndIndex) { - return s.substring(beginIndex); - } else { - return s.substring(beginIndex, endIndex); - } - } - - /** - * If an array only consists of zero or one element, which is "*" or "_all" return an empty array - * which is usually used as everything - */ - public static boolean isAllOrWildcard(String[] data) { - return CollectionUtils.isEmpty(data) || data.length == 1 && isAllOrWildcard(data[0]); - } - - /** - * Returns `true` if the string is `_all` or `*`. - */ - public static boolean isAllOrWildcard(String data) { - return "_all".equals(data) || "*".equals(data); - } - /** * Return a {@link String} that is the json representation of the provided {@link ToXContent}. * Wraps the output into an anonymous object if needed. The content is not pretty-printed @@ -276,59 +218,4 @@ private static XContentBuilder createBuilder(MediaType mediaType, boolean pretty } return builder; } - - /** - * Truncates string to a length less than length. Backtracks to throw out - * high surrogates. - */ - public static String cleanTruncate(String s, int length) { - if (s == null) { - return s; - } - /* - * Its pretty silly for you to truncate to 0 length but just in case - * someone does this shouldn't break. - */ - if (length == 0) { - return ""; - } - if (length >= s.length()) { - return s; - } - if (Character.isHighSurrogate(s.charAt(length - 1))) { - length--; - } - return s.substring(0, length); - } - - public static String padStart(String s, int minimumLength, char c) { - if (s == null) { - throw new NullPointerException("s"); - } - if (s.length() >= minimumLength) { - return s; - } else { - StringBuilder sb = new StringBuilder(minimumLength); - for (int i = s.length(); i < minimumLength; i++) { - sb.append(c); - } - - sb.append(s); - return sb.toString(); - } - } - - public static String toLowercaseAscii(String in) { - StringBuilder out = new StringBuilder(); - Iterator iter = in.codePoints().iterator(); - while (iter.hasNext()) { - int codepoint = iter.next(); - if (codepoint > 128) { - out.appendCodePoint(codepoint); - } else { - out.appendCodePoint(Character.toLowerCase(codepoint)); - } - } - return out.toString(); - } } diff --git a/server/src/main/java/org/opensearch/common/logging/Loggers.java b/server/src/main/java/org/opensearch/common/logging/Loggers.java index 7b19bd4b4be25..9a88afef90867 100644 --- a/server/src/main/java/org/opensearch/common/logging/Loggers.java +++ b/server/src/main/java/org/opensearch/common/logging/Loggers.java @@ -46,7 +46,7 @@ import java.util.Map; -import static org.opensearch.common.util.CollectionUtils.asArrayList; +import static org.opensearch.core.common.util.CollectionUtils.asArrayList; /** * A set of utilities around Logging. diff --git a/server/src/main/java/org/opensearch/common/regex/Regex.java b/server/src/main/java/org/opensearch/common/regex/Regex.java index 14716fdff6d2e..396af77c8a751 100644 --- a/server/src/main/java/org/opensearch/common/regex/Regex.java +++ b/server/src/main/java/org/opensearch/common/regex/Regex.java @@ -122,8 +122,8 @@ public static boolean simpleMatch(String pattern, String str, boolean caseInsens return false; } if (caseInsensitive) { - pattern = org.opensearch.common.Strings.toLowercaseAscii(pattern); - str = org.opensearch.common.Strings.toLowercaseAscii(str); + pattern = Strings.toLowercaseAscii(pattern); + str = Strings.toLowercaseAscii(str); } return simpleMatchWithNormalizedStrings(pattern, str); } diff --git a/server/src/main/java/org/opensearch/identity/tokens/RestTokenExtractor.java b/server/src/main/java/org/opensearch/identity/tokens/RestTokenExtractor.java index ae200c7461a60..2d1d4d619f44d 100644 --- a/server/src/main/java/org/opensearch/identity/tokens/RestTokenExtractor.java +++ b/server/src/main/java/org/opensearch/identity/tokens/RestTokenExtractor.java @@ -9,7 +9,7 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.LogManager; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.rest.RestRequest; import java.util.Collections; diff --git a/server/src/main/java/org/opensearch/index/IndexingSlowLog.java b/server/src/main/java/org/opensearch/index/IndexingSlowLog.java index 86c5d4542e5fa..1899c9c77f26a 100644 --- a/server/src/main/java/org/opensearch/index/IndexingSlowLog.java +++ b/server/src/main/java/org/opensearch/index/IndexingSlowLog.java @@ -36,13 +36,13 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.util.StringBuilders; import org.opensearch.common.Booleans; -import org.opensearch.common.Strings; import org.opensearch.common.logging.OpenSearchLogMessage; import org.opensearch.common.logging.Loggers; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Setting.Property; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.xcontent.XContentHelper; +import org.opensearch.core.common.Strings; import org.opensearch.core.index.Index; import org.opensearch.index.engine.Engine; import org.opensearch.index.mapper.ParsedDocument; diff --git a/server/src/main/java/org/opensearch/index/analysis/CustomAnalyzer.java b/server/src/main/java/org/opensearch/index/analysis/CustomAnalyzer.java index 8db5c89d75808..b0f4687c0b8b1 100644 --- a/server/src/main/java/org/opensearch/index/analysis/CustomAnalyzer.java +++ b/server/src/main/java/org/opensearch/index/analysis/CustomAnalyzer.java @@ -35,7 +35,7 @@ import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.TokenStream; import org.apache.lucene.analysis.Tokenizer; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import java.io.Reader; diff --git a/server/src/main/java/org/opensearch/index/analysis/ReloadableCustomAnalyzer.java b/server/src/main/java/org/opensearch/index/analysis/ReloadableCustomAnalyzer.java index ba8996eb9c17b..c1bfca93b90f7 100644 --- a/server/src/main/java/org/opensearch/index/analysis/ReloadableCustomAnalyzer.java +++ b/server/src/main/java/org/opensearch/index/analysis/ReloadableCustomAnalyzer.java @@ -37,7 +37,7 @@ import org.apache.lucene.analysis.Tokenizer; import org.apache.lucene.util.CloseableThreadLocal; import org.opensearch.common.settings.Settings; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import java.io.Reader; import java.util.Map; diff --git a/server/src/main/java/org/opensearch/index/engine/CompletionStatsCache.java b/server/src/main/java/org/opensearch/index/engine/CompletionStatsCache.java index c6cf7e8c1b53f..fb4e868a8903b 100644 --- a/server/src/main/java/org/opensearch/index/engine/CompletionStatsCache.java +++ b/server/src/main/java/org/opensearch/index/engine/CompletionStatsCache.java @@ -42,7 +42,7 @@ import org.opensearch.common.FieldMemoryStats; import org.opensearch.common.Nullable; import org.opensearch.common.regex.Regex; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.search.suggest.completion.CompletionStats; import java.util.HashMap; diff --git a/server/src/main/java/org/opensearch/index/fielddata/ShardFieldData.java b/server/src/main/java/org/opensearch/index/fielddata/ShardFieldData.java index 138b417571784..16f0fb90ddc19 100644 --- a/server/src/main/java/org/opensearch/index/fielddata/ShardFieldData.java +++ b/server/src/main/java/org/opensearch/index/fielddata/ShardFieldData.java @@ -36,7 +36,7 @@ import org.opensearch.common.FieldMemoryStats; import org.opensearch.common.metrics.CounterMetric; import org.opensearch.common.regex.Regex; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.common.util.concurrent.ConcurrentCollections; import org.opensearch.core.index.shard.ShardId; diff --git a/server/src/main/java/org/opensearch/index/mapper/BinaryFieldMapper.java b/server/src/main/java/org/opensearch/index/mapper/BinaryFieldMapper.java index a55e55111ffa2..f83ef8aaf6845 100644 --- a/server/src/main/java/org/opensearch/index/mapper/BinaryFieldMapper.java +++ b/server/src/main/java/org/opensearch/index/mapper/BinaryFieldMapper.java @@ -41,7 +41,7 @@ import org.opensearch.core.common.bytes.BytesArray; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.common.io.stream.BytesStreamOutput; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.index.fielddata.IndexFieldData; import org.opensearch.index.fielddata.plain.BytesBinaryIndexFieldData; diff --git a/server/src/main/java/org/opensearch/index/mapper/IndexFieldMapper.java b/server/src/main/java/org/opensearch/index/mapper/IndexFieldMapper.java index 8cc307f7697c6..982b0eb909443 100644 --- a/server/src/main/java/org/opensearch/index/mapper/IndexFieldMapper.java +++ b/server/src/main/java/org/opensearch/index/mapper/IndexFieldMapper.java @@ -34,7 +34,7 @@ import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.Query; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.index.fielddata.IndexFieldData; import org.opensearch.index.fielddata.plain.ConstantIndexFieldData; import org.opensearch.index.query.QueryShardContext; diff --git a/server/src/main/java/org/opensearch/index/mapper/SourceFieldMapper.java b/server/src/main/java/org/opensearch/index/mapper/SourceFieldMapper.java index deda9883bc07b..685d0871d0230 100644 --- a/server/src/main/java/org/opensearch/index/mapper/SourceFieldMapper.java +++ b/server/src/main/java/org/opensearch/index/mapper/SourceFieldMapper.java @@ -41,7 +41,7 @@ import org.opensearch.common.Nullable; import org.opensearch.common.collect.Tuple; import org.opensearch.common.io.stream.BytesStreamOutput; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.common.xcontent.XContentHelper; import org.opensearch.common.xcontent.support.XContentMapValues; import org.opensearch.core.common.bytes.BytesReference; diff --git a/server/src/main/java/org/opensearch/index/query/GeoValidationMethod.java b/server/src/main/java/org/opensearch/index/query/GeoValidationMethod.java index aad3e50a0acd5..559c084325abb 100644 --- a/server/src/main/java/org/opensearch/index/query/GeoValidationMethod.java +++ b/server/src/main/java/org/opensearch/index/query/GeoValidationMethod.java @@ -35,7 +35,7 @@ import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/index/query/Operator.java b/server/src/main/java/org/opensearch/index/query/Operator.java index 31caca0f60caa..ee8c93ce76ecb 100644 --- a/server/src/main/java/org/opensearch/index/query/Operator.java +++ b/server/src/main/java/org/opensearch/index/query/Operator.java @@ -36,7 +36,7 @@ import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import java.io.IOException; import java.util.Locale; diff --git a/server/src/main/java/org/opensearch/index/query/SimpleQueryStringBuilder.java b/server/src/main/java/org/opensearch/index/query/SimpleQueryStringBuilder.java index cf9f86b6f6341..2a759415c2992 100644 --- a/server/src/main/java/org/opensearch/index/query/SimpleQueryStringBuilder.java +++ b/server/src/main/java/org/opensearch/index/query/SimpleQueryStringBuilder.java @@ -40,7 +40,7 @@ import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.common.lucene.search.Queries; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; diff --git a/server/src/main/java/org/opensearch/index/reindex/ClientScrollableHitSource.java b/server/src/main/java/org/opensearch/index/reindex/ClientScrollableHitSource.java index 2f03eb66bd71e..3159d8bfc3c85 100644 --- a/server/src/main/java/org/opensearch/index/reindex/ClientScrollableHitSource.java +++ b/server/src/main/java/org/opensearch/index/reindex/ClientScrollableHitSource.java @@ -62,7 +62,7 @@ import static java.util.Collections.emptyList; import static java.util.Collections.unmodifiableList; import static org.opensearch.common.unit.TimeValue.timeValueNanos; -import static org.opensearch.common.util.CollectionUtils.isEmpty; +import static org.opensearch.core.common.util.CollectionUtils.isEmpty; /** * A scrollable source of hits from a {@linkplain Client} instance. diff --git a/server/src/main/java/org/opensearch/index/search/stats/ShardSearchStats.java b/server/src/main/java/org/opensearch/index/search/stats/ShardSearchStats.java index 6d0eb3a5949ca..6f6ebd5545c7a 100644 --- a/server/src/main/java/org/opensearch/index/search/stats/ShardSearchStats.java +++ b/server/src/main/java/org/opensearch/index/search/stats/ShardSearchStats.java @@ -36,7 +36,7 @@ import org.opensearch.common.metrics.CounterMetric; import org.opensearch.common.metrics.MeanMetric; import org.opensearch.common.regex.Regex; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.index.shard.SearchOperationListener; import org.opensearch.search.internal.ReaderContext; import org.opensearch.search.internal.SearchContext; diff --git a/server/src/main/java/org/opensearch/indices/IndicesService.java b/server/src/main/java/org/opensearch/indices/IndicesService.java index 7be824d95b421..844f4fec706f3 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesService.java +++ b/server/src/main/java/org/opensearch/indices/IndicesService.java @@ -193,7 +193,7 @@ import static java.util.Collections.emptyMap; import static java.util.Collections.unmodifiableMap; import static org.opensearch.common.collect.MapBuilder.newMapBuilder; -import static org.opensearch.common.util.CollectionUtils.arrayAsArrayList; +import static org.opensearch.core.common.util.CollectionUtils.arrayAsArrayList; import static org.opensearch.common.util.concurrent.OpenSearchExecutors.daemonThreadFactory; import static org.opensearch.index.IndexService.IndexCreationContext.CREATE_INDEX; import static org.opensearch.index.IndexService.IndexCreationContext.METADATA_VERIFICATION; diff --git a/server/src/main/java/org/opensearch/rest/RestRequest.java b/server/src/main/java/org/opensearch/rest/RestRequest.java index fb497a17cde72..b61a998276532 100644 --- a/server/src/main/java/org/opensearch/rest/RestRequest.java +++ b/server/src/main/java/org/opensearch/rest/RestRequest.java @@ -37,18 +37,17 @@ import org.opensearch.common.CheckedConsumer; import org.opensearch.common.Nullable; import org.opensearch.common.SetOnce; -import org.opensearch.common.Strings; -import org.opensearch.core.common.bytes.BytesArray; -import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.common.collect.Tuple; -import org.opensearch.core.common.unit.ByteSizeValue; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.xcontent.LoggingDeprecationHandler; +import org.opensearch.core.common.Strings; +import org.opensearch.core.common.bytes.BytesArray; +import org.opensearch.core.common.bytes.BytesReference; +import org.opensearch.core.common.unit.ByteSizeValue; import org.opensearch.core.xcontent.MediaType; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.XContentParser; -import org.opensearch.common.xcontent.XContentType; import org.opensearch.http.HttpChannel; import org.opensearch.http.HttpRequest; @@ -337,7 +336,7 @@ public final long getRequestId() { } /** - * The {@link XContentType} that was parsed from the {@code Content-Type} header. This value will be {@code null} in the case of + * The {@link MediaType} that was parsed from the {@code Content-Type} header. This value will be {@code null} in the case of * a request without a valid {@code Content-Type} header, a request without content ({@link #hasContent()}, or a plain text request */ @Nullable @@ -462,7 +461,7 @@ public String[] paramAsStringArray(String key, String[] defaultValue) { if (value == null) { return defaultValue; } - return org.opensearch.core.common.Strings.splitStringByCommaToArray(value); + return Strings.splitStringByCommaToArray(value); } public String[] paramAsStringArrayOrEmptyIfAll(String key) { diff --git a/server/src/main/java/org/opensearch/rest/action/RestActions.java b/server/src/main/java/org/opensearch/rest/action/RestActions.java index 7508871d3ab75..fc3d595afa70e 100644 --- a/server/src/main/java/org/opensearch/rest/action/RestActions.java +++ b/server/src/main/java/org/opensearch/rest/action/RestActions.java @@ -41,7 +41,7 @@ import org.opensearch.core.ParseField; import org.opensearch.core.common.ParsingException; import org.opensearch.common.lucene.uid.Versions; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.ToXContent.Params; import org.opensearch.core.xcontent.XContentBuilder; diff --git a/server/src/main/java/org/opensearch/rest/action/cat/RestNodeAttrsAction.java b/server/src/main/java/org/opensearch/rest/action/cat/RestNodeAttrsAction.java index 7b84b3f655522..5ead69320fefd 100644 --- a/server/src/main/java/org/opensearch/rest/action/cat/RestNodeAttrsAction.java +++ b/server/src/main/java/org/opensearch/rest/action/cat/RestNodeAttrsAction.java @@ -40,9 +40,9 @@ import org.opensearch.client.node.NodeClient; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodes; -import org.opensearch.common.Strings; import org.opensearch.common.Table; import org.opensearch.common.logging.DeprecationLogger; +import org.opensearch.core.common.Strings; import org.opensearch.monitor.process.ProcessInfo; import org.opensearch.rest.RestRequest; import org.opensearch.rest.RestResponse; diff --git a/server/src/main/java/org/opensearch/rest/action/cat/RestNodesAction.java b/server/src/main/java/org/opensearch/rest/action/cat/RestNodesAction.java index e29898624386c..5a0f85fea8e5d 100644 --- a/server/src/main/java/org/opensearch/rest/action/cat/RestNodesAction.java +++ b/server/src/main/java/org/opensearch/rest/action/cat/RestNodesAction.java @@ -44,10 +44,10 @@ import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodeRole; import org.opensearch.cluster.node.DiscoveryNodes; -import org.opensearch.common.Strings; import org.opensearch.common.Table; import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.network.NetworkAddress; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.transport.TransportAddress; import org.opensearch.core.common.unit.ByteSizeValue; import org.opensearch.http.HttpInfo; diff --git a/server/src/main/java/org/opensearch/rest/action/cat/RestTasksAction.java b/server/src/main/java/org/opensearch/rest/action/cat/RestTasksAction.java index d30086fb8cd55..560b88787ae09 100644 --- a/server/src/main/java/org/opensearch/rest/action/cat/RestTasksAction.java +++ b/server/src/main/java/org/opensearch/rest/action/cat/RestTasksAction.java @@ -38,10 +38,10 @@ import org.opensearch.client.node.NodeClient; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodes; -import org.opensearch.common.Strings; import org.opensearch.common.Table; import org.opensearch.common.time.DateFormatter; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import org.opensearch.rest.RestRequest; import org.opensearch.rest.RestResponse; import org.opensearch.rest.action.RestResponseListener; diff --git a/server/src/main/java/org/opensearch/search/SearchService.java b/server/src/main/java/org/opensearch/search/SearchService.java index b244290e8ae74..0a7e355e29c4d 100644 --- a/server/src/main/java/org/opensearch/search/SearchService.java +++ b/server/src/main/java/org/opensearch/search/SearchService.java @@ -63,7 +63,7 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.BigArrays; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.common.util.concurrent.ConcurrentCollections; import org.opensearch.common.util.concurrent.ConcurrentMapLong; import org.opensearch.common.util.io.IOUtils; diff --git a/server/src/main/java/org/opensearch/search/aggregations/metrics/InternalScriptedMetric.java b/server/src/main/java/org/opensearch/search/aggregations/metrics/InternalScriptedMetric.java index 6150940512406..fbcf4a4d48603 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/metrics/InternalScriptedMetric.java +++ b/server/src/main/java/org/opensearch/search/aggregations/metrics/InternalScriptedMetric.java @@ -34,7 +34,7 @@ import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.script.Script; import org.opensearch.script.ScriptedMetricAggContexts; diff --git a/server/src/main/java/org/opensearch/search/aggregations/metrics/ScriptedMetricAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/metrics/ScriptedMetricAggregator.java index f60f402939462..99a546ae4050a 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/metrics/ScriptedMetricAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/metrics/ScriptedMetricAggregator.java @@ -37,7 +37,7 @@ import org.apache.lucene.search.ScoreMode; import org.opensearch.common.Nullable; import org.opensearch.core.common.io.stream.StreamOutput; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.common.util.ObjectArray; import org.opensearch.common.lease.Releasables; import org.opensearch.script.Script; diff --git a/server/src/main/java/org/opensearch/search/aggregations/support/ValuesSource.java b/server/src/main/java/org/opensearch/search/aggregations/support/ValuesSource.java index e53118669385b..67d311db964d6 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/support/ValuesSource.java +++ b/server/src/main/java/org/opensearch/search/aggregations/support/ValuesSource.java @@ -44,7 +44,7 @@ import org.opensearch.common.Rounding; import org.opensearch.common.Rounding.Prepared; import org.opensearch.common.lucene.ScorerAware; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.index.fielddata.AbstractSortingNumericDocValues; import org.opensearch.index.fielddata.DocValueBits; import org.opensearch.index.fielddata.GeoShapeValue; diff --git a/server/src/main/java/org/opensearch/search/aggregations/support/values/ScriptBytesValues.java b/server/src/main/java/org/opensearch/search/aggregations/support/values/ScriptBytesValues.java index e53d9b83a814c..349bd8e14edf6 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/support/values/ScriptBytesValues.java +++ b/server/src/main/java/org/opensearch/search/aggregations/support/values/ScriptBytesValues.java @@ -33,7 +33,7 @@ import org.apache.lucene.search.Scorable; import org.opensearch.common.lucene.ScorerAware; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.index.fielddata.SortedBinaryDocValues; import org.opensearch.index.fielddata.SortingBinaryDocValues; import org.opensearch.script.AggregationScript; diff --git a/server/src/main/java/org/opensearch/search/fetch/subphase/ScriptFieldsPhase.java b/server/src/main/java/org/opensearch/search/fetch/subphase/ScriptFieldsPhase.java index aad20f0746f58..67d1863050a7b 100644 --- a/server/src/main/java/org/opensearch/search/fetch/subphase/ScriptFieldsPhase.java +++ b/server/src/main/java/org/opensearch/search/fetch/subphase/ScriptFieldsPhase.java @@ -33,7 +33,7 @@ import org.apache.lucene.index.LeafReaderContext; import org.opensearch.common.document.DocumentField; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.script.FieldScript; import org.opensearch.search.fetch.FetchContext; import org.opensearch.search.fetch.FetchSubPhase; diff --git a/server/src/main/java/org/opensearch/search/fetch/subphase/highlight/FastVectorHighlighter.java b/server/src/main/java/org/opensearch/search/fetch/subphase/highlight/FastVectorHighlighter.java index c087ad6b71344..69f86bb91cc6e 100644 --- a/server/src/main/java/org/opensearch/search/fetch/subphase/highlight/FastVectorHighlighter.java +++ b/server/src/main/java/org/opensearch/search/fetch/subphase/highlight/FastVectorHighlighter.java @@ -48,7 +48,7 @@ import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; import org.opensearch.core.common.text.Text; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.index.mapper.MappedFieldType; import org.opensearch.index.mapper.TextSearchInfo; import org.opensearch.search.fetch.FetchSubPhase; diff --git a/server/src/main/java/org/opensearch/tasks/TaskId.java b/server/src/main/java/org/opensearch/tasks/TaskId.java index e7420fce397e2..50e1e0a8c00b5 100644 --- a/server/src/main/java/org/opensearch/tasks/TaskId.java +++ b/server/src/main/java/org/opensearch/tasks/TaskId.java @@ -33,7 +33,7 @@ package org.opensearch.tasks; import org.opensearch.OpenSearchParseException; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; diff --git a/server/src/main/java/org/opensearch/watcher/FileWatcher.java b/server/src/main/java/org/opensearch/watcher/FileWatcher.java index c27ade408e3af..2bf93804c6ecd 100644 --- a/server/src/main/java/org/opensearch/watcher/FileWatcher.java +++ b/server/src/main/java/org/opensearch/watcher/FileWatcher.java @@ -34,7 +34,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.opensearch.core.util.FileSystemUtils; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import java.io.IOException; import java.nio.file.Files; diff --git a/server/src/test/java/org/opensearch/action/admin/indices/settings/put/UpdateSettingsRequestSerializationTests.java b/server/src/test/java/org/opensearch/action/admin/indices/settings/put/UpdateSettingsRequestSerializationTests.java index ae632d22a3310..e7e077c55fdc5 100644 --- a/server/src/test/java/org/opensearch/action/admin/indices/settings/put/UpdateSettingsRequestSerializationTests.java +++ b/server/src/test/java/org/opensearch/action/admin/indices/settings/put/UpdateSettingsRequestSerializationTests.java @@ -37,7 +37,7 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.settings.Settings.Builder; import org.opensearch.common.unit.TimeValue; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.test.AbstractWireSerializingTestCase; import org.opensearch.test.OpenSearchTestCase; diff --git a/server/src/test/java/org/opensearch/cluster/routing/allocation/AddIncrementallyTests.java b/server/src/test/java/org/opensearch/cluster/routing/allocation/AddIncrementallyTests.java index 16ff5f753d286..89cf5fe4f76c3 100644 --- a/server/src/test/java/org/opensearch/cluster/routing/allocation/AddIncrementallyTests.java +++ b/server/src/test/java/org/opensearch/cluster/routing/allocation/AddIncrementallyTests.java @@ -46,7 +46,7 @@ import org.opensearch.cluster.routing.RoutingTable; import org.opensearch.cluster.routing.allocation.decider.ClusterRebalanceAllocationDecider; import org.opensearch.common.settings.Settings; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.hamcrest.Matcher; import org.hamcrest.Matchers; diff --git a/server/src/test/java/org/opensearch/common/StringsTests.java b/server/src/test/java/org/opensearch/common/StringsTests.java index 50f7be8be170d..ea756373181c5 100644 --- a/server/src/test/java/org/opensearch/common/StringsTests.java +++ b/server/src/test/java/org/opensearch/common/StringsTests.java @@ -32,7 +32,7 @@ package org.opensearch.common; -import org.opensearch.common.xcontent.XContentType; +import org.opensearch.core.xcontent.MediaTypeRegistry; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.ToXContentObject; import org.opensearch.test.OpenSearchTestCase; @@ -40,46 +40,9 @@ import java.util.Collections; import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.is; public class StringsTests extends OpenSearchTestCase { - public void testIsAllOrWildCardString() { - assertThat(Strings.isAllOrWildcard("_all"), is(true)); - assertThat(Strings.isAllOrWildcard("*"), is(true)); - assertThat(Strings.isAllOrWildcard("foo"), is(false)); - assertThat(Strings.isAllOrWildcard(""), is(false)); - assertThat(Strings.isAllOrWildcard((String) null), is(false)); - } - - public void testSubstring() { - assertEquals(null, Strings.substring(null, 0, 1000)); - assertEquals("foo", Strings.substring("foo", 0, 1000)); - assertEquals("foo", Strings.substring("foo", 0, 3)); - assertEquals("oo", Strings.substring("foo", 1, 3)); - assertEquals("oo", Strings.substring("foo", 1, 100)); - assertEquals("f", Strings.substring("foo", 0, 1)); - } - - public void testCleanTruncate() { - assertEquals(null, Strings.cleanTruncate(null, 10)); - assertEquals("foo", Strings.cleanTruncate("foo", 10)); - assertEquals("foo", Strings.cleanTruncate("foo", 3)); - // Throws out high surrogates - assertEquals("foo", Strings.cleanTruncate("foo\uD83D\uDEAB", 4)); - // But will keep the whole character - assertEquals("foo\uD83D\uDEAB", Strings.cleanTruncate("foo\uD83D\uDEAB", 5)); - /* - * Doesn't take care around combining marks. This example has its - * meaning changed because that last codepoint is supposed to combine - * backwards into the find "o" and be represented as the "o" with a - * circle around it with a slash through it. As in "no 'o's allowed - * here. - */ - assertEquals("o", Strings.cleanTruncate("o\uD83D\uDEAB", 1)); - assertEquals("", Strings.cleanTruncate("foo", 0)); - } - public void testToStringToXContent() { final ToXContent toXContent; final boolean error; @@ -104,7 +67,7 @@ public void testToStringToXContent() { } } - String toString = Strings.toString(XContentType.JSON, toXContent); + String toString = Strings.toString(MediaTypeRegistry.JSON, toXContent); if (error) { assertThat(toString, containsString("\"error\":\"error building toString out of XContent:")); assertThat(toString, containsString("\"stack_trace\":")); @@ -117,10 +80,10 @@ public void testToStringToXContent() { public void testToStringToXContentWithOrWithoutParams() { ToXContent toXContent = (builder, params) -> builder.field("color_from_param", params.param("color", "red")); // Rely on the default value of "color" param when params are not passed - assertThat(Strings.toString(XContentType.JSON, toXContent), containsString("\"color_from_param\":\"red\"")); + assertThat(Strings.toString(MediaTypeRegistry.JSON, toXContent), containsString("\"color_from_param\":\"red\"")); // Pass "color" param explicitly assertThat( - Strings.toString(XContentType.JSON, toXContent, new ToXContent.MapParams(Collections.singletonMap("color", "blue"))), + Strings.toString(MediaTypeRegistry.JSON, toXContent, new ToXContent.MapParams(Collections.singletonMap("color", "blue"))), containsString("\"color_from_param\":\"blue\"") ); } diff --git a/server/src/test/java/org/opensearch/common/collect/EvictingQueueTests.java b/server/src/test/java/org/opensearch/common/collect/EvictingQueueTests.java index 9a9e0efcdb190..e976697bd59c8 100644 --- a/server/src/test/java/org/opensearch/common/collect/EvictingQueueTests.java +++ b/server/src/test/java/org/opensearch/common/collect/EvictingQueueTests.java @@ -29,7 +29,7 @@ package org.opensearch.common.collect; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.test.OpenSearchTestCase; import java.util.Collections; diff --git a/server/src/test/java/org/opensearch/common/util/CollectionUtilsTests.java b/server/src/test/java/org/opensearch/common/util/CollectionUtilsTests.java index f0a0d41aa7aab..1c629bd77fe93 100644 --- a/server/src/test/java/org/opensearch/common/util/CollectionUtilsTests.java +++ b/server/src/test/java/org/opensearch/common/util/CollectionUtilsTests.java @@ -32,6 +32,7 @@ package org.opensearch.common.util; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.test.OpenSearchTestCase; import java.util.ArrayList; @@ -45,7 +46,7 @@ import java.util.Map; import static java.util.Collections.emptyMap; -import static org.opensearch.common.util.CollectionUtils.eagerPartition; +import static org.opensearch.core.common.util.CollectionUtils.eagerPartition; import static org.hamcrest.Matchers.containsString; public class CollectionUtilsTests extends OpenSearchTestCase { diff --git a/server/src/test/java/org/opensearch/common/xcontent/BaseXContentTestCase.java b/server/src/test/java/org/opensearch/common/xcontent/BaseXContentTestCase.java index 00ab96dab6a5c..eba41620aef84 100644 --- a/server/src/test/java/org/opensearch/common/xcontent/BaseXContentTestCase.java +++ b/server/src/test/java/org/opensearch/common/xcontent/BaseXContentTestCase.java @@ -47,7 +47,7 @@ import org.opensearch.common.io.PathUtils; import org.opensearch.core.common.text.Text; import org.opensearch.common.unit.DistanceUnit; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.core.xcontent.DeprecationHandler; import org.opensearch.core.xcontent.MediaTypeRegistry; import org.opensearch.core.xcontent.NamedObjectNotFoundException; diff --git a/server/src/test/java/org/opensearch/index/query/MoreLikeThisQueryBuilderTests.java b/server/src/test/java/org/opensearch/index/query/MoreLikeThisQueryBuilderTests.java index 2d950f0994976..49242d34fafb4 100644 --- a/server/src/test/java/org/opensearch/index/query/MoreLikeThisQueryBuilderTests.java +++ b/server/src/test/java/org/opensearch/index/query/MoreLikeThisQueryBuilderTests.java @@ -50,7 +50,7 @@ import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.lucene.search.MoreLikeThisQuery; import org.opensearch.common.settings.Settings; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.common.xcontent.XContentFactory; diff --git a/test/framework/src/main/java/org/opensearch/cluster/OpenSearchAllocationWithConstraintsTestCase.java b/test/framework/src/main/java/org/opensearch/cluster/OpenSearchAllocationWithConstraintsTestCase.java index 55f9f6947b999..7f1d1d3381751 100644 --- a/test/framework/src/main/java/org/opensearch/cluster/OpenSearchAllocationWithConstraintsTestCase.java +++ b/test/framework/src/main/java/org/opensearch/cluster/OpenSearchAllocationWithConstraintsTestCase.java @@ -14,7 +14,7 @@ import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.UnassignedInfo; import org.opensearch.common.settings.Settings; -import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.util.CollectionUtils; import org.junit.Before; import java.util.Arrays; diff --git a/test/framework/src/main/java/org/opensearch/cluster/coordination/LinearizabilityChecker.java b/test/framework/src/main/java/org/opensearch/cluster/coordination/LinearizabilityChecker.java index 3d3cff6f3be9b..60aacb83e0dc1 100644 --- a/test/framework/src/main/java/org/opensearch/cluster/coordination/LinearizabilityChecker.java +++ b/test/framework/src/main/java/org/opensearch/cluster/coordination/LinearizabilityChecker.java @@ -34,8 +34,8 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.lucene.util.FixedBitSet; -import org.opensearch.common.Strings; import org.opensearch.common.collect.Tuple; +import org.opensearch.core.common.Strings; import java.util.ArrayList; import java.util.Collection; diff --git a/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java b/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java index 422f6d8dfbe7d..2fe3b9ff38ff3 100644 --- a/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java +++ b/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java @@ -202,7 +202,7 @@ import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_SHARDS; import static org.opensearch.common.unit.TimeValue.timeValueMillis; -import static org.opensearch.common.util.CollectionUtils.eagerPartition; +import static org.opensearch.core.common.util.CollectionUtils.eagerPartition; import static org.opensearch.discovery.DiscoveryModule.DISCOVERY_SEED_PROVIDERS_SETTING; import static org.opensearch.discovery.SettingsBasedSeedHostsProvider.DISCOVERY_SEED_HOSTS_SETTING; import static org.opensearch.index.IndexSettings.INDEX_SOFT_DELETES_RETENTION_LEASE_PERIOD_SETTING; diff --git a/test/framework/src/main/java/org/opensearch/test/OpenSearchTestCase.java b/test/framework/src/main/java/org/opensearch/test/OpenSearchTestCase.java index 7b787e12be64a..f0453ab11f2ac 100644 --- a/test/framework/src/main/java/org/opensearch/test/OpenSearchTestCase.java +++ b/test/framework/src/main/java/org/opensearch/test/OpenSearchTestCase.java @@ -174,7 +174,7 @@ import java.util.stream.Stream; import static java.util.Collections.emptyMap; -import static org.opensearch.common.util.CollectionUtils.arrayAsArrayList; +import static org.opensearch.core.common.util.CollectionUtils.arrayAsArrayList; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasItem; From 1d28fac490406217e86fee5ffd50fb8fa45934fa Mon Sep 17 00:00:00 2001 From: Nick Knize Date: Fri, 4 Aug 2023 19:04:03 -0500 Subject: [PATCH 08/19] [Refactor] Strings methods other than MediaType (#9126) This refactors all of the remaining utility methods (except those that convert MediaType and XContentBuilder to String) from the :server to :libs:opensearch-core library. This commit is to keep the Strings refactor surface area lean in preparation for cloud native and serverless refactoring. Signed-off-by: Nicholas Walter Knize --- .../org/opensearch/core/common/Strings.java | 359 +++++++++++------- .../create/CreateSnapshotRequest.java | 2 +- .../metadata/MetadataCreateIndexService.java | 7 +- .../MetadataIndexTemplateService.java | 7 +- .../java/org/opensearch/common/Strings.java | 88 ----- .../common/settings/PropertyPlaceholder.java | 2 +- .../BlobStoreIndexShardSnapshot.java | 2 +- .../repositories/RepositoriesService.java | 4 +- .../blobstore/BlobStoreRepository.java | 2 +- .../snapshots/SnapshotsService.java | 8 +- .../MetadataCreateIndexServiceTests.java | 6 +- .../RepositoriesServiceTests.java | 2 +- 12 files changed, 244 insertions(+), 245 deletions(-) diff --git a/libs/core/src/main/java/org/opensearch/core/common/Strings.java b/libs/core/src/main/java/org/opensearch/core/common/Strings.java index 2e610f6047fd4..f0ca12a307313 100644 --- a/libs/core/src/main/java/org/opensearch/core/common/Strings.java +++ b/libs/core/src/main/java/org/opensearch/core/common/Strings.java @@ -8,7 +8,9 @@ package org.opensearch.core.common; +import org.apache.lucene.util.BytesRefBuilder; import org.opensearch.common.Nullable; +import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.common.util.CollectionUtils; import java.io.BufferedReader; @@ -25,6 +27,9 @@ import java.util.TreeSet; import java.util.function.Supplier; +import static java.util.Collections.unmodifiableSet; +import static org.opensearch.common.util.set.Sets.newHashSet; + /** * String utility class. * @@ -35,103 +40,29 @@ public class Strings { public static final String UNKNOWN_UUID_VALUE = "_na_"; public static final String[] EMPTY_ARRAY = new String[0]; + public static final Set INVALID_FILENAME_CHARS = unmodifiableSet( + newHashSet('\\', '/', '*', '?', '"', '<', '>', '|', ' ', ',') + ); - /** - * Convenience method to return a Collection as a delimited (e.g. CSV) - * String. E.g. useful for toString() implementations. - * - * @param coll the Collection to display - * @param delim the delimiter to use (probably a ",") - * @param prefix the String to start each element with - * @param suffix the String to end each element with - * @return the delimited String - */ - public static String collectionToDelimitedString(Iterable coll, String delim, String prefix, String suffix) { - StringBuilder sb = new StringBuilder(); - collectionToDelimitedString(coll, delim, prefix, suffix, sb); - return sb.toString(); - } - - public static void collectionToDelimitedString(Iterable coll, String delim, String prefix, String suffix, StringBuilder sb) { - Iterator it = coll.iterator(); - while (it.hasNext()) { - sb.append(prefix).append(it.next()).append(suffix); - if (it.hasNext()) { - sb.append(delim); - } - } - } + // no instance: + private Strings() {} - /** - * Convenience method to return a Collection as a delimited (e.g. CSV) - * String. E.g. useful for toString() implementations. - * - * @param coll the Collection to display - * @param delim the delimiter to use (probably a ",") - * @return the delimited String - */ - public static String collectionToDelimitedString(Iterable coll, String delim) { - return collectionToDelimitedString(coll, delim, "", ""); - } + // --------------------------------------------------------------------- + // General convenience methods for working with Strings + // --------------------------------------------------------------------- - /** - * Convenience method to return a Collection as a CSV String. - * E.g. useful for toString() implementations. - * - * @param coll the Collection to display - * @return the delimited String - */ - public static String collectionToCommaDelimitedString(Iterable coll) { - return collectionToDelimitedString(coll, ","); - } - - /** - * Convenience method to return a String array as a delimited (e.g. CSV) - * String. E.g. useful for toString() implementations. - * - * @param arr the array to display - * @param delim the delimiter to use (probably a ",") - * @return the delimited String - */ - public static String arrayToDelimitedString(Object[] arr, String delim) { - StringBuilder sb = new StringBuilder(); - arrayToDelimitedString(arr, delim, sb); - return sb.toString(); - } - - public static void arrayToDelimitedString(Object[] arr, String delim, StringBuilder sb) { - if (isEmpty(arr)) { - return; - } - for (int i = 0; i < arr.length; i++) { - if (i > 0) { - sb.append(delim); + public static void spaceify(int spaces, String from, StringBuilder to) throws Exception { + try (BufferedReader reader = new BufferedReader(new StringReader(from))) { + String line; + while ((line = reader.readLine()) != null) { + for (int i = 0; i < spaces; i++) { + to.append(' '); + } + to.append(line).append('\n'); } - sb.append(arr[i]); } } - /** - * Convenience method to return a String array as a CSV String. - * E.g. useful for toString() implementations. - * - * @param arr the array to display - * @return the delimited String - */ - public static String arrayToCommaDelimitedString(Object[] arr) { - return arrayToDelimitedString(arr, ","); - } - - /** - * Determine whether the given array is empty: - * i.e. null or of zero length. - * - * @param array the array to check - */ - private static boolean isEmpty(Object[] array) { - return (array == null || array.length == 0); - } - /** * Check that the given CharSequence is neither null nor of length 0. * Note: Will return true for a CharSequence that purely consists of whitespace. @@ -150,6 +81,18 @@ public static boolean hasLength(CharSequence str) { return (str != null && str.length() > 0); } + /** + * Check that the given BytesReference is neither null nor of length 0 + * Note: Will return true for a BytesReference that purely consists of whitespace. + * + * @param bytesReference the BytesReference to check (may be null) + * @return true if the BytesReference is not null and has length + * @see Strings#hasLength(CharSequence) + */ + public static boolean hasLength(final BytesReference bytesReference) { + return (bytesReference != null && bytesReference.length() > 0); + } + /** * Check that the given String is neither null nor of length 0. * Note: Will return true for a String that purely consists of whitespace. @@ -158,7 +101,7 @@ public static boolean hasLength(CharSequence str) { * @return true if the String is not null and has length * @see Strings#hasLength(CharSequence) */ - public static boolean hasLength(String str) { + public static boolean hasLength(final String str) { return hasLength((CharSequence) str); } @@ -175,7 +118,7 @@ public static boolean hasLength(String str) { * @param str the CharSequence to check (may be null) * @return true if the CharSequence is either null or has a zero length */ - public static boolean isEmpty(CharSequence str) { + public static boolean isEmpty(final CharSequence str) { return hasLength(str) == false; } @@ -223,6 +166,42 @@ public static boolean hasText(String str) { return hasText((CharSequence) str); } + /** + * Trim all occurrences of the supplied leading character from the given String. + * + * @param str the String to check + * @param leadingCharacter the leading character to be trimmed + * @return the trimmed String + */ + public static String trimLeadingCharacter(String str, char leadingCharacter) { + if (hasLength(str) == false) { + return str; + } + StringBuilder sb = new StringBuilder(str); + while (sb.length() > 0 && sb.charAt(0) == leadingCharacter) { + sb.deleteCharAt(0); + } + return sb.toString(); + } + + /** + * Test whether the given string matches the given substring + * at the given index. + * + * @param str the original string (or StringBuilder) + * @param index the index in the original string to start matching against + * @param substring the substring to match at the given index + */ + public static boolean substringMatch(CharSequence str, int index, CharSequence substring) { + for (int j = 0; j < substring.length(); j++) { + int i = index + j; + if (i >= str.length() || str.charAt(i) != substring.charAt(j)) { + return false; + } + } + return true; + } + /** * Replace all occurrences of a substring within a string with * another string. @@ -252,24 +231,6 @@ public static String replace(String inString, String oldPattern, String newPatte return sb.toString(); } - /** - * Trim all occurrences of the supplied leading character from the given String. - * - * @param str the String to check - * @param leadingCharacter the leading character to be trimmed - * @return the trimmed String - */ - public static String trimLeadingCharacter(String str, char leadingCharacter) { - if (hasLength(str) == false) { - return str; - } - StringBuilder sb = new StringBuilder(str); - while (sb.length() > 0 && sb.charAt(0) == leadingCharacter) { - sb.deleteCharAt(0); - } - return sb.toString(); - } - /** * Delete all occurrences of the given substring. * @@ -303,16 +264,65 @@ public static String deleteAny(String inString, String charsToDelete) { return sb.toString(); } - public static void spaceify(int spaces, String from, StringBuilder to) throws Exception { - try (BufferedReader reader = new BufferedReader(new StringReader(from))) { - String line; - while ((line = reader.readLine()) != null) { - for (int i = 0; i < spaces; i++) { - to.append(' '); - } - to.append(line).append('\n'); + // --------------------------------------------------------------------- + // Convenience methods for working with formatted Strings + // --------------------------------------------------------------------- + + /** + * Quote the given String with single quotes. + * + * @param str the input String (e.g. "myString") + * @return the quoted String (e.g. "'myString'"), + * or null if the input was null + */ + public static String quote(String str) { + return (str != null ? "'" + str + "'" : null); + } + + /** + * Capitalize a String, changing the first letter to + * upper case as per {@link Character#toUpperCase(char)}. + * No other letters are changed. + * + * @param str the String to capitalize, may be null + * @return the capitalized String, null if null + */ + public static String capitalize(String str) { + return changeFirstCharacterCase(str, true); + } + + private static String changeFirstCharacterCase(String str, boolean capitalize) { + if (str == null || str.length() == 0) { + return str; + } + StringBuilder sb = new StringBuilder(str.length()); + if (capitalize) { + sb.append(Character.toUpperCase(str.charAt(0))); + } else { + sb.append(Character.toLowerCase(str.charAt(0))); + } + sb.append(str.substring(1)); + return sb.toString(); + } + + public static boolean validFileName(String fileName) { + for (int i = 0; i < fileName.length(); i++) { + char c = fileName.charAt(i); + if (INVALID_FILENAME_CHARS.contains(c)) { + return false; } } + return true; + } + + public static boolean validFileNameExcludingAstrix(String fileName) { + for (int i = 0; i < fileName.length(); i++) { + char c = fileName.charAt(i); + if (c != '*' && INVALID_FILENAME_CHARS.contains(c)) { + return false; + } + } + return true; } /** @@ -511,31 +521,91 @@ public static Set commaDelimitedListToSet(String str) { } /** - * Capitalize a String, changing the first letter to - * upper case as per {@link Character#toUpperCase(char)}. - * No other letters are changed. + * Convenience method to return a Collection as a delimited (e.g. CSV) + * String. E.g. useful for toString() implementations. * - * @param str the String to capitalize, may be null - * @return the capitalized String, null if null + * @param coll the Collection to display + * @param delim the delimiter to use (probably a ",") + * @param prefix the String to start each element with + * @param suffix the String to end each element with + * @return the delimited String */ - public static String capitalize(String str) { - return changeFirstCharacterCase(str, true); + public static String collectionToDelimitedString(Iterable coll, String delim, String prefix, String suffix) { + StringBuilder sb = new StringBuilder(); + collectionToDelimitedString(coll, delim, prefix, suffix, sb); + return sb.toString(); } - private static String changeFirstCharacterCase(String str, boolean capitalize) { - if (str == null || str.length() == 0) { - return str; - } - StringBuilder sb = new StringBuilder(str.length()); - if (capitalize) { - sb.append(Character.toUpperCase(str.charAt(0))); - } else { - sb.append(Character.toLowerCase(str.charAt(0))); + public static void collectionToDelimitedString(Iterable coll, String delim, String prefix, String suffix, StringBuilder sb) { + Iterator it = coll.iterator(); + while (it.hasNext()) { + sb.append(prefix).append(it.next()).append(suffix); + if (it.hasNext()) { + sb.append(delim); + } } - sb.append(str.substring(1)); + } + + /** + * Convenience method to return a Collection as a delimited (e.g. CSV) + * String. E.g. useful for toString() implementations. + * + * @param coll the Collection to display + * @param delim the delimiter to use (probably a ",") + * @return the delimited String + */ + public static String collectionToDelimitedString(Iterable coll, String delim) { + return collectionToDelimitedString(coll, delim, "", ""); + } + + /** + * Convenience method to return a Collection as a CSV String. + * E.g. useful for toString() implementations. + * + * @param coll the Collection to display + * @return the delimited String + */ + public static String collectionToCommaDelimitedString(Iterable coll) { + return collectionToDelimitedString(coll, ","); + } + + /** + * Convenience method to return a String array as a delimited (e.g. CSV) + * String. E.g. useful for toString() implementations. + * + * @param arr the array to display + * @param delim the delimiter to use (probably a ",") + * @return the delimited String + */ + public static String arrayToDelimitedString(Object[] arr, String delim) { + StringBuilder sb = new StringBuilder(); + arrayToDelimitedString(arr, delim, sb); return sb.toString(); } + public static void arrayToDelimitedString(Object[] arr, String delim, StringBuilder sb) { + if (isEmpty(arr)) { + return; + } + for (int i = 0; i < arr.length; i++) { + if (i > 0) { + sb.append(delim); + } + sb.append(arr[i]); + } + } + + /** + * Convenience method to return a String array as a CSV String. + * E.g. useful for toString() implementations. + * + * @param arr the array to display + * @return the delimited String + */ + public static String arrayToCommaDelimitedString(Object[] arr) { + return arrayToDelimitedString(arr, ","); + } + /** * Format the double value with a single decimal points, trimming trailing '.0'. */ @@ -559,6 +629,25 @@ public static String format1Decimals(double value, String suffix) { } } + /** + * Determine whether the given array is empty: + * i.e. null or of zero length. + * + * @param array the array to check + */ + private static boolean isEmpty(final Object[] array) { + return (array == null || array.length == 0); + } + + public static byte[] toUTF8Bytes(CharSequence charSequence) { + return toUTF8Bytes(charSequence, new BytesRefBuilder()); + } + + public static byte[] toUTF8Bytes(CharSequence charSequence, BytesRefBuilder spare) { + spare.copyChars(charSequence); + return Arrays.copyOf(spare.bytes(), spare.length()); + } + /** * Return substring(beginIndex, endIndex) that is impervious to string length. */ diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/create/CreateSnapshotRequest.java b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/create/CreateSnapshotRequest.java index 3bb552b80084d..ce41066868202 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/create/CreateSnapshotRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/create/CreateSnapshotRequest.java @@ -74,7 +74,7 @@ *

  • must not contain hash sign ('#')
  • *
  • must not start with underscore ('_')
  • *
  • must be lowercase
  • - *
  • must not contain invalid file name characters {@link org.opensearch.common.Strings#INVALID_FILENAME_CHARS}
  • + *
  • must not contain invalid file name characters {@link org.opensearch.core.common.Strings#INVALID_FILENAME_CHARS}
  • * * * @opensearch.internal diff --git a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java index db9964b1a2ff8..fd38763023532 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java @@ -257,11 +257,8 @@ public boolean validateDotIndex(String index, @Nullable Boolean isHidden) { * Validate the name for an index or alias against some static rules. */ public static void validateIndexOrAliasName(String index, BiFunction exceptionCtor) { - if (org.opensearch.common.Strings.validFileName(index) == false) { - throw exceptionCtor.apply( - index, - "must not contain the following characters " + org.opensearch.common.Strings.INVALID_FILENAME_CHARS - ); + if (Strings.validFileName(index) == false) { + throw exceptionCtor.apply(index, "must not contain the following characters " + Strings.INVALID_FILENAME_CHARS); } if (index.isEmpty()) { throw exceptionCtor.apply(index, "must not be empty"); diff --git a/server/src/main/java/org/opensearch/cluster/metadata/MetadataIndexTemplateService.java b/server/src/main/java/org/opensearch/cluster/metadata/MetadataIndexTemplateService.java index 4f12313f951bc..2c4e6bdec95b0 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataIndexTemplateService.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataIndexTemplateService.java @@ -1502,12 +1502,9 @@ private void validate(String name, @Nullable Settings settings, List ind if (indexPattern.startsWith("_")) { validationErrors.add("index_pattern [" + indexPattern + "] must not start with '_'"); } - if (org.opensearch.common.Strings.validFileNameExcludingAstrix(indexPattern) == false) { + if (Strings.validFileNameExcludingAstrix(indexPattern) == false) { validationErrors.add( - "index_pattern [" - + indexPattern - + "] must not contain the following characters " - + org.opensearch.common.Strings.INVALID_FILENAME_CHARS + "index_pattern [" + indexPattern + "] must not contain the following characters " + Strings.INVALID_FILENAME_CHARS ); } } diff --git a/server/src/main/java/org/opensearch/common/Strings.java b/server/src/main/java/org/opensearch/common/Strings.java index e3404d6ffb297..cd56fff7b2c62 100644 --- a/server/src/main/java/org/opensearch/common/Strings.java +++ b/server/src/main/java/org/opensearch/common/Strings.java @@ -32,7 +32,6 @@ package org.opensearch.common; -import org.apache.lucene.util.BytesRefBuilder; import org.opensearch.ExceptionsHelper; import org.opensearch.OpenSearchException; import org.opensearch.core.common.bytes.BytesReference; @@ -41,11 +40,6 @@ import org.opensearch.core.xcontent.XContentBuilder; import java.io.IOException; -import java.util.Arrays; -import java.util.Set; - -import static java.util.Collections.unmodifiableSet; -import static org.opensearch.common.util.set.Sets.newHashSet; /** * String utility class. @@ -56,90 +50,8 @@ public class Strings { public static final String[] EMPTY_ARRAY = org.opensearch.core.common.Strings.EMPTY_ARRAY; - // --------------------------------------------------------------------- - // General convenience methods for working with Strings - // --------------------------------------------------------------------- - - /** - * Check that the given BytesReference is neither null nor of length 0 - * Note: Will return true for a BytesReference that purely consists of whitespace. - * - * @param bytesReference the BytesReference to check (may be null) - * @return true if the BytesReference is not null and has length - * @see org.opensearch.core.common.Strings#hasLength(CharSequence) - */ - public static boolean hasLength(BytesReference bytesReference) { - return (bytesReference != null && bytesReference.length() > 0); - } - - /** - * Test whether the given string matches the given substring - * at the given index. - * - * @param str the original string (or StringBuilder) - * @param index the index in the original string to start matching against - * @param substring the substring to match at the given index - */ - public static boolean substringMatch(CharSequence str, int index, CharSequence substring) { - for (int j = 0; j < substring.length(); j++) { - int i = index + j; - if (i >= str.length() || str.charAt(i) != substring.charAt(j)) { - return false; - } - } - return true; - } - - // --------------------------------------------------------------------- - // Convenience methods for working with formatted Strings - // --------------------------------------------------------------------- - - /** - * Quote the given String with single quotes. - * - * @param str the input String (e.g. "myString") - * @return the quoted String (e.g. "'myString'"), - * or null if the input was null - */ - public static String quote(String str) { - return (str != null ? "'" + str + "'" : null); - } - - public static final Set INVALID_FILENAME_CHARS = unmodifiableSet( - newHashSet('\\', '/', '*', '?', '"', '<', '>', '|', ' ', ',') - ); - - public static boolean validFileName(String fileName) { - for (int i = 0; i < fileName.length(); i++) { - char c = fileName.charAt(i); - if (INVALID_FILENAME_CHARS.contains(c)) { - return false; - } - } - return true; - } - - public static boolean validFileNameExcludingAstrix(String fileName) { - for (int i = 0; i < fileName.length(); i++) { - char c = fileName.charAt(i); - if (c != '*' && INVALID_FILENAME_CHARS.contains(c)) { - return false; - } - } - return true; - } - private Strings() {} - public static byte[] toUTF8Bytes(CharSequence charSequence) { - return toUTF8Bytes(charSequence, new BytesRefBuilder()); - } - - public static byte[] toUTF8Bytes(CharSequence charSequence, BytesRefBuilder spare) { - spare.copyChars(charSequence); - return Arrays.copyOf(spare.bytes(), spare.length()); - } - /** * Return a {@link String} that is the json representation of the provided {@link ToXContent}. * Wraps the output into an anonymous object if needed. The content is not pretty-printed diff --git a/server/src/main/java/org/opensearch/common/settings/PropertyPlaceholder.java b/server/src/main/java/org/opensearch/common/settings/PropertyPlaceholder.java index 76b5642949e03..655ffc461cd91 100644 --- a/server/src/main/java/org/opensearch/common/settings/PropertyPlaceholder.java +++ b/server/src/main/java/org/opensearch/common/settings/PropertyPlaceholder.java @@ -32,7 +32,7 @@ package org.opensearch.common.settings; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import java.util.HashSet; import java.util.Objects; diff --git a/server/src/main/java/org/opensearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshot.java b/server/src/main/java/org/opensearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshot.java index 0b27afa5e9517..6a48007234260 100644 --- a/server/src/main/java/org/opensearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshot.java +++ b/server/src/main/java/org/opensearch/index/snapshots/blobstore/BlobStoreIndexShardSnapshot.java @@ -36,8 +36,8 @@ import org.apache.lucene.util.Version; import org.opensearch.OpenSearchParseException; import org.opensearch.core.ParseField; -import org.opensearch.common.Strings; import org.opensearch.common.lucene.Lucene; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.unit.ByteSizeValue; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.ToXContentFragment; diff --git a/server/src/main/java/org/opensearch/repositories/RepositoriesService.java b/server/src/main/java/org/opensearch/repositories/RepositoriesService.java index e66f8ddee5678..b113a8e32c40e 100644 --- a/server/src/main/java/org/opensearch/repositories/RepositoriesService.java +++ b/server/src/main/java/org/opensearch/repositories/RepositoriesService.java @@ -57,7 +57,6 @@ import org.opensearch.cluster.service.ClusterManagerTaskKeys; import org.opensearch.cluster.service.ClusterManagerTaskThrottler; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.Strings; import org.opensearch.common.lifecycle.AbstractLifecycleComponent; import org.opensearch.common.regex.Regex; import org.opensearch.common.settings.Setting; @@ -66,6 +65,7 @@ import org.opensearch.common.util.FeatureFlags; import org.opensearch.common.util.concurrent.ConcurrentCollections; import org.opensearch.common.util.io.IOUtils; +import org.opensearch.core.common.Strings; import org.opensearch.repositories.blobstore.MeteredBlobStoreRepository; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportService; @@ -599,7 +599,7 @@ private Repository createRepository(RepositoryMetadata repositoryMetadata, Map new ParameterizedMessage( "deleting snapshots [{}] from repository [{}]", - org.opensearch.core.common.Strings.arrayToCommaDelimitedString(snapshotNames), + Strings.arrayToCommaDelimitedString(snapshotNames), repoName ) ); diff --git a/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java b/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java index 63c3511a97d2b..b8a31f4d7dda9 100644 --- a/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java +++ b/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java @@ -671,7 +671,11 @@ public void testValidateIndexName() throws Exception { false, new AwarenessReplicaBalance(Settings.EMPTY, clusterService.getClusterSettings()) ); - validateIndexName(checkerService, "index?name", "must not contain the following characters " + Strings.INVALID_FILENAME_CHARS); + validateIndexName( + checkerService, + "index?name", + "must not contain the following characters " + org.opensearch.core.common.Strings.INVALID_FILENAME_CHARS + ); validateIndexName(checkerService, "index#name", "must not contain '#'"); diff --git a/server/src/test/java/org/opensearch/repositories/RepositoriesServiceTests.java b/server/src/test/java/org/opensearch/repositories/RepositoriesServiceTests.java index 085a64b439bbe..076f11b679d16 100644 --- a/server/src/test/java/org/opensearch/repositories/RepositoriesServiceTests.java +++ b/server/src/test/java/org/opensearch/repositories/RepositoriesServiceTests.java @@ -47,13 +47,13 @@ import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.service.ClusterApplierService; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.Strings; import org.opensearch.common.UUIDs; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.blobstore.BlobStore; import org.opensearch.common.lifecycle.Lifecycle; import org.opensearch.common.lifecycle.LifecycleListener; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.index.mapper.MapperService; import org.opensearch.core.index.shard.ShardId; From c6e4bcd097969f00956c0f4c152540cb610e4f93 Mon Sep 17 00:00:00 2001 From: Poojita Raj Date: Fri, 4 Aug 2023 19:33:18 -0700 Subject: [PATCH 09/19] [Segment Replication] Prioritize replica shard movement during shard relocation (#8875) * add shard movement strategy setting Signed-off-by: Poojita Raj * add tests Signed-off-by: Poojita Raj * add changelog Signed-off-by: Poojita Raj * Add NodeVersionAllocationDecider check Signed-off-by: Poojita Raj * refactoring Signed-off-by: Poojita Raj * add annotation + refactor Signed-off-by: Poojita Raj --------- Signed-off-by: Poojita Raj --- CHANGELOG.md | 1 + .../org/opensearch/cluster/ClusterModule.java | 2 +- .../cluster/routing/RoutingNodes.java | 167 +++++++++++------- .../routing/ShardMovementStrategy.java | 57 ++++++ .../allocator/BalancedShardsAllocator.java | 27 ++- .../allocator/LocalShardsBalancer.java | 27 ++- .../decider/NodeVersionAllocationDecider.java | 32 ++++ .../common/settings/ClusterSettings.java | 1 + .../cluster/routing/RoutingNodesTests.java | 52 +++++- ...s.java => ShardMovementStrategyTests.java} | 65 +++++-- .../NodeVersionAllocationDeciderTests.java | 151 +++++++++++++++- 11 files changed, 491 insertions(+), 91 deletions(-) create mode 100644 server/src/main/java/org/opensearch/cluster/routing/ShardMovementStrategy.java rename server/src/test/java/org/opensearch/cluster/routing/{MovePrimaryFirstTests.java => ShardMovementStrategyTests.java} (60%) diff --git a/CHANGELOG.md b/CHANGELOG.md index 3b81cd3a60deb..0bb21fb425ccf 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -82,6 +82,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - [distribution/archives] [Linux] [x64] Provide the variant of the distributions bundled with JRE ([#8195]()https://github.com/opensearch-project/OpenSearch/pull/8195) - Add configuration for file cache size to max remote data ratio to prevent oversubscription of file cache ([#8606](https://github.com/opensearch-project/OpenSearch/pull/8606)) - Disallow compression level to be set for default and best_compression index codecs ([#8737]()https://github.com/opensearch-project/OpenSearch/pull/8737) +- Prioritize replica shard movement during shard relocation ([#8875](https://github.com/opensearch-project/OpenSearch/pull/8875)) ### Dependencies - Bump `org.apache.logging.log4j:log4j-core` from 2.17.1 to 2.20.0 ([#8307](https://github.com/opensearch-project/OpenSearch/pull/8307)) diff --git a/server/src/main/java/org/opensearch/cluster/ClusterModule.java b/server/src/main/java/org/opensearch/cluster/ClusterModule.java index b80fd1d746831..e797a08aba3cd 100644 --- a/server/src/main/java/org/opensearch/cluster/ClusterModule.java +++ b/server/src/main/java/org/opensearch/cluster/ClusterModule.java @@ -359,7 +359,7 @@ public static Collection createAllocationDeciders( addAllocationDecider(deciders, new ConcurrentRebalanceAllocationDecider(settings, clusterSettings)); addAllocationDecider(deciders, new ConcurrentRecoveriesAllocationDecider(settings, clusterSettings)); addAllocationDecider(deciders, new EnableAllocationDecider(settings, clusterSettings)); - addAllocationDecider(deciders, new NodeVersionAllocationDecider()); + addAllocationDecider(deciders, new NodeVersionAllocationDecider(settings)); addAllocationDecider(deciders, new SnapshotInProgressAllocationDecider()); addAllocationDecider(deciders, new RestoreInProgressAllocationDecider()); addAllocationDecider(deciders, new FilterAllocationDecider(settings, clusterSettings)); diff --git a/server/src/main/java/org/opensearch/cluster/routing/RoutingNodes.java b/server/src/main/java/org/opensearch/cluster/routing/RoutingNodes.java index 5402218664f6f..c5c480dc25d6e 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/RoutingNodes.java +++ b/server/src/main/java/org/opensearch/cluster/routing/RoutingNodes.java @@ -1310,100 +1310,131 @@ private void ensureMutable() { } /** - * Creates an iterator over shards interleaving between nodes: The iterator returns the first shard from - * the first node, then the first shard of the second node, etc. until one shard from each node has been returned. - * The iterator then resumes on the first node by returning the second shard and continues until all shards from - * all the nodes have been returned. - * @param movePrimaryFirst if true, all primary shards are iterated over before iterating replica for any node - * @return iterator of shard routings + * Returns iterator of shard routings used by {@link #nodeInterleavedShardIterator(ShardMovementStrategy)} + * @param primaryFirst true when ShardMovementStrategy = ShardMovementStrategy.PRIMARY_FIRST, false when it is ShardMovementStrategy.REPLICA_FIRST */ - public Iterator nodeInterleavedShardIterator(boolean movePrimaryFirst) { + private Iterator buildIteratorForMovementStrategy(boolean primaryFirst) { final Queue> queue = new ArrayDeque<>(); for (Map.Entry entry : nodesToShards.entrySet()) { queue.add(entry.getValue().copyShards().iterator()); } - if (movePrimaryFirst) { - return new Iterator() { - private Queue replicaShards = new ArrayDeque<>(); - private Queue> replicaIterators = new ArrayDeque<>(); - - public boolean hasNext() { - while (!queue.isEmpty()) { - if (queue.peek().hasNext()) { - return true; - } - queue.poll(); - } - if (!replicaShards.isEmpty()) { + return new Iterator() { + private Queue shardRoutings = new ArrayDeque<>(); + private Queue> shardIterators = new ArrayDeque<>(); + + public boolean hasNext() { + while (queue.isEmpty() == false) { + if (queue.peek().hasNext()) { return true; } - while (!replicaIterators.isEmpty()) { - if (replicaIterators.peek().hasNext()) { - return true; - } - replicaIterators.poll(); + queue.poll(); + } + if (!shardRoutings.isEmpty()) { + return true; + } + while (!shardIterators.isEmpty()) { + if (shardIterators.peek().hasNext()) { + return true; } - return false; + shardIterators.poll(); } + return false; + } - public ShardRouting next() { - if (hasNext() == false) { - throw new NoSuchElementException(); - } - while (!queue.isEmpty()) { - Iterator iter = queue.poll(); + public ShardRouting next() { + if (hasNext() == false) { + throw new NoSuchElementException(); + } + while (!queue.isEmpty()) { + Iterator iter = queue.poll(); + if (primaryFirst) { if (iter.hasNext()) { ShardRouting result = iter.next(); if (result.primary()) { queue.offer(iter); return result; } - replicaShards.offer(result); - replicaIterators.offer(iter); + shardRoutings.offer(result); + shardIterators.offer(iter); + } + } else { + while (iter.hasNext()) { + ShardRouting result = iter.next(); + if (result.primary() == false) { + queue.offer(iter); + return result; + } + shardRoutings.offer(result); + shardIterators.offer(iter); } } - if (!replicaShards.isEmpty()) { - return replicaShards.poll(); - } - Iterator replicaIterator = replicaIterators.poll(); - ShardRouting replicaShard = replicaIterator.next(); - replicaIterators.offer(replicaIterator); - - assert !replicaShard.primary(); - return replicaShard; } - - public void remove() { - throw new UnsupportedOperationException(); + if (!shardRoutings.isEmpty()) { + return shardRoutings.poll(); } - }; + Iterator replicaIterator = shardIterators.poll(); + ShardRouting replicaShard = replicaIterator.next(); + shardIterators.offer(replicaIterator); + + assert replicaShard.primary() != primaryFirst; + return replicaShard; + } + + public void remove() { + throw new UnsupportedOperationException(); + } + + }; + } + + /** + * Creates an iterator over shards interleaving between nodes: The iterator returns the first shard from + * the first node, then the first shard of the second node, etc. until one shard from each node has been returned. + * The iterator then resumes on the first node by returning the second shard and continues until all shards from + * all the nodes have been returned. + * @param shardMovementStrategy if ShardMovementStrategy.PRIMARY_FIRST, all primary shards are iterated over before iterating replica for any node + * if ShardMovementStrategy.REPLICA_FIRST, all replica shards are iterated over before iterating primary for any node + * if ShardMovementStrategy.NO_PREFERENCE, order of replica and primary shards doesn't matter in iteration + * @return iterator of shard routings + */ + public Iterator nodeInterleavedShardIterator(ShardMovementStrategy shardMovementStrategy) { + final Queue> queue = new ArrayDeque<>(); + for (Map.Entry entry : nodesToShards.entrySet()) { + queue.add(entry.getValue().copyShards().iterator()); + } + if (shardMovementStrategy == ShardMovementStrategy.PRIMARY_FIRST) { + return buildIteratorForMovementStrategy(true); } else { - return new Iterator() { - @Override - public boolean hasNext() { - while (!queue.isEmpty()) { - if (queue.peek().hasNext()) { - return true; + if (shardMovementStrategy == ShardMovementStrategy.REPLICA_FIRST) { + return buildIteratorForMovementStrategy(false); + } else { + return new Iterator() { + @Override + public boolean hasNext() { + while (!queue.isEmpty()) { + if (queue.peek().hasNext()) { + return true; + } + queue.poll(); } - queue.poll(); + return false; } - return false; - } - @Override - public ShardRouting next() { - if (hasNext() == false) { - throw new NoSuchElementException(); + @Override + public ShardRouting next() { + if (hasNext() == false) { + throw new NoSuchElementException(); + } + Iterator iter = queue.poll(); + queue.offer(iter); + return iter.next(); } - Iterator iter = queue.poll(); - queue.offer(iter); - return iter.next(); - } - public void remove() { - throw new UnsupportedOperationException(); - } - }; + public void remove() { + throw new UnsupportedOperationException(); + } + }; + } } } diff --git a/server/src/main/java/org/opensearch/cluster/routing/ShardMovementStrategy.java b/server/src/main/java/org/opensearch/cluster/routing/ShardMovementStrategy.java new file mode 100644 index 0000000000000..cfdeed5c227b6 --- /dev/null +++ b/server/src/main/java/org/opensearch/cluster/routing/ShardMovementStrategy.java @@ -0,0 +1,57 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.cluster.routing; + +import org.opensearch.cluster.routing.allocation.allocator.BalancedShardsAllocator; + +import java.util.Locale; + +/** + * ShardMovementStrategy defines the order in which shard movement occurs. + * + * ShardMovementStrategy values or rather their string representation to be used with + * {@link BalancedShardsAllocator#SHARD_MOVEMENT_STRATEGY_SETTING} via cluster settings. + * + * @opensearch.internal + */ +public enum ShardMovementStrategy { + /** + * default behavior in which order of shard movement doesn't matter. + */ + NO_PREFERENCE, + + /** + * primary shards are moved first + */ + PRIMARY_FIRST, + + /** + * replica shards are moved first + */ + REPLICA_FIRST; + + public static ShardMovementStrategy parse(String strValue) { + if (strValue == null) { + return null; + } else { + strValue = strValue.toUpperCase(Locale.ROOT); + try { + return ShardMovementStrategy.valueOf(strValue); + } catch (IllegalArgumentException e) { + throw new IllegalArgumentException("Illegal allocation.shard_movement_strategy value [" + strValue + "]"); + } + } + } + + @Override + public String toString() { + return name().toLowerCase(Locale.ROOT); + } + +} diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java index 59d7fab59c266..19e0e318eb805 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java @@ -37,6 +37,7 @@ import org.apache.lucene.util.IntroSorter; import org.opensearch.cluster.routing.RoutingNode; import org.opensearch.cluster.routing.RoutingNodes; +import org.opensearch.cluster.routing.ShardMovementStrategy; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.UnassignedInfo; import org.opensearch.cluster.routing.UnassignedInfo.AllocationStatus; @@ -107,8 +108,22 @@ public class BalancedShardsAllocator implements ShardsAllocator { "cluster.routing.allocation.move.primary_first", false, Property.Dynamic, + Property.NodeScope, + Property.Deprecated + ); + + /** + * Decides order in which to move shards from node when shards can not stay on node anymore. {@link LocalShardsBalancer#moveShards()} + * Encapsulates behavior of above SHARD_MOVE_PRIMARY_FIRST_SETTING. + */ + public static final Setting SHARD_MOVEMENT_STRATEGY_SETTING = new Setting( + "cluster.routing.allocation.shard_movement_strategy", + ShardMovementStrategy.NO_PREFERENCE.toString(), + ShardMovementStrategy::parse, + Property.Dynamic, Property.NodeScope ); + public static final Setting THRESHOLD_SETTING = Setting.floatSetting( "cluster.routing.allocation.balance.threshold", 1.0f, @@ -131,6 +146,7 @@ public class BalancedShardsAllocator implements ShardsAllocator { ); private volatile boolean movePrimaryFirst; + private volatile ShardMovementStrategy shardMovementStrategy; private volatile boolean preferPrimaryShardBalance; private volatile WeightFunction weightFunction; @@ -145,8 +161,10 @@ public BalancedShardsAllocator(Settings settings, ClusterSettings clusterSetting setWeightFunction(INDEX_BALANCE_FACTOR_SETTING.get(settings), SHARD_BALANCE_FACTOR_SETTING.get(settings)); setThreshold(THRESHOLD_SETTING.get(settings)); setPreferPrimaryShardBalance(PREFER_PRIMARY_SHARD_BALANCE.get(settings)); + setShardMovementStrategy(SHARD_MOVEMENT_STRATEGY_SETTING.get(settings)); clusterSettings.addSettingsUpdateConsumer(PREFER_PRIMARY_SHARD_BALANCE, this::setPreferPrimaryShardBalance); clusterSettings.addSettingsUpdateConsumer(SHARD_MOVE_PRIMARY_FIRST_SETTING, this::setMovePrimaryFirst); + clusterSettings.addSettingsUpdateConsumer(SHARD_MOVEMENT_STRATEGY_SETTING, this::setShardMovementStrategy); clusterSettings.addSettingsUpdateConsumer(INDEX_BALANCE_FACTOR_SETTING, SHARD_BALANCE_FACTOR_SETTING, this::setWeightFunction); clusterSettings.addSettingsUpdateConsumer(THRESHOLD_SETTING, this::setThreshold); } @@ -155,6 +173,10 @@ private void setMovePrimaryFirst(boolean movePrimaryFirst) { this.movePrimaryFirst = movePrimaryFirst; } + private void setShardMovementStrategy(ShardMovementStrategy shardMovementStrategy) { + this.shardMovementStrategy = shardMovementStrategy; + } + private void setWeightFunction(float indexBalance, float shardBalanceFactor) { weightFunction = new WeightFunction(indexBalance, shardBalanceFactor); } @@ -184,6 +206,7 @@ public void allocate(RoutingAllocation allocation) { logger, allocation, movePrimaryFirst, + shardMovementStrategy, weightFunction, threshold, preferPrimaryShardBalance @@ -205,6 +228,7 @@ public ShardAllocationDecision decideShardAllocation(final ShardRouting shard, f logger, allocation, movePrimaryFirst, + shardMovementStrategy, weightFunction, threshold, preferPrimaryShardBalance @@ -456,11 +480,12 @@ public Balancer( Logger logger, RoutingAllocation allocation, boolean movePrimaryFirst, + ShardMovementStrategy shardMovementStrategy, BalancedShardsAllocator.WeightFunction weight, float threshold, boolean preferPrimaryBalance ) { - super(logger, allocation, movePrimaryFirst, weight, threshold, preferPrimaryBalance); + super(logger, allocation, movePrimaryFirst, shardMovementStrategy, weight, threshold, preferPrimaryBalance); } } diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/LocalShardsBalancer.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/LocalShardsBalancer.java index 80b4f720bd104..e1e6b696e3ad2 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/LocalShardsBalancer.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/allocator/LocalShardsBalancer.java @@ -16,6 +16,7 @@ import org.opensearch.cluster.routing.RoutingNode; import org.opensearch.cluster.routing.RoutingNodes; import org.opensearch.cluster.routing.RoutingPool; +import org.opensearch.cluster.routing.ShardMovementStrategy; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.ShardRoutingState; import org.opensearch.cluster.routing.UnassignedInfo; @@ -58,6 +59,7 @@ public class LocalShardsBalancer extends ShardsBalancer { private final RoutingAllocation allocation; private final RoutingNodes routingNodes; private final boolean movePrimaryFirst; + private final ShardMovementStrategy shardMovementStrategy; private final boolean preferPrimaryBalance; private final BalancedShardsAllocator.WeightFunction weight; @@ -74,6 +76,7 @@ public LocalShardsBalancer( Logger logger, RoutingAllocation allocation, boolean movePrimaryFirst, + ShardMovementStrategy shardMovementStrategy, BalancedShardsAllocator.WeightFunction weight, float threshold, boolean preferPrimaryBalance @@ -93,6 +96,7 @@ public LocalShardsBalancer( sorter = newNodeSorter(); inEligibleTargetNode = new HashSet<>(); this.preferPrimaryBalance = preferPrimaryBalance; + this.shardMovementStrategy = shardMovementStrategy; } /** @@ -527,6 +531,22 @@ private void checkAndAddInEligibleTargetNode(RoutingNode targetNode) { } } + /** + * Returns the correct Shard movement strategy to use. + * If users are still using deprecated setting "move_primary_first", we want behavior to remain unchanged. + * In the event of changing ShardMovementStrategy setting from default setting NO_PREFERENCE to either PRIMARY_FIRST or REPLICA_FIRST, we want that + * to have priority over values set in move_primary_first setting. + */ + private ShardMovementStrategy getShardMovementStrategy() { + if (shardMovementStrategy != ShardMovementStrategy.NO_PREFERENCE) { + return shardMovementStrategy; + } + if (movePrimaryFirst) { + return ShardMovementStrategy.PRIMARY_FIRST; + } + return ShardMovementStrategy.NO_PREFERENCE; + } + /** * Move started shards that can not be allocated to a node anymore * @@ -549,7 +569,8 @@ void moveShards() { checkAndAddInEligibleTargetNode(currentNode.getRoutingNode()); } boolean primariesThrottled = false; - for (Iterator it = allocation.routingNodes().nodeInterleavedShardIterator(movePrimaryFirst); it.hasNext();) { + for (Iterator it = allocation.routingNodes().nodeInterleavedShardIterator(getShardMovementStrategy()); it + .hasNext();) { // Verify if the cluster concurrent recoveries have been reached. if (allocation.deciders().canMoveAnyShard(allocation).type() != Decision.Type.YES) { logger.info( @@ -573,8 +594,8 @@ void moveShards() { continue; } - // Ensure that replicas don't relocate if primaries are being throttled and primary first is enabled - if (movePrimaryFirst && primariesThrottled && !shardRouting.primary()) { + // Ensure that replicas don't relocate if primaries are being throttled and primary first shard movement strategy is enabled + if ((shardMovementStrategy == ShardMovementStrategy.PRIMARY_FIRST) && primariesThrottled && !shardRouting.primary()) { logger.info( "Cannot move any replica shard in the cluster as movePrimaryFirst is enabled and primary shards" + "are being throttled. Skipping shard iteration" diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/NodeVersionAllocationDecider.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/NodeVersionAllocationDecider.java index 19b7494c000de..9344b4c87830d 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/NodeVersionAllocationDecider.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/NodeVersionAllocationDecider.java @@ -32,12 +32,18 @@ package org.opensearch.cluster.routing.allocation.decider; +import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.routing.RecoverySource; import org.opensearch.cluster.routing.RecoverySource.SnapshotRecoverySource; import org.opensearch.cluster.routing.RoutingNode; import org.opensearch.cluster.routing.RoutingNodes; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.allocation.RoutingAllocation; +import org.opensearch.common.settings.Settings; +import org.opensearch.indices.replication.common.ReplicationType; + +import java.util.List; +import java.util.stream.Collectors; /** * An allocation decider that prevents relocation or allocation from nodes @@ -52,9 +58,35 @@ public class NodeVersionAllocationDecider extends AllocationDecider { public static final String NAME = "node_version"; + private final ReplicationType replicationType; + + public NodeVersionAllocationDecider(Settings settings) { + replicationType = IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.get(settings); + } + @Override public Decision canAllocate(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) { if (shardRouting.primary()) { + if (replicationType == ReplicationType.SEGMENT) { + List replicas = allocation.routingNodes() + .assignedShards(shardRouting.shardId()) + .stream() + .filter(shr -> !shr.primary() && shr.active()) + .collect(Collectors.toList()); + for (ShardRouting replica : replicas) { + // can not allocate if target node version > any existing replica version + RoutingNode replicaNode = allocation.routingNodes().node(replica.currentNodeId()); + if (node.node().getVersion().after(replicaNode.node().getVersion())) { + return allocation.decision( + Decision.NO, + NAME, + "When segment replication is enabled, cannot relocate primary shard to a node with version [%s] if it has a replica on older version [%s]", + node.node().getVersion(), + replicaNode.node().getVersion() + ); + } + } + } if (shardRouting.currentNodeId() == null) { if (shardRouting.recoverySource() != null && shardRouting.recoverySource().getType() == RecoverySource.Type.SNAPSHOT) { // restoring from a snapshot - check that the node can handle the version diff --git a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java index 802eb7bd01254..064edbc76bd8b 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -240,6 +240,7 @@ public void apply(Settings value, Settings current, Settings previous) { BalancedShardsAllocator.SHARD_BALANCE_FACTOR_SETTING, BalancedShardsAllocator.PREFER_PRIMARY_SHARD_BALANCE, BalancedShardsAllocator.SHARD_MOVE_PRIMARY_FIRST_SETTING, + BalancedShardsAllocator.SHARD_MOVEMENT_STRATEGY_SETTING, BalancedShardsAllocator.THRESHOLD_SETTING, BreakerSettings.CIRCUIT_BREAKER_LIMIT_SETTING, BreakerSettings.CIRCUIT_BREAKER_OVERHEAD_SETTING, diff --git a/server/src/test/java/org/opensearch/cluster/routing/RoutingNodesTests.java b/server/src/test/java/org/opensearch/cluster/routing/RoutingNodesTests.java index 73136a71bc12a..23056ec5782bb 100644 --- a/server/src/test/java/org/opensearch/cluster/routing/RoutingNodesTests.java +++ b/server/src/test/java/org/opensearch/cluster/routing/RoutingNodesTests.java @@ -124,7 +124,7 @@ private IndexMetadata.Builder createIndexMetadata(String indexName) { .numberOfShards(this.numberOfShards); } - public void testInterleavedShardIterator() { + public void testInterleavedShardIteratorPrimaryFirst() { // Initialize all the shards for test index 1 and 2 initPrimaries(); startInitializingShards(TEST_INDEX_1); @@ -147,7 +147,8 @@ public void testInterleavedShardIterator() { } // Get primary first shard iterator and assert primary shards are iterated over first - final Iterator iterator = this.clusterState.getRoutingNodes().nodeInterleavedShardIterator(true); + final Iterator iterator = this.clusterState.getRoutingNodes() + .nodeInterleavedShardIterator(ShardMovementStrategy.PRIMARY_FIRST); boolean iteratingPrimary = true; int shardCount = 0; while (iterator.hasNext()) { @@ -155,11 +156,54 @@ public void testInterleavedShardIterator() { if (iteratingPrimary) { iteratingPrimary = shard.primary(); } else { - assert shard.primary() == false; + assertFalse(shard.primary()); } shardCount++; } - assert shardCount == this.totalNumberOfShards; + assertEquals(shardCount, this.totalNumberOfShards); + } + + public void testInterleavedShardIteratorNoPreference() { + // Initialize all the shards for test index 1 and 2 + initPrimaries(); + startInitializingShards(TEST_INDEX_1); + startInitializingShards(TEST_INDEX_1); + startInitializingShards(TEST_INDEX_2); + startInitializingShards(TEST_INDEX_2); + + final Iterator iterator = this.clusterState.getRoutingNodes() + .nodeInterleavedShardIterator(ShardMovementStrategy.NO_PREFERENCE); + int shardCount = 0; + while (iterator.hasNext()) { + final ShardRouting shard = iterator.next(); + shardCount++; + } + assertEquals(shardCount, this.totalNumberOfShards); + } + + public void testInterleavedShardIteratorReplicaFirst() { + // Initialize all the shards for test index 1 and 2 + initPrimaries(); + startInitializingShards(TEST_INDEX_1); + startInitializingShards(TEST_INDEX_1); + startInitializingShards(TEST_INDEX_2); + startInitializingShards(TEST_INDEX_2); + + // Get replica first shard iterator and assert replica shards are iterated over first + final Iterator iterator = this.clusterState.getRoutingNodes() + .nodeInterleavedShardIterator(ShardMovementStrategy.REPLICA_FIRST); + boolean iteratingReplica = true; + int shardCount = 0; + while (iterator.hasNext()) { + final ShardRouting shard = iterator.next(); + if (iteratingReplica) { + iteratingReplica = shard.primary() == false; + } else { + assertTrue(shard.primary()); + } + shardCount++; + } + assertEquals(shardCount, this.totalNumberOfShards); } public void testSwapPrimaryWithReplica() { diff --git a/server/src/test/java/org/opensearch/cluster/routing/MovePrimaryFirstTests.java b/server/src/test/java/org/opensearch/cluster/routing/ShardMovementStrategyTests.java similarity index 60% rename from server/src/test/java/org/opensearch/cluster/routing/MovePrimaryFirstTests.java rename to server/src/test/java/org/opensearch/cluster/routing/ShardMovementStrategyTests.java index a30581e2576e2..12994bdfcf6d5 100644 --- a/server/src/test/java/org/opensearch/cluster/routing/MovePrimaryFirstTests.java +++ b/server/src/test/java/org/opensearch/cluster/routing/ShardMovementStrategyTests.java @@ -22,7 +22,7 @@ @OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) @ThreadLeakScope(ThreadLeakScope.Scope.NONE) -public class MovePrimaryFirstTests extends OpenSearchIntegTestCase { +public class ShardMovementStrategyTests extends OpenSearchIntegTestCase { protected String startDataOnlyNode(final String zone) { final Settings settings = Settings.builder().put("node.attr.zone", zone).build(); @@ -48,14 +48,48 @@ protected void createAndIndex(String index, int replicaCount, int shardCount) { flushAndRefresh(index); } + private static Settings.Builder getSettings(ShardMovementStrategy shardMovementStrategy, boolean movePrimaryFirst) { + return Settings.builder() + .put("cluster.routing.allocation.shard_movement_strategy", shardMovementStrategy) + .put("cluster.routing.allocation.move.primary_first", movePrimaryFirst); + } + + public void testClusterGreenAfterPartialRelocationPrimaryFirstShardMovementMovePrimarySettingEnabled() throws InterruptedException { + testClusterGreenAfterPartialRelocation(ShardMovementStrategy.PRIMARY_FIRST, true); + } + + public void testClusterGreenAfterPartialRelocationPrimaryFirstShardMovementMovePrimarySettingDisabled() throws InterruptedException { + testClusterGreenAfterPartialRelocation(ShardMovementStrategy.PRIMARY_FIRST, false); + } + + public void testClusterGreenAfterPartialRelocationReplicaFirstShardMovementPrimaryFirstEnabled() throws InterruptedException { + testClusterGreenAfterPartialRelocation(ShardMovementStrategy.REPLICA_FIRST, true); + } + + public void testClusterGreenAfterPartialRelocationReplicaFirstShardMovementPrimaryFirstDisabled() throws InterruptedException { + testClusterGreenAfterPartialRelocation(ShardMovementStrategy.REPLICA_FIRST, false); + } + + public void testClusterGreenAfterPartialRelocationNoPreferenceShardMovementPrimaryFirstEnabled() throws InterruptedException { + testClusterGreenAfterPartialRelocation(ShardMovementStrategy.NO_PREFERENCE, true); + } + + private boolean shouldMovePrimaryShardsFirst(ShardMovementStrategy shardMovementStrategy, boolean movePrimaryFirst) { + if (shardMovementStrategy == ShardMovementStrategy.NO_PREFERENCE && movePrimaryFirst) { + return true; + } + return shardMovementStrategy == ShardMovementStrategy.PRIMARY_FIRST; + } + /** * Creates two nodes each in two zones and shuts down nodes in zone1 after * relocating half the number of shards. Shards per node constraint ensures * that exactly 50% of shards relocate to nodes in zone2 giving time to shut down - * nodes in zone1. Since primaries are relocated first as movePrimaryFirst is - * enabled, cluster should not become red and zone2 nodes have all the primaries + * nodes in zone1. Depending on the shard movement strategy, we check whether the + * primary or replica shards are moved first, and zone2 nodes have all the shards */ - public void testClusterGreenAfterPartialRelocation() throws InterruptedException { + private void testClusterGreenAfterPartialRelocation(ShardMovementStrategy shardMovementStrategy, boolean movePrimaryFirst) + throws InterruptedException { internalCluster().startClusterManagerOnlyNodes(1); final String z1 = "zone-1", z2 = "zone-2"; final int primaryShardCount = 6; @@ -73,7 +107,7 @@ public void testClusterGreenAfterPartialRelocation() throws InterruptedException // zone nodes excluded to prevent any shard relocation ClusterUpdateSettingsRequest settingsRequest = new ClusterUpdateSettingsRequest(); settingsRequest.persistentSettings( - Settings.builder().put("cluster.routing.allocation.move.primary_first", true).put("cluster.routing.allocation.exclude.zone", z2) + getSettings(shardMovementStrategy, movePrimaryFirst).put("cluster.routing.allocation.exclude.zone", z2) ); client().admin().cluster().updateSettings(settingsRequest).actionGet(); @@ -82,7 +116,7 @@ public void testClusterGreenAfterPartialRelocation() throws InterruptedException // Create cluster state listener to compute number of shards on new zone // nodes before counting down the latch - final CountDownLatch primaryMoveLatch = new CountDownLatch(1); + final CountDownLatch shardMoveLatch = new CountDownLatch(1); final ClusterStateListener listener = event -> { if (event.routingTableChanged()) { final RoutingNodes routingNodes = event.state().getRoutingNodes(); @@ -91,13 +125,22 @@ public void testClusterGreenAfterPartialRelocation() throws InterruptedException RoutingNode routingNode = it.next(); final String nodeName = routingNode.node().getName(); if (nodeName.equals(z2n1) || nodeName.equals(z2n2)) { - startedCount += routingNode.numberOfShardsWithState(ShardRoutingState.STARTED); + int count = 0; + for (ShardRouting shardEntry : routingNode) { + // If shard movement strategy is primary first, asserting that primary shards are moved first; else assert + // shards are replicas + if ((shardEntry.primary() == shouldMovePrimaryShardsFirst(shardMovementStrategy, movePrimaryFirst)) + && shardEntry.state() == ShardRoutingState.STARTED) { + count++; + } + } + startedCount += count; } } - // Count down the latch once all the primary shards have initialized on nodes in zone-2 + // Count down the latch once all the shards have initialized on nodes in zone-2 if (startedCount == primaryShardCount) { - primaryMoveLatch.countDown(); + shardMoveLatch.countDown(); } } }; @@ -108,12 +151,12 @@ public void testClusterGreenAfterPartialRelocation() throws InterruptedException settingsRequest.persistentSettings( Settings.builder() .put("cluster.routing.allocation.exclude.zone", z1) - // Total shards per node constraint is added to pause the relocation after primary shards + // Total shards per node constraint is added to pause the relocation after shards // have relocated to allow time for node shutdown and validate yellow cluster .put("cluster.routing.allocation.total_shards_per_node", primaryShardCount / 2) ); client().admin().cluster().updateSettings(settingsRequest); - primaryMoveLatch.await(); + shardMoveLatch.await(); // Shutdown both nodes in zone 1 and ensure cluster does not become red try { diff --git a/server/src/test/java/org/opensearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java b/server/src/test/java/org/opensearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java index 557d7db142671..d4ec340766ec9 100644 --- a/server/src/test/java/org/opensearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java +++ b/server/src/test/java/org/opensearch/cluster/routing/allocation/NodeVersionAllocationDeciderTests.java @@ -67,6 +67,7 @@ import org.opensearch.common.util.set.Sets; import org.opensearch.core.index.Index; import org.opensearch.core.index.shard.ShardId; +import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.repositories.IndexId; import org.opensearch.snapshots.EmptySnapshotsInfoService; import org.opensearch.snapshots.InternalSnapshotsInfoService; @@ -439,7 +440,9 @@ public void testRebalanceDoesNotAllocatePrimaryAndReplicasOnDifferentVersionNode .routingTable(routingTable) .nodes(DiscoveryNodes.builder().add(newNode).add(oldNode1).add(oldNode2)) .build(); - AllocationDeciders allocationDeciders = new AllocationDeciders(Collections.singleton(new NodeVersionAllocationDecider())); + AllocationDeciders allocationDeciders = new AllocationDeciders( + Collections.singleton(new NodeVersionAllocationDecider(Settings.EMPTY)) + ); AllocationService strategy = new MockAllocationService( allocationDeciders, new TestGatewayAllocator(), @@ -509,7 +512,7 @@ public void testRestoreDoesNotAllocateSnapshotOnOlderNodes() { .nodes(DiscoveryNodes.builder().add(newNode).add(oldNode1).add(oldNode2)) .build(); AllocationDeciders allocationDeciders = new AllocationDeciders( - Arrays.asList(new ReplicaAfterPrimaryActiveAllocationDecider(), new NodeVersionAllocationDecider()) + Arrays.asList(new ReplicaAfterPrimaryActiveAllocationDecider(), new NodeVersionAllocationDecider(Settings.EMPTY)) ); AllocationService strategy = new MockAllocationService( allocationDeciders, @@ -526,6 +529,148 @@ public void testRestoreDoesNotAllocateSnapshotOnOlderNodes() { } } + public void testRebalanceDoesNotAllocatePrimaryOnHigherVersionNodesSegrepEnabled() { + ShardId shard1 = new ShardId("test1", "_na_", 0); + ShardId shard2 = new ShardId("test2", "_na_", 0); + final DiscoveryNode newNode1 = new DiscoveryNode( + "newNode1", + buildNewFakeTransportAddress(), + emptyMap(), + CLUSTER_MANAGER_DATA_ROLES, + Version.CURRENT + ); + final DiscoveryNode newNode2 = new DiscoveryNode( + "newNode2", + buildNewFakeTransportAddress(), + emptyMap(), + CLUSTER_MANAGER_DATA_ROLES, + Version.CURRENT + ); + final DiscoveryNode oldNode1 = new DiscoveryNode( + "oldNode1", + buildNewFakeTransportAddress(), + emptyMap(), + CLUSTER_MANAGER_DATA_ROLES, + VersionUtils.getPreviousVersion() + ); + final DiscoveryNode oldNode2 = new DiscoveryNode( + "oldNode2", + buildNewFakeTransportAddress(), + emptyMap(), + CLUSTER_MANAGER_DATA_ROLES, + VersionUtils.getPreviousVersion() + ); + AllocationId allocationId1P = AllocationId.newInitializing(); + AllocationId allocationId1R = AllocationId.newInitializing(); + AllocationId allocationId2P = AllocationId.newInitializing(); + AllocationId allocationId2R = AllocationId.newInitializing(); + + Settings segmentReplicationSettings = Settings.builder() + .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) + .build(); + Metadata metadata = Metadata.builder() + .put( + IndexMetadata.builder(shard1.getIndexName()) + .settings(settings(Version.CURRENT).put(segmentReplicationSettings)) + .numberOfShards(1) + .numberOfReplicas(1) + .putInSyncAllocationIds(0, Sets.newHashSet(allocationId1P.getId(), allocationId1R.getId())) + ) + .put( + IndexMetadata.builder(shard2.getIndexName()) + .settings(settings(Version.CURRENT).put(segmentReplicationSettings)) + .numberOfShards(1) + .numberOfReplicas(1) + .putInSyncAllocationIds(0, Sets.newHashSet(allocationId2P.getId(), allocationId2R.getId())) + ) + .build(); + RoutingTable routingTable = RoutingTable.builder() + .add( + IndexRoutingTable.builder(shard1.getIndex()) + .addIndexShard( + new IndexShardRoutingTable.Builder(shard1).addShard( + TestShardRouting.newShardRouting( + shard1.getIndexName(), + shard1.getId(), + oldNode1.getId(), + null, + true, + ShardRoutingState.STARTED, + allocationId1P + ) + ) + .addShard( + TestShardRouting.newShardRouting( + shard1.getIndexName(), + shard1.getId(), + oldNode2.getId(), + null, + false, + ShardRoutingState.STARTED, + allocationId1R + ) + ) + .build() + ) + ) + .add( + IndexRoutingTable.builder(shard2.getIndex()) + .addIndexShard( + new IndexShardRoutingTable.Builder(shard2).addShard( + TestShardRouting.newShardRouting( + shard2.getIndexName(), + shard2.getId(), + oldNode2.getId(), + null, + true, + ShardRoutingState.STARTED, + allocationId2P + ) + ) + .addShard( + TestShardRouting.newShardRouting( + shard2.getIndexName(), + shard2.getId(), + oldNode1.getId(), + null, + false, + ShardRoutingState.STARTED, + allocationId2R + ) + ) + .build() + ) + ) + .build(); + ClusterState state = ClusterState.builder(org.opensearch.cluster.ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)) + .metadata(metadata) + .routingTable(routingTable) + .nodes(DiscoveryNodes.builder().add(newNode1).add(newNode2).add(oldNode1).add(oldNode2)) + .build(); + AllocationDeciders allocationDeciders = new AllocationDeciders( + Collections.singleton(new NodeVersionAllocationDecider(segmentReplicationSettings)) + ); + AllocationService strategy = new MockAllocationService( + allocationDeciders, + new TestGatewayAllocator(), + new BalancedShardsAllocator(Settings.EMPTY), + EmptyClusterInfoService.INSTANCE, + EmptySnapshotsInfoService.INSTANCE + ); + state = strategy.reroute(state, new AllocationCommands(), true, false).getClusterState(); + // the two indices must stay as is, the replicas cannot move to oldNode2 because versions don't match + assertThat(state.routingTable().index(shard2.getIndex()).shardsWithState(ShardRoutingState.RELOCATING).size(), equalTo(1)); + assertThat( + state.routingTable().index(shard2.getIndex()).shardsWithState(ShardRoutingState.RELOCATING).get(0).primary(), + equalTo(false) + ); + assertThat(state.routingTable().index(shard1.getIndex()).shardsWithState(ShardRoutingState.RELOCATING).size(), equalTo(1)); + assertThat( + state.routingTable().index(shard1.getIndex()).shardsWithState(ShardRoutingState.RELOCATING).get(0).primary(), + equalTo(false) + ); + } + private ClusterState stabilize(ClusterState clusterState, AllocationService service) { logger.trace("RoutingNodes: {}", clusterState.getRoutingNodes()); @@ -626,7 +771,7 @@ public void testMessages() { RoutingAllocation routingAllocation = new RoutingAllocation(null, clusterState.getRoutingNodes(), clusterState, null, null, 0); routingAllocation.debugDecision(true); - final NodeVersionAllocationDecider allocationDecider = new NodeVersionAllocationDecider(); + final NodeVersionAllocationDecider allocationDecider = new NodeVersionAllocationDecider(Settings.EMPTY); Decision decision = allocationDecider.canAllocate(primaryShard, newNode, routingAllocation); assertThat(decision.type(), is(Decision.Type.YES)); assertThat(decision.getExplanation(), is("the primary shard is new or already existed on the node")); From bb7893048153c1bbd205b2cc4c5321504b37efc3 Mon Sep 17 00:00:00 2001 From: Poojita Raj Date: Fri, 4 Aug 2023 20:10:24 -0700 Subject: [PATCH 10/19] Mute flaky test ReplicaToPrimaryPromotionIT.testFailoverWhileIndexing (#9131) Signed-off-by: Poojita Raj --- .../org/opensearch/remotestore/ReplicaToPrimaryPromotionIT.java | 1 + 1 file changed, 1 insertion(+) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/ReplicaToPrimaryPromotionIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/ReplicaToPrimaryPromotionIT.java index c73e7f603b09b..b68fd1f764a63 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/ReplicaToPrimaryPromotionIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/ReplicaToPrimaryPromotionIT.java @@ -122,6 +122,7 @@ public void testPromoteReplicaToPrimary() throws Exception { assertHitCount(client().prepareSearch(indexName).setSize(0).get(), numOfDocs); } + @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/9130") public void testFailoverWhileIndexing() throws Exception { internalCluster().startNode(); internalCluster().startNode(); From 4c5981091fcff67830bc6a9f2f9dee9469dffae7 Mon Sep 17 00:00:00 2001 From: Nick Knize Date: Sat, 5 Aug 2023 00:34:36 -0500 Subject: [PATCH 11/19] [Refactor] Remaining Strings utility methods to core library (#9103) This commit refactors the remaining Strings# utility methods from the duplicate :server:Strings class to the :libs:opensearch-core:Strings class. The CollectionUtils class (used by the Strings utility) is also refactored to the :libs:opensearch-common library and the unnecessary Strings.toString(XContentBuilder... static method is finally refactored as an instance method in XContentBuilder. The Strings class is also cleaned up for clarity. Signed-off-by: Nicholas Walter Knize --- .../client/indices/CreateIndexRequest.java | 3 +- .../client/slm/SnapshotLifecyclePolicy.java | 2 +- .../slm/SnapshotLifecyclePolicyMetadata.java | 2 +- .../client/slm/SnapshotLifecycleStats.java | 2 +- .../slm/SnapshotRetentionConfiguration.java | 2 +- .../opensearch/client/BulkProcessorIT.java | 3 +- .../java/org/opensearch/client/SearchIT.java | 5 +- .../documentation/CRUDDocumentationIT.java | 19 +- .../opensearch/test/rest/NodeRestUsageIT.java | 2 +- .../org/opensearch/core/common/Strings.java | 77 ++ .../core/xcontent/XContentBuilder.java | 10 + ...ultShardOperationFailedExceptionTests.java | 2 +- .../opensearch/core/common/StringsTests.java | 51 + .../common/xcontent/ObjectParserTests.java | 3 +- .../common/xcontent/SimpleStruct.java | 2 +- .../common/xcontent/XContentParserTests.java | 17 +- ...ractGeoAggregatorModulePluginTestCase.java | 3 +- .../mustache/MultiSearchTemplateIT.java | 18 +- .../mustache/CustomMustacheFactory.java | 3 +- .../mustache/MultiSearchTemplateResponse.java | 2 +- .../mustache/SearchTemplateRequest.java | 7 +- .../mapper/RankFeatureFieldMapperTests.java | 3 +- .../RankFeatureMetaFieldMapperTests.java | 32 +- .../mapper/RankFeaturesFieldMapperTests.java | 3 +- .../mapper/ScaledFloatFieldMapperTests.java | 3 +- .../SearchAsYouTypeFieldMapperTests.java | 2 +- .../query/RankFeatureQueryBuilderTests.java | 19 +- .../join/query/ParentChildTestCase.java | 3 +- .../mapper/ParentJoinFieldMapperTests.java | 474 +++---- .../join/query/HasChildQueryBuilderTests.java | 3 +- .../query/HasParentQueryBuilderTests.java | 3 +- .../join/query/ParentIdQueryBuilderTests.java | 3 +- .../percolator/CandidateQueryTests.java | 83 +- .../PercolateQueryBuilderTests.java | 7 +- .../PercolateWithNestedQueryBuilderTests.java | 3 +- .../PercolatorFieldMapperTests.java | 221 ++- .../index/rankeval/RankEvalResponse.java | 2 +- .../index/rankeval/RankEvalSpec.java | 2 +- .../index/rankeval/RatedDocument.java | 2 +- .../index/rankeval/RatedRequest.java | 2 +- .../DiscountedCumulativeGainTests.java | 2 +- .../index/rankeval/RankEvalSpecTests.java | 3 +- .../reindex/remote/RemoteRequestBuilders.java | 7 +- .../RepositoryURLClientYamlTestSuiteIT.java | 3 +- .../rest/Netty4HeadBodyIsEmptyIT.java | 9 +- .../ICUCollationKeywordFieldMapperTests.java | 3 +- .../org/opensearch/cloud/gce/GCEFixture.java | 61 +- .../CorrelationVectorsEngineIT.java | 12 +- .../query/CorrelationQueryBuilderTests.java | 2 +- .../AnnotatedTextFieldMapperTests.java | 192 ++- .../index/mapper/size/SizeMappingTests.java | 20 +- ...rossClusterSearchUnavailableClusterIT.java | 3 +- .../upgrades/FullClusterRestartIT.java | 44 +- .../upgrades/QueryBuilderBWCIT.java | 3 +- .../org/opensearch/backwards/IndexingIT.java | 6 +- .../java/org/opensearch/search/CCSDuelIT.java | 4 +- .../org/opensearch/upgrades/RecoveryIT.java | 2 +- .../http/SearchRestCancellationIT.java | 3 +- .../opensearch/upgrades/TranslogPolicyIT.java | 2 +- .../ClusterAllocationExplainIT.java | 3 +- .../action/bulk/BulkWithUpdatesIT.java | 2 +- .../cluster/routing/PrimaryAllocationIT.java | 2 +- .../document/DocumentActionsIT.java | 15 +- .../gateway/RecoveryFromGatewayIT.java | 45 +- .../java/org/opensearch/get/GetActionIT.java | 25 +- .../mapper/CopyToMapperIntegrationIT.java | 20 +- .../opensearch/index/shard/IndexShardIT.java | 2 +- .../mapping/SimpleGetFieldMappingsIT.java | 9 +- .../RandomExceptionCircuitBreakerIT.java | 31 +- .../indices/recovery/IndexRecoveryIT.java | 2 +- .../indices/state/OpenCloseIndexIT.java | 20 +- .../search/SearchCancellationIT.java | 2 +- .../aggregations/bucket/DateHistogramIT.java | 20 +- .../SignificantTermsSignificanceScoreIT.java | 3 +- .../basic/SearchWithRandomExceptionsIT.java | 20 +- .../basic/SearchWithRandomIOExceptionsIT.java | 20 +- .../highlight/HighlighterSearchIT.java | 77 +- .../search/fields/SearchFieldsIT.java | 326 +++-- .../opensearch/search/geo/GeoFilterIT.java | 22 +- .../search/geo/GeoShapeIntegrationIT.java | 118 +- .../geo/LegacyGeoShapeIntegrationIT.java | 122 +- .../search/morelikethis/MoreLikeThisIT.java | 9 +- .../org/opensearch/search/query/ExistsIT.java | 10 +- .../search/query/SimpleQueryStringIT.java | 43 +- .../search/slice/SearchSliceIT.java | 38 +- .../opensearch/search/sort/SimpleSortIT.java | 59 +- .../search/suggest/SuggestSearchIT.java | 61 +- .../cluster/health/ClusterHealthResponse.java | 2 +- .../cluster/node/info/NodesInfoResponse.java | 3 +- .../NodesReloadSecureSettingsResponse.java | 3 +- .../node/stats/NodesStatsResponse.java | 3 +- .../tasks/cancel/CancelTasksResponse.java | 2 +- .../node/tasks/get/GetTaskResponse.java | 2 +- .../node/tasks/list/ListTasksResponse.java | 2 +- .../node/usage/NodesUsageResponse.java | 3 +- .../restore/RestoreRemoteStoreRequest.java | 2 +- .../stats/RemoteStoreStatsResponse.java | 2 +- .../verify/VerifyRepositoryResponse.java | 2 +- .../settings/ClusterGetSettingsResponse.java | 2 +- .../snapshots/clone/CloneSnapshotRequest.java | 2 +- .../create/CreateSnapshotRequest.java | 8 +- .../snapshots/get/GetSnapshotsResponse.java | 2 +- .../restore/RestoreSnapshotRequest.java | 2 +- .../snapshots/status/SnapshotStats.java | 2 +- .../snapshots/status/SnapshotStatus.java | 2 +- .../admin/cluster/stats/AnalysisStats.java | 2 +- .../cluster/stats/ClusterStatsResponse.java | 3 +- .../admin/cluster/stats/MappingStats.java | 2 +- .../action/admin/indices/alias/Alias.java | 16 +- .../indices/alias/IndicesAliasesRequest.java | 4 +- .../admin/indices/analyze/AnalyzeAction.java | 2 +- .../indices/close/CloseIndexResponse.java | 2 +- .../indices/create/CreateIndexRequest.java | 5 +- .../admin/indices/get/GetIndexResponse.java | 2 +- .../mapping/get/GetMappingsResponse.java | 2 +- .../readonly/AddIndexBlockResponse.java | 2 +- .../indices/recovery/RecoveryResponse.java | 2 +- .../SegmentReplicationStatsResponse.java | 2 +- .../admin/indices/rollover/RolloverInfo.java | 2 +- .../settings/get/GetSettingsResponse.java | 3 +- .../settings/put/UpdateSettingsRequest.java | 2 +- .../indices/stats/IndicesStatsResponse.java | 2 +- .../template/put/PutIndexTemplateRequest.java | 3 +- .../action/bulk/BulkItemRequest.java | 2 +- .../action/bulk/BulkItemResponse.java | 2 +- .../action/fieldcaps/FieldCapabilities.java | 2 +- .../fieldcaps/FieldCapabilitiesResponse.java | 2 +- .../opensearch/action/get/GetResponse.java | 2 +- .../action/get/MultiGetRequest.java | 2 +- .../action/index/IndexResponse.java | 2 +- .../action/ingest/GetPipelineResponse.java | 2 +- .../search/GetSearchPipelineResponse.java | 2 +- .../action/search/MultiSearchResponse.java | 2 +- .../action/search/SearchResponse.java | 2 +- .../support/replication/ReplicationTask.java | 2 +- .../cluster/RepositoryCleanupInProgress.java | 2 +- .../cluster/SnapshotsInProgress.java | 2 +- .../ClusterAwarenessAttributeValueHealth.java | 2 +- .../ClusterAwarenessAttributesHealth.java | 2 +- .../cluster/coordination/Coordinator.java | 2 +- .../DecommissionAttributeMetadata.java | 2 +- .../cluster/health/ClusterShardHealth.java | 2 +- .../cluster/metadata/AliasMetadata.java | 2 +- .../cluster/metadata/ComponentTemplate.java | 2 +- .../metadata/ComponentTemplateMetadata.java | 2 +- .../metadata/ComposableIndexTemplate.java | 2 +- .../ComposableIndexTemplateMetadata.java | 2 +- .../cluster/metadata/DataStreamMetadata.java | 2 +- .../metadata/IndexTemplateMetadata.java | 7 +- .../MetadataIndexTemplateService.java | 16 +- .../metadata/RepositoriesMetadata.java | 2 +- .../opensearch/cluster/metadata/Template.java | 22 +- .../metadata/WeightedRoutingMetadata.java | 2 +- .../allocation/DiskThresholdSettings.java | 4 +- .../command/AllocationCommands.java | 2 +- .../java/org/opensearch/common/Strings.java | 133 -- .../common/geo/builders/ShapeBuilder.java | 2 +- .../opensearch/common/settings/Setting.java | 6 +- .../opensearch/common/settings/Settings.java | 12 +- .../common/settings/SettingsModule.java | 3 +- .../common/xcontent/XContentHelper.java | 3 +- .../org/opensearch/index/get/GetResult.java | 2 +- .../index/mapper/MapperService.java | 10 +- .../org/opensearch/index/mapper/Mapping.java | 3 +- .../index/mapper/RootObjectMapper.java | 2 +- .../index/query/AbstractQueryBuilder.java | 2 +- .../index/query/InnerHitBuilder.java | 2 +- .../index/query/MoreLikeThisQueryBuilder.java | 4 +- .../index/query/SpanNearQueryBuilder.java | 4 +- .../index/reindex/BulkByScrollTask.java | 2 +- .../index/reindex/ScrollableHitSource.java | 4 +- .../index/search/stats/SearchStats.java | 2 +- .../index/shard/PrimaryReplicaSyncer.java | 2 +- .../RemoveCorruptedShardDataCommand.java | 2 +- .../index/translog/TranslogStats.java | 2 +- .../common/ReplicationLuceneIndex.java | 3 +- .../ingest/PipelineConfiguration.java | 2 +- .../PersistentTasksCustomMetadata.java | 2 +- .../PersistentTasksNodeService.java | 2 +- .../repositories/RepositoryCleanupResult.java | 2 +- .../repositories/RepositoryInfo.java | 2 +- .../repositories/RepositoryStatsSnapshot.java | 2 +- .../java/org/opensearch/script/Script.java | 3 +- .../opensearch/script/ScriptException.java | 3 +- .../opensearch/script/StoredScriptSource.java | 3 +- .../java/org/opensearch/search/SearchHit.java | 2 +- .../aggregations/AggregationBuilder.java | 2 +- .../aggregations/AggregatorFactories.java | 2 +- .../search/aggregations/BucketOrder.java | 2 +- .../aggregations/InternalAggregation.java | 2 +- .../PipelineAggregationBuilder.java | 2 +- .../BaseMultiValuesSourceFieldConfig.java | 2 +- .../highlight/AbstractHighlighterBuilder.java | 2 +- .../pipeline/PipelineConfiguration.java | 2 +- .../search/rescore/RescorerBuilder.java | 2 +- .../searchafter/SearchAfterBuilder.java | 3 +- .../opensearch/search/slice/SliceBuilder.java | 4 +- .../opensearch/search/sort/SortBuilder.java | 2 +- .../opensearch/search/suggest/Suggest.java | 2 +- .../search/suggest/SuggestBuilder.java | 2 +- .../completion/context/ContextMapping.java | 3 +- .../DirectCandidateGeneratorBuilder.java | 3 +- .../org/opensearch/snapshots/RestoreInfo.java | 2 +- .../org/opensearch/tasks/RawTaskStatus.java | 2 +- .../java/org/opensearch/tasks/TaskId.java | 2 +- .../java/org/opensearch/tasks/TaskInfo.java | 2 +- .../opensearch/tasks/TaskResourceStats.java | 2 +- .../opensearch/tasks/TaskResourceUsage.java | 2 +- .../java/org/opensearch/tasks/TaskResult.java | 2 +- .../org/opensearch/tasks/TaskThreadUsage.java | 2 +- .../transport/RemoteClusterService.java | 2 +- .../ExceptionSerializationTests.java | 2 +- .../opensearch/OpenSearchExceptionTests.java | 35 +- .../ClusterAllocationExplainActionTests.java | 3 +- .../ClusterAllocationExplanationTests.java | 3 +- .../node/tasks/TransportTasksActionTests.java | 10 +- .../reroute/ClusterRerouteResponseTests.java | 7 +- .../indices/alias/AliasActionsTests.java | 3 +- .../close/CloseIndexResponseTests.java | 2 +- .../create/CreateIndexRequestTests.java | 10 +- .../create/CreateIndexResponseTests.java | 2 +- .../get/GetFieldMappingsResponseTests.java | 2 +- .../mapping/put/PutMappingRequestTests.java | 2 +- .../MetadataRolloverServiceTests.java | 3 +- .../indices/shrink/ResizeRequestTests.java | 2 +- .../indices/shrink/ResizeResponseTests.java | 2 +- .../indices/stats/IndicesStatsTests.java | 2 +- .../query/ValidateQueryResponseTests.java | 2 +- .../action/delete/DeleteResponseTests.java | 2 +- .../action/get/GetResponseTests.java | 2 +- .../action/index/IndexResponseTests.java | 2 +- .../action/main/MainResponseTests.java | 3 +- .../search/ClearScrollRequestTests.java | 3 +- .../search/MultiSearchRequestTests.java | 2 +- .../SearchPhaseExecutionExceptionTests.java | 2 +- .../action/search/SearchResponseTests.java | 13 +- .../search/SearchScrollRequestTests.java | 3 +- .../replication/ReplicationResponseTests.java | 2 +- .../action/update/UpdateResponseTests.java | 2 +- .../opensearch/cluster/ClusterStateTests.java | 9 +- .../cluster/metadata/IndexGraveyardTests.java | 2 +- .../cluster/metadata/IndexMetadataTests.java | 2 +- .../MetadataCreateIndexServiceTests.java | 24 +- .../MetadataIndexTemplateServiceTests.java | 22 +- .../cluster/metadata/MetadataTests.java | 4 +- .../metadata/ToAndFromJsonMetadataTests.java | 11 +- .../ClusterStateToStringTests.java | 2 +- .../org/opensearch/common/StringsTests.java | 90 -- .../common/geo/GeoJsonParserTests.java | 322 +++-- .../common/geo/GeoJsonShapeParserTests.java | 1228 ++++++++--------- .../common/geo/GeometryParserTests.java | 9 +- .../common/settings/SettingsFilterTests.java | 3 +- .../common/settings/SettingsTests.java | 11 +- .../common/xcontent/BaseXContentTestCase.java | 10 +- .../common/xcontent/XContentFactoryTests.java | 5 +- .../builder/XContentBuilderTests.java | 40 +- .../support/XContentMapValuesTests.java | 24 +- .../AbstractXContentFilteringTestCase.java | 3 +- .../opensearch/index/IndexServiceTests.java | 3 +- .../fielddata/BinaryDVFieldDataTests.java | 26 +- .../index/get/DocumentFieldTests.java | 2 +- .../opensearch/index/get/GetResultTests.java | 2 +- .../index/mapper/BooleanFieldMapperTests.java | 5 +- .../mapper/CompletionFieldMapperTests.java | 2 +- .../mapper/DataStreamFieldMapperTests.java | 70 +- .../index/mapper/DocumentParserTests.java | 24 +- .../index/mapper/DynamicMappingTests.java | 2 +- .../index/mapper/DynamicTemplateTests.java | 9 +- .../index/mapper/FieldAliasMapperTests.java | 32 +- .../mapper/FieldNamesFieldMapperTests.java | 102 +- .../mapper/FlatObjectFieldDataTests.java | 24 +- .../mapper/FlatObjectFieldMapperTests.java | 17 +- .../mapper/GeoShapeFieldMapperTests.java | 2 +- .../index/mapper/IdFieldMapperTests.java | 3 +- .../index/mapper/IndexFieldMapperTests.java | 14 +- .../index/mapper/IpRangeFieldMapperTests.java | 53 +- .../index/mapper/KeywordFieldMapperTests.java | 3 +- .../LegacyGeoShapeFieldMapperTests.java | 4 +- .../index/mapper/MapperServiceTests.java | 32 +- .../index/mapper/MultiFieldTests.java | 67 +- .../index/mapper/NestedObjectMapperTests.java | 554 ++++---- .../mapper/NullValueObjectMappingTests.java | 24 +- .../index/mapper/NullValueTests.java | 3 +- .../index/mapper/NumberFieldMapperTests.java | 3 +- .../index/mapper/ObjectMapperTests.java | 501 ++++--- .../index/mapper/ParametrizedMapperTests.java | 60 +- .../index/mapper/RangeFieldMapperTests.java | 5 +- ...angeFieldQueryStringQueryBuilderTests.java | 31 +- .../index/mapper/RootObjectMapperTests.java | 317 +++-- .../index/mapper/RoutingFieldMapperTests.java | 5 +- .../index/mapper/SourceFieldMapperTests.java | 231 ++-- .../mapper/StoredNumericValuesTests.java | 106 +- .../index/mapper/TextFieldMapperTests.java | 27 +- .../index/mapper/UpdateMappingTests.java | 49 +- .../query/GeoShapeQueryBuilderTests.java | 3 +- .../query/IntervalQueryBuilderTests.java | 3 +- .../index/query/MatchQueryBuilderTests.java | 5 +- .../query/MoreLikeThisQueryBuilderTests.java | 3 +- .../index/query/NestedQueryBuilderTests.java | 39 +- .../query/QueryStringQueryBuilderTests.java | 11 +- .../index/query/RangeQueryRewriteTests.java | 47 +- .../query/SpanFirstQueryBuilderTests.java | 5 +- .../query/SpanMultiTermQueryBuilderTests.java | 3 +- .../index/query/SpanNotQueryBuilderTests.java | 9 +- .../index/query/TermsQueryBuilderTests.java | 14 +- .../query/TermsSetQueryBuilderTests.java | 3 +- .../FunctionScoreQueryBuilderTests.java | 76 +- .../index/reindex/BulkByScrollTaskTests.java | 7 +- .../index/search/MultiMatchQueryTests.java | 50 +- .../index/shard/IndexShardTests.java | 18 +- .../shard/PrimaryReplicaSyncerTests.java | 3 +- .../index/similarity/SimilarityTests.java | 22 +- ...oteStoreShardShallowCopySnapshotTests.java | 3 +- .../index/translog/LocalTranslogTests.java | 3 +- .../nodesinfo/NodeInfoStreamingTests.java | 3 +- .../persistent/TestPersistentTasksPlugin.java | 2 +- .../rest/action/RestActionsTests.java | 3 +- .../org/opensearch/script/ScriptTests.java | 12 +- .../script/StoredScriptSourceTests.java | 3 +- .../opensearch/script/StoredScriptTests.java | 30 +- .../search/NestedIdentityTests.java | 5 +- .../org/opensearch/search/SearchHitTests.java | 3 +- .../opensearch/search/SearchHitsTests.java | 3 +- .../search/SearchSortValuesTests.java | 5 +- .../terms/SignificanceHeuristicTests.java | 3 +- .../metrics/AbstractGeoTestCase.java | 3 +- .../metrics/AbstractPercentilesTestCase.java | 3 +- .../metrics/InternalStatsTests.java | 5 +- .../InternalPercentilesBucketTests.java | 3 +- .../highlight/HighlightFieldTests.java | 5 +- .../search/geo/GeoShapeQueryTests.java | 67 +- .../internal/ShardSearchRequestTests.java | 4 +- .../search/profile/ProfileResultTests.java | 13 +- .../profile/query/CollectorResultTests.java | 11 +- .../search/sort/SortBuilderTests.java | 3 +- .../search/sort/SortValueTests.java | 2 +- .../CategoryContextMappingTests.java | 270 ++-- .../RemoteClusterConnectionTests.java | 8 +- .../index/engine/EngineTestCase.java | 24 +- .../index/mapper/FieldMapperTestCase.java | 3 +- .../index/mapper/FieldMapperTestCase2.java | 5 +- .../index/mapper/MapperTestCase.java | 5 +- .../aggregations/BaseAggregationTestCase.java | 2 +- .../AbstractSnapshotIntegTestCase.java | 4 +- .../AbstractBroadcastResponseTestCase.java | 2 +- .../test/AbstractBuilderTestCase.java | 71 +- .../test/AbstractQueryTestCase.java | 8 +- .../test/rest/OpenSearchRestTestCase.java | 18 +- .../rest/yaml/ClientYamlTestResponse.java | 3 +- .../OpenSearchClientYamlSuiteTestCase.java | 4 +- 350 files changed, 4002 insertions(+), 4410 deletions(-) delete mode 100644 server/src/main/java/org/opensearch/common/Strings.java delete mode 100644 server/src/test/java/org/opensearch/common/StringsTests.java diff --git a/client/rest-high-level/src/main/java/org/opensearch/client/indices/CreateIndexRequest.java b/client/rest-high-level/src/main/java/org/opensearch/client/indices/CreateIndexRequest.java index cffed98fce3aa..5f0931d98ee9d 100644 --- a/client/rest-high-level/src/main/java/org/opensearch/client/indices/CreateIndexRequest.java +++ b/client/rest-high-level/src/main/java/org/opensearch/client/indices/CreateIndexRequest.java @@ -38,7 +38,6 @@ import org.opensearch.action.support.ActiveShardCount; import org.opensearch.client.TimedRequest; import org.opensearch.client.Validatable; -import org.opensearch.common.Strings; import org.opensearch.core.common.bytes.BytesArray; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.common.settings.Settings; @@ -135,7 +134,7 @@ public CreateIndexRequest settings(String source, MediaType mediaType) { * Allows to set the settings using a json builder. */ public CreateIndexRequest settings(XContentBuilder builder) { - settings(Strings.toString(builder), builder.contentType()); + settings(builder.toString(), builder.contentType()); return this; } diff --git a/client/rest-high-level/src/main/java/org/opensearch/client/slm/SnapshotLifecyclePolicy.java b/client/rest-high-level/src/main/java/org/opensearch/client/slm/SnapshotLifecyclePolicy.java index 3566d6aef1ac3..a6f8bfdee4d68 100644 --- a/client/rest-high-level/src/main/java/org/opensearch/client/slm/SnapshotLifecyclePolicy.java +++ b/client/rest-high-level/src/main/java/org/opensearch/client/slm/SnapshotLifecyclePolicy.java @@ -33,9 +33,9 @@ package org.opensearch.client.slm; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.xcontent.XContentType; import org.opensearch.core.ParseField; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ConstructingObjectParser; import org.opensearch.core.xcontent.ToXContentObject; import org.opensearch.core.xcontent.XContentBuilder; diff --git a/client/rest-high-level/src/main/java/org/opensearch/client/slm/SnapshotLifecyclePolicyMetadata.java b/client/rest-high-level/src/main/java/org/opensearch/client/slm/SnapshotLifecyclePolicyMetadata.java index c0b542557159f..0fcb6f7c6a29e 100644 --- a/client/rest-high-level/src/main/java/org/opensearch/client/slm/SnapshotLifecyclePolicyMetadata.java +++ b/client/rest-high-level/src/main/java/org/opensearch/client/slm/SnapshotLifecyclePolicyMetadata.java @@ -33,9 +33,9 @@ package org.opensearch.client.slm; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.xcontent.XContentType; import org.opensearch.core.ParseField; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ConstructingObjectParser; import org.opensearch.core.xcontent.ToXContentObject; import org.opensearch.core.xcontent.XContentBuilder; diff --git a/client/rest-high-level/src/main/java/org/opensearch/client/slm/SnapshotLifecycleStats.java b/client/rest-high-level/src/main/java/org/opensearch/client/slm/SnapshotLifecycleStats.java index 2126ced76b5ae..a2e43325ccd6f 100644 --- a/client/rest-high-level/src/main/java/org/opensearch/client/slm/SnapshotLifecycleStats.java +++ b/client/rest-high-level/src/main/java/org/opensearch/client/slm/SnapshotLifecycleStats.java @@ -32,10 +32,10 @@ package org.opensearch.client.slm; -import org.opensearch.common.Strings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.xcontent.XContentType; import org.opensearch.core.ParseField; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ConstructingObjectParser; import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.ToXContentObject; diff --git a/client/rest-high-level/src/main/java/org/opensearch/client/slm/SnapshotRetentionConfiguration.java b/client/rest-high-level/src/main/java/org/opensearch/client/slm/SnapshotRetentionConfiguration.java index 2a7ab6bb04095..9982c0f2cef7d 100644 --- a/client/rest-high-level/src/main/java/org/opensearch/client/slm/SnapshotRetentionConfiguration.java +++ b/client/rest-high-level/src/main/java/org/opensearch/client/slm/SnapshotRetentionConfiguration.java @@ -33,10 +33,10 @@ package org.opensearch.client.slm; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.xcontent.XContentType; import org.opensearch.core.ParseField; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ConstructingObjectParser; import org.opensearch.core.xcontent.ToXContentObject; import org.opensearch.core.xcontent.XContentBuilder; diff --git a/client/rest-high-level/src/test/java/org/opensearch/client/BulkProcessorIT.java b/client/rest-high-level/src/test/java/org/opensearch/client/BulkProcessorIT.java index dd793fdbb7ff6..2aeb47de150d6 100644 --- a/client/rest-high-level/src/test/java/org/opensearch/client/BulkProcessorIT.java +++ b/client/rest-high-level/src/test/java/org/opensearch/client/BulkProcessorIT.java @@ -42,7 +42,6 @@ import org.opensearch.action.get.MultiGetResponse; import org.opensearch.action.index.IndexRequest; import org.opensearch.action.search.SearchRequest; -import org.opensearch.common.Strings; import org.opensearch.core.common.bytes.BytesArray; import org.opensearch.core.common.unit.ByteSizeUnit; import org.opensearch.core.common.unit.ByteSizeValue; @@ -423,7 +422,7 @@ private static BytesArray bytesBulkRequest(String localIndex, int id) throws IOE XContentBuilder source = jsonBuilder().startObject().field("field", randomRealisticUnicodeOfLengthBetween(1, 30)).endObject(); - String request = Strings.toString(action) + "\n" + Strings.toString(source) + "\n"; + String request = action + "\n" + source + "\n"; return new BytesArray(request); } diff --git a/client/rest-high-level/src/test/java/org/opensearch/client/SearchIT.java b/client/rest-high-level/src/test/java/org/opensearch/client/SearchIT.java index fbfab016b1ead..233ea0ca5f48d 100644 --- a/client/rest-high-level/src/test/java/org/opensearch/client/SearchIT.java +++ b/client/rest-high-level/src/test/java/org/opensearch/client/SearchIT.java @@ -52,7 +52,6 @@ import org.opensearch.action.search.SearchScrollRequest; import org.opensearch.client.core.CountRequest; import org.opensearch.client.core.CountResponse; -import org.opensearch.common.Strings; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.common.unit.TimeValue; import org.opensearch.core.xcontent.XContentBuilder; @@ -769,7 +768,7 @@ public void testSearchScroll() throws Exception { for (int i = 0; i < 100; i++) { XContentBuilder builder = jsonBuilder().startObject().field("field", i).endObject(); Request doc = new Request(HttpPut.METHOD_NAME, "/test/_doc/" + Integer.toString(i)); - doc.setJsonEntity(Strings.toString(builder)); + doc.setJsonEntity(builder.toString()); client().performRequest(doc); } client().performRequest(new Request(HttpPost.METHOD_NAME, "/test/_refresh")); @@ -837,7 +836,7 @@ public void testSearchWithPit() throws Exception { for (int i = 0; i < 100; i++) { XContentBuilder builder = jsonBuilder().startObject().field("field", i).endObject(); Request doc = new Request(HttpPut.METHOD_NAME, "/test/_doc/" + Integer.toString(i)); - doc.setJsonEntity(Strings.toString(builder)); + doc.setJsonEntity(builder.toString()); client().performRequest(doc); } client().performRequest(new Request(HttpPost.METHOD_NAME, "/test/_refresh")); diff --git a/client/rest-high-level/src/test/java/org/opensearch/client/documentation/CRUDDocumentationIT.java b/client/rest-high-level/src/test/java/org/opensearch/client/documentation/CRUDDocumentationIT.java index 178d9296bd242..010b16a3f5720 100644 --- a/client/rest-high-level/src/test/java/org/opensearch/client/documentation/CRUDDocumentationIT.java +++ b/client/rest-high-level/src/test/java/org/opensearch/client/documentation/CRUDDocumentationIT.java @@ -74,7 +74,7 @@ import org.opensearch.client.core.TermVectorsResponse; import org.opensearch.client.indices.CreateIndexRequest; import org.opensearch.client.indices.CreateIndexResponse; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.bytes.BytesArray; import org.opensearch.common.settings.Settings; import org.opensearch.core.common.unit.ByteSizeUnit; @@ -298,15 +298,14 @@ public void testUpdate() throws Exception { Request request = new Request("POST", "/_scripts/increment-field"); request.setJsonEntity( - Strings.toString( - JsonXContent.contentBuilder() - .startObject() - .startObject("script") - .field("lang", "painless") - .field("source", "ctx._source.field += params.count") - .endObject() - .endObject() - ) + JsonXContent.contentBuilder() + .startObject() + .startObject("script") + .field("lang", "painless") + .field("source", "ctx._source.field += params.count") + .endObject() + .endObject() + .toString() ); Response response = client().performRequest(request); assertEquals(RestStatus.OK.getStatus(), response.getStatusLine().getStatusCode()); diff --git a/distribution/archives/integ-test-zip/src/test/java/org/opensearch/test/rest/NodeRestUsageIT.java b/distribution/archives/integ-test-zip/src/test/java/org/opensearch/test/rest/NodeRestUsageIT.java index d397a1c967ad0..0f21984e5d8b9 100644 --- a/distribution/archives/integ-test-zip/src/test/java/org/opensearch/test/rest/NodeRestUsageIT.java +++ b/distribution/archives/integ-test-zip/src/test/java/org/opensearch/test/rest/NodeRestUsageIT.java @@ -35,8 +35,8 @@ import org.opensearch.client.Request; import org.opensearch.client.Response; import org.opensearch.client.ResponseException; -import org.opensearch.common.Strings; import org.opensearch.common.xcontent.XContentType; +import org.opensearch.core.common.Strings; import org.opensearch.search.aggregations.AggregationBuilders; import org.opensearch.search.builder.SearchSourceBuilder; diff --git a/libs/core/src/main/java/org/opensearch/core/common/Strings.java b/libs/core/src/main/java/org/opensearch/core/common/Strings.java index f0ca12a307313..6227716af9cc9 100644 --- a/libs/core/src/main/java/org/opensearch/core/common/Strings.java +++ b/libs/core/src/main/java/org/opensearch/core/common/Strings.java @@ -9,11 +9,17 @@ package org.opensearch.core.common; import org.apache.lucene.util.BytesRefBuilder; +import org.opensearch.ExceptionsHelper; +import org.opensearch.OpenSearchException; import org.opensearch.common.Nullable; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.common.util.CollectionUtils; +import org.opensearch.core.xcontent.MediaType; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.XContentBuilder; import java.io.BufferedReader; +import java.io.IOException; import java.io.StringReader; import java.util.ArrayList; import java.util.Arrays; @@ -680,6 +686,77 @@ public static boolean isAllOrWildcard(String data) { return "_all".equals(data) || "*".equals(data); } + /** + * Return a {@link String} that is the json representation of the provided {@link ToXContent}. + * Wraps the output into an anonymous object if needed. The content is not pretty-printed + * nor human readable. + */ + public static String toString(MediaType mediaType, ToXContent toXContent) { + return toString(mediaType, toXContent, false, false); + } + + /** + * Return a {@link String} that is the json representation of the provided {@link ToXContent}. + * Wraps the output into an anonymous object if needed. + * Allows to configure the params. + * The content is not pretty-printed nor human readable. + */ + public static String toString(MediaType mediaType, ToXContent toXContent, ToXContent.Params params) { + return toString(mediaType, toXContent, params, false, false); + } + + /** + * Return a {@link String} that is the json representation of the provided {@link ToXContent}. + * Wraps the output into an anonymous object if needed. Allows to control whether the outputted + * json needs to be pretty printed and human readable. + * + */ + public static String toString(MediaType mediaType, ToXContent toXContent, boolean pretty, boolean human) { + return toString(mediaType, toXContent, ToXContent.EMPTY_PARAMS, pretty, human); + } + + /** + * Return a {@link String} that is the json representation of the provided {@link ToXContent}. + * Wraps the output into an anonymous object if needed. + * Allows to configure the params. + * Allows to control whether the outputted json needs to be pretty printed and human readable. + */ + private static String toString(MediaType mediaType, ToXContent toXContent, ToXContent.Params params, boolean pretty, boolean human) { + try { + XContentBuilder builder = createBuilder(mediaType, pretty, human); + if (toXContent.isFragment()) { + builder.startObject(); + } + toXContent.toXContent(builder, params); + if (toXContent.isFragment()) { + builder.endObject(); + } + return builder.toString(); + } catch (IOException e) { + try { + XContentBuilder builder = createBuilder(mediaType, pretty, human); + builder.startObject(); + builder.field("error", "error building toString out of XContent: " + e.getMessage()); + builder.field("stack_trace", ExceptionsHelper.stackTrace(e)); + builder.endObject(); + return builder.toString(); + } catch (IOException e2) { + throw new OpenSearchException("cannot generate error message for deserialization", e); + } + } + } + + private static XContentBuilder createBuilder(MediaType mediaType, boolean pretty, boolean human) throws IOException { + XContentBuilder builder = XContentBuilder.builder(mediaType.xContent()); + if (pretty) { + builder.prettyPrint(); + } + if (human) { + builder.humanReadable(true); + } + return builder; + } + /** * Truncates string to a length less than length. Backtracks to throw out * high surrogates. diff --git a/libs/core/src/main/java/org/opensearch/core/xcontent/XContentBuilder.java b/libs/core/src/main/java/org/opensearch/core/xcontent/XContentBuilder.java index 061837d27ed0a..dfd1449ef0e0b 100644 --- a/libs/core/src/main/java/org/opensearch/core/xcontent/XContentBuilder.java +++ b/libs/core/src/main/java/org/opensearch/core/xcontent/XContentBuilder.java @@ -32,6 +32,8 @@ package org.opensearch.core.xcontent; +import org.opensearch.core.common.bytes.BytesReference; + import java.io.ByteArrayOutputStream; import java.io.Closeable; import java.io.Flushable; @@ -151,6 +153,14 @@ public static XContentBuilder builder(XContent xContent, Set includes, S DATE_TRANSFORMERS = Collections.unmodifiableMap(dateTransformers); } + /** + * Returns a string representation of the builder (only applicable for text based xcontent). + */ + @Override + public String toString() { + return BytesReference.bytes(this).utf8ToString(); + } + /** * The writer interface for the serializable content builder * diff --git a/libs/core/src/test/java/org/opensearch/core/action/support/DefaultShardOperationFailedExceptionTests.java b/libs/core/src/test/java/org/opensearch/core/action/support/DefaultShardOperationFailedExceptionTests.java index d037b062e95d0..9801e9cbcdb44 100644 --- a/libs/core/src/test/java/org/opensearch/core/action/support/DefaultShardOperationFailedExceptionTests.java +++ b/libs/core/src/test/java/org/opensearch/core/action/support/DefaultShardOperationFailedExceptionTests.java @@ -39,7 +39,7 @@ import org.apache.lucene.store.LockObtainFailedException; import org.opensearch.OpenSearchException; import org.opensearch.action.support.broadcast.BroadcastShardOperationFailedException; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.core.common.io.stream.StreamInput; diff --git a/libs/core/src/test/java/org/opensearch/core/common/StringsTests.java b/libs/core/src/test/java/org/opensearch/core/common/StringsTests.java index ca0bd788c1991..b79bb6fc89f9e 100644 --- a/libs/core/src/test/java/org/opensearch/core/common/StringsTests.java +++ b/libs/core/src/test/java/org/opensearch/core/common/StringsTests.java @@ -9,8 +9,14 @@ package org.opensearch.core.common; import org.opensearch.common.util.set.Sets; +import org.opensearch.core.xcontent.MediaTypeRegistry; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.ToXContentObject; import org.opensearch.test.OpenSearchTestCase; +import java.util.Collections; + +import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.is; /** tests for Strings utility class */ @@ -63,4 +69,49 @@ public void testSplitStringToSet() { assertEquals(Strings.tokenizeByCommaToSet(" aa "), Sets.newHashSet("aa")); assertEquals(Strings.tokenizeByCommaToSet(" "), Sets.newHashSet()); } + + public void testToStringToXContent() { + final ToXContent toXContent; + final boolean error; + if (randomBoolean()) { + if (randomBoolean()) { + error = false; + toXContent = (builder, params) -> builder.field("ok", "here").field("catastrophe", ""); + } else { + error = true; + toXContent = (builder, params) -> builder.startObject().field("ok", "here").field("catastrophe", "").endObject(); + } + } else { + if (randomBoolean()) { + error = false; + toXContent = (ToXContentObject) (builder, params) -> builder.startObject() + .field("ok", "here") + .field("catastrophe", "") + .endObject(); + } else { + error = true; + toXContent = (ToXContentObject) (builder, params) -> builder.field("ok", "here").field("catastrophe", ""); + } + } + + String toString = Strings.toString(MediaTypeRegistry.JSON, toXContent); + if (error) { + assertThat(toString, containsString("\"error\":\"error building toString out of XContent:")); + assertThat(toString, containsString("\"stack_trace\":")); + } else { + assertThat(toString, containsString("\"ok\":\"here\"")); + assertThat(toString, containsString("\"catastrophe\":\"\"")); + } + } + + public void testToStringToXContentWithOrWithoutParams() { + ToXContent toXContent = (builder, params) -> builder.field("color_from_param", params.param("color", "red")); + // Rely on the default value of "color" param when params are not passed + assertThat(Strings.toString(MediaTypeRegistry.JSON, toXContent), containsString("\"color_from_param\":\"red\"")); + // Pass "color" param explicitly + assertThat( + Strings.toString(MediaTypeRegistry.JSON, toXContent, new ToXContent.MapParams(Collections.singletonMap("color", "blue"))), + containsString("\"color_from_param\":\"blue\"") + ); + } } diff --git a/libs/x-content/src/test/java/org/opensearch/common/xcontent/ObjectParserTests.java b/libs/x-content/src/test/java/org/opensearch/common/xcontent/ObjectParserTests.java index cd59bf59fe15d..6e7de4aa6bfe5 100644 --- a/libs/x-content/src/test/java/org/opensearch/common/xcontent/ObjectParserTests.java +++ b/libs/x-content/src/test/java/org/opensearch/common/xcontent/ObjectParserTests.java @@ -33,7 +33,6 @@ import org.opensearch.common.CheckedFunction; import org.opensearch.core.ParseField; -import org.opensearch.common.Strings; import org.opensearch.core.xcontent.XContentParserUtils; import org.opensearch.core.xcontent.ObjectParser; import org.opensearch.core.xcontent.ObjectParser.NamedObjectParser; @@ -449,7 +448,7 @@ public void testAllVariants() throws IOException { } builder.field("string_or_null", nullValue ? null : "5"); builder.endObject(); - XContentParser parser = createParser(JsonXContent.jsonXContent, Strings.toString(builder)); + XContentParser parser = createParser(JsonXContent.jsonXContent, builder.toString()); class TestStruct { int int_field; int nullableIntField; diff --git a/libs/x-content/src/test/java/org/opensearch/common/xcontent/SimpleStruct.java b/libs/x-content/src/test/java/org/opensearch/common/xcontent/SimpleStruct.java index a4aca80918284..1d2a66ea1f78f 100644 --- a/libs/x-content/src/test/java/org/opensearch/common/xcontent/SimpleStruct.java +++ b/libs/x-content/src/test/java/org/opensearch/common/xcontent/SimpleStruct.java @@ -33,7 +33,7 @@ package org.opensearch.common.xcontent; import org.opensearch.core.ParseField; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ConstructingObjectParser; import org.opensearch.core.xcontent.ToXContentObject; import org.opensearch.core.xcontent.XContentBuilder; diff --git a/libs/x-content/src/test/java/org/opensearch/common/xcontent/XContentParserTests.java b/libs/x-content/src/test/java/org/opensearch/common/xcontent/XContentParserTests.java index 3552514af0aa8..fab61753eb739 100644 --- a/libs/x-content/src/test/java/org/opensearch/common/xcontent/XContentParserTests.java +++ b/libs/x-content/src/test/java/org/opensearch/common/xcontent/XContentParserTests.java @@ -37,7 +37,6 @@ import com.fasterxml.jackson.dataformat.yaml.JacksonYAMLParseException; import org.opensearch.common.CheckedSupplier; -import org.opensearch.common.Strings; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.common.xcontent.cbor.CborXContent; import org.opensearch.common.xcontent.json.JsonXContent; @@ -366,7 +365,7 @@ public void testReadBooleans() throws IOException { public void testEmptyList() throws IOException { XContentBuilder builder = XContentFactory.jsonBuilder().startObject().startArray("some_array").endArray().endObject(); - try (XContentParser parser = createParser(JsonXContent.jsonXContent, Strings.toString(builder))) { + try (XContentParser parser = createParser(JsonXContent.jsonXContent, builder.toString())) { assertEquals(XContentParser.Token.START_OBJECT, parser.nextToken()); assertEquals(XContentParser.Token.FIELD_NAME, parser.nextToken()); assertEquals("some_array", parser.currentName()); @@ -388,7 +387,7 @@ public void testSimpleList() throws IOException { .endArray() .endObject(); - try (XContentParser parser = createParser(JsonXContent.jsonXContent, Strings.toString(builder))) { + try (XContentParser parser = createParser(JsonXContent.jsonXContent, builder.toString())) { assertEquals(XContentParser.Token.START_OBJECT, parser.nextToken()); assertEquals(XContentParser.Token.FIELD_NAME, parser.nextToken()); assertEquals("some_array", parser.currentName()); @@ -416,7 +415,7 @@ public void testNestedList() throws IOException { .endArray() .endObject(); - try (XContentParser parser = createParser(JsonXContent.jsonXContent, Strings.toString(builder))) { + try (XContentParser parser = createParser(JsonXContent.jsonXContent, builder.toString())) { assertEquals(XContentParser.Token.START_OBJECT, parser.nextToken()); assertEquals(XContentParser.Token.FIELD_NAME, parser.nextToken()); assertEquals("some_array", parser.currentName()); @@ -440,7 +439,7 @@ public void testNestedMapInList() throws IOException { .endArray() .endObject(); - try (XContentParser parser = createParser(JsonXContent.jsonXContent, Strings.toString(builder))) { + try (XContentParser parser = createParser(JsonXContent.jsonXContent, builder.toString())) { assertEquals(XContentParser.Token.START_OBJECT, parser.nextToken()); assertEquals(XContentParser.Token.FIELD_NAME, parser.nextToken()); assertEquals("some_array", parser.currentName()); @@ -516,7 +515,7 @@ public void testSubParserObject() throws IOException { XContentBuilder builder = XContentFactory.jsonBuilder(); int numberOfTokens; numberOfTokens = generateRandomObjectForMarking(builder); - String content = Strings.toString(builder); + String content = builder.toString(); try (XContentParser parser = createParser(JsonXContent.jsonXContent, content)) { assertEquals(XContentParser.Token.START_OBJECT, parser.nextToken()); @@ -564,7 +563,7 @@ public void testSubParserArray() throws IOException { builder.endArray(); builder.endObject(); - String content = Strings.toString(builder); + String content = builder.toString(); try (XContentParser parser = createParser(JsonXContent.jsonXContent, content)) { assertEquals(XContentParser.Token.START_OBJECT, parser.nextToken()); @@ -597,7 +596,7 @@ public void testSubParserArray() throws IOException { public void testCreateSubParserAtAWrongPlace() throws IOException { XContentBuilder builder = XContentFactory.jsonBuilder(); generateRandomObjectForMarking(builder); - String content = Strings.toString(builder); + String content = builder.toString(); try (XContentParser parser = createParser(JsonXContent.jsonXContent, content)) { assertEquals(XContentParser.Token.START_OBJECT, parser.nextToken()); @@ -611,7 +610,7 @@ public void testCreateSubParserAtAWrongPlace() throws IOException { public void testCreateRootSubParser() throws IOException { XContentBuilder builder = XContentFactory.jsonBuilder(); int numberOfTokens = generateRandomObjectForMarking(builder); - String content = Strings.toString(builder); + String content = builder.toString(); try (XContentParser parser = createParser(JsonXContent.jsonXContent, content)) { assertEquals(XContentParser.Token.START_OBJECT, parser.nextToken()); diff --git a/modules/geo/src/internalClusterTest/java/org/opensearch/geo/search/aggregations/metrics/AbstractGeoAggregatorModulePluginTestCase.java b/modules/geo/src/internalClusterTest/java/org/opensearch/geo/search/aggregations/metrics/AbstractGeoAggregatorModulePluginTestCase.java index f7c9747e1a163..5df5912daf461 100644 --- a/modules/geo/src/internalClusterTest/java/org/opensearch/geo/search/aggregations/metrics/AbstractGeoAggregatorModulePluginTestCase.java +++ b/modules/geo/src/internalClusterTest/java/org/opensearch/geo/search/aggregations/metrics/AbstractGeoAggregatorModulePluginTestCase.java @@ -10,7 +10,6 @@ import org.opensearch.action.index.IndexRequestBuilder; import org.opensearch.action.search.SearchResponse; -import org.opensearch.common.Strings; import org.opensearch.common.document.DocumentField; import org.opensearch.common.geo.GeoPoint; import org.opensearch.common.settings.Settings; @@ -256,7 +255,7 @@ public void setupSuiteScopeCluster() throws Exception { long totalHits = response.getHits().getTotalHits().value; XContentBuilder builder = XContentFactory.jsonBuilder(); response.toXContent(builder, ToXContent.EMPTY_PARAMS); - logger.info("Full high_card_idx Response Content:\n{ {} }", Strings.toString(builder)); + logger.info("Full high_card_idx Response Content:\n{ {} }", builder.toString()); for (int i = 0; i < totalHits; i++) { SearchHit searchHit = response.getHits().getAt(i); assertThat("Hit " + i + " with id: " + searchHit.getId(), searchHit.getIndex(), equalTo("high_card_idx")); diff --git a/modules/lang-mustache/src/internalClusterTest/java/org/opensearch/script/mustache/MultiSearchTemplateIT.java b/modules/lang-mustache/src/internalClusterTest/java/org/opensearch/script/mustache/MultiSearchTemplateIT.java index fbb8ebdec384c..bb11e493ba3d1 100644 --- a/modules/lang-mustache/src/internalClusterTest/java/org/opensearch/script/mustache/MultiSearchTemplateIT.java +++ b/modules/lang-mustache/src/internalClusterTest/java/org/opensearch/script/mustache/MultiSearchTemplateIT.java @@ -34,7 +34,6 @@ import org.opensearch.action.index.IndexRequestBuilder; import org.opensearch.action.search.SearchRequest; -import org.opensearch.common.Strings; import org.opensearch.index.IndexNotFoundException; import org.opensearch.plugins.Plugin; import org.opensearch.script.ScriptType; @@ -72,15 +71,14 @@ public void testBasic() throws Exception { } indexRandom(true, indexRequestBuilders); - final String template = Strings.toString( - jsonBuilder().startObject() - .startObject("query") - .startObject("{{query_type}}") - .field("{{field_name}}", "{{field_value}}") - .endObject() - .endObject() - .endObject() - ); + final String template = jsonBuilder().startObject() + .startObject("query") + .startObject("{{query_type}}") + .field("{{field_name}}", "{{field_value}}") + .endObject() + .endObject() + .endObject() + .toString(); MultiSearchTemplateRequest multiRequest = new MultiSearchTemplateRequest(); diff --git a/modules/lang-mustache/src/main/java/org/opensearch/script/mustache/CustomMustacheFactory.java b/modules/lang-mustache/src/main/java/org/opensearch/script/mustache/CustomMustacheFactory.java index a4be60d2b6900..0cf1ed525fbfe 100644 --- a/modules/lang-mustache/src/main/java/org/opensearch/script/mustache/CustomMustacheFactory.java +++ b/modules/lang-mustache/src/main/java/org/opensearch/script/mustache/CustomMustacheFactory.java @@ -44,7 +44,6 @@ import com.github.mustachejava.codes.IterableCode; import com.github.mustachejava.codes.WriteCode; -import org.opensearch.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.common.xcontent.XContentType; @@ -228,7 +227,7 @@ protected Function createFunction(Object resolved) { // Do not handle as JSON return oh.stringify(resolved); } - return Strings.toString(builder); + return builder.toString(); } catch (IOException e) { throw new MustacheException("Failed to convert object to JSON", e); } diff --git a/modules/lang-mustache/src/main/java/org/opensearch/script/mustache/MultiSearchTemplateResponse.java b/modules/lang-mustache/src/main/java/org/opensearch/script/mustache/MultiSearchTemplateResponse.java index f31e5be078a28..49f5d4194d446 100644 --- a/modules/lang-mustache/src/main/java/org/opensearch/script/mustache/MultiSearchTemplateResponse.java +++ b/modules/lang-mustache/src/main/java/org/opensearch/script/mustache/MultiSearchTemplateResponse.java @@ -36,7 +36,7 @@ import org.opensearch.action.ActionResponse; import org.opensearch.action.search.MultiSearchResponse; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; diff --git a/modules/lang-mustache/src/main/java/org/opensearch/script/mustache/SearchTemplateRequest.java b/modules/lang-mustache/src/main/java/org/opensearch/script/mustache/SearchTemplateRequest.java index c963ea7ba7da9..166778bd02cee 100644 --- a/modules/lang-mustache/src/main/java/org/opensearch/script/mustache/SearchTemplateRequest.java +++ b/modules/lang-mustache/src/main/java/org/opensearch/script/mustache/SearchTemplateRequest.java @@ -38,13 +38,12 @@ import org.opensearch.action.search.SearchRequest; import org.opensearch.core.ParseField; import org.opensearch.core.common.ParsingException; -import org.opensearch.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; +import org.opensearch.core.xcontent.MediaTypeRegistry; import org.opensearch.core.xcontent.ObjectParser; import org.opensearch.core.xcontent.ToXContentObject; import org.opensearch.core.xcontent.XContentBuilder; -import org.opensearch.common.xcontent.XContentFactory; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.script.ScriptType; @@ -207,8 +206,8 @@ public ActionRequestValidationException validate() { request.setScriptType(ScriptType.INLINE); if (parser.currentToken() == XContentParser.Token.START_OBJECT) { // convert the template to json which is the only supported XContentType (see CustomMustacheFactory#createEncoder) - try (XContentBuilder builder = XContentFactory.jsonBuilder()) { - request.setScript(Strings.toString(builder.copyCurrentStructure(parser))); + try (XContentBuilder builder = MediaTypeRegistry.JSON.contentBuilder()) { + request.setScript(builder.copyCurrentStructure(parser).toString()); } catch (IOException e) { throw new ParsingException(parser.getTokenLocation(), "Could not parse inline template", e); } diff --git a/modules/mapper-extras/src/test/java/org/opensearch/index/mapper/RankFeatureFieldMapperTests.java b/modules/mapper-extras/src/test/java/org/opensearch/index/mapper/RankFeatureFieldMapperTests.java index 6412059075e5c..fee9471444c19 100644 --- a/modules/mapper-extras/src/test/java/org/opensearch/index/mapper/RankFeatureFieldMapperTests.java +++ b/modules/mapper-extras/src/test/java/org/opensearch/index/mapper/RankFeatureFieldMapperTests.java @@ -38,7 +38,6 @@ import org.apache.lucene.index.IndexableField; import org.apache.lucene.search.Query; import org.apache.lucene.search.TermQuery; -import org.opensearch.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.plugins.Plugin; @@ -91,7 +90,7 @@ protected void minimalMapping(XContentBuilder b) throws IOException { public void testDefaults() throws Exception { DocumentMapper mapper = createDocumentMapper(fieldMapping(this::minimalMapping)); - assertEquals(Strings.toString(fieldMapping(this::minimalMapping)), mapper.mappingSource().toString()); + assertEquals(fieldMapping(this::minimalMapping).toString(), mapper.mappingSource().toString()); ParsedDocument doc1 = mapper.parse(source(b -> b.field("field", 10))); IndexableField[] fields = doc1.rootDoc().getFields("_feature"); diff --git a/modules/mapper-extras/src/test/java/org/opensearch/index/mapper/RankFeatureMetaFieldMapperTests.java b/modules/mapper-extras/src/test/java/org/opensearch/index/mapper/RankFeatureMetaFieldMapperTests.java index 63b1b4760b6fe..d35368350592c 100644 --- a/modules/mapper-extras/src/test/java/org/opensearch/index/mapper/RankFeatureMetaFieldMapperTests.java +++ b/modules/mapper-extras/src/test/java/org/opensearch/index/mapper/RankFeatureMetaFieldMapperTests.java @@ -32,11 +32,10 @@ package org.opensearch.index.mapper; -import org.opensearch.common.Strings; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.common.compress.CompressedXContent; -import org.opensearch.common.xcontent.XContentFactory; import org.opensearch.common.xcontent.XContentType; +import org.opensearch.core.xcontent.MediaTypeRegistry; import org.opensearch.index.IndexService; import org.opensearch.plugins.Plugin; import org.opensearch.test.OpenSearchSingleNodeTestCase; @@ -61,18 +60,17 @@ protected Collection> getPlugins() { } public void testBasics() throws Exception { - String mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("type") - .startObject("properties") - .startObject("field") - .field("type", "rank_feature") - .endObject() - .endObject() - .endObject() - .endObject() - ); + String mapping = MediaTypeRegistry.JSON.contentBuilder() + .startObject() + .startObject("type") + .startObject("properties") + .startObject("field") + .field("type", "rank_feature") + .endObject() + .endObject() + .endObject() + .endObject() + .toString(); DocumentMapper mapper = parser.parse("type", new CompressedXContent(mapping)); @@ -85,10 +83,12 @@ public void testBasics() throws Exception { * and parsing of a document fails if the document contains these meta-fields. */ public void testDocumentParsingFailsOnMetaField() throws Exception { - String mapping = Strings.toString(XContentFactory.jsonBuilder().startObject().startObject("_doc").endObject().endObject()); + String mapping = MediaTypeRegistry.JSON.contentBuilder().startObject().startObject("_doc").endObject().endObject().toString(); DocumentMapper mapper = parser.parse("_doc", new CompressedXContent(mapping)); String rfMetaField = RankFeatureMetaFieldMapper.CONTENT_TYPE; - BytesReference bytes = BytesReference.bytes(XContentFactory.jsonBuilder().startObject().field(rfMetaField, 0).endObject()); + BytesReference bytes = BytesReference.bytes( + MediaTypeRegistry.JSON.contentBuilder().startObject().field(rfMetaField, 0).endObject() + ); MapperParsingException e = expectThrows( MapperParsingException.class, () -> mapper.parse(new SourceToParse("test", "1", bytes, XContentType.JSON)) diff --git a/modules/mapper-extras/src/test/java/org/opensearch/index/mapper/RankFeaturesFieldMapperTests.java b/modules/mapper-extras/src/test/java/org/opensearch/index/mapper/RankFeaturesFieldMapperTests.java index 6c844bae73da4..b95572835e612 100644 --- a/modules/mapper-extras/src/test/java/org/opensearch/index/mapper/RankFeaturesFieldMapperTests.java +++ b/modules/mapper-extras/src/test/java/org/opensearch/index/mapper/RankFeaturesFieldMapperTests.java @@ -34,7 +34,6 @@ import org.apache.lucene.document.FeatureField; import org.apache.lucene.index.IndexableField; -import org.opensearch.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.plugins.Plugin; import org.hamcrest.Matchers; @@ -79,7 +78,7 @@ protected boolean supportsMeta() { public void testDefaults() throws Exception { DocumentMapper mapper = createDocumentMapper(fieldMapping(this::minimalMapping)); - assertEquals(Strings.toString(fieldMapping(this::minimalMapping)), mapper.mappingSource().toString()); + assertEquals(fieldMapping(this::minimalMapping).toString(), mapper.mappingSource().toString()); ParsedDocument doc1 = mapper.parse(source(this::writeField)); diff --git a/modules/mapper-extras/src/test/java/org/opensearch/index/mapper/ScaledFloatFieldMapperTests.java b/modules/mapper-extras/src/test/java/org/opensearch/index/mapper/ScaledFloatFieldMapperTests.java index 390f10c0684bd..4de11d7f64e8e 100644 --- a/modules/mapper-extras/src/test/java/org/opensearch/index/mapper/ScaledFloatFieldMapperTests.java +++ b/modules/mapper-extras/src/test/java/org/opensearch/index/mapper/ScaledFloatFieldMapperTests.java @@ -34,7 +34,6 @@ import org.apache.lucene.index.DocValuesType; import org.apache.lucene.index.IndexableField; -import org.opensearch.common.Strings; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.common.xcontent.XContentFactory; @@ -95,7 +94,7 @@ public void testExistsQueryDocValuesDisabled() throws IOException { public void testDefaults() throws Exception { XContentBuilder mapping = fieldMapping(b -> b.field("type", "scaled_float").field("scaling_factor", 10.0)); DocumentMapper mapper = createDocumentMapper(mapping); - assertEquals(Strings.toString(mapping), mapper.mappingSource().toString()); + assertEquals(mapping.toString(), mapper.mappingSource().toString()); ParsedDocument doc = mapper.parse(source(b -> b.field("field", 123))); IndexableField[] fields = doc.rootDoc().getFields("field"); diff --git a/modules/mapper-extras/src/test/java/org/opensearch/index/mapper/SearchAsYouTypeFieldMapperTests.java b/modules/mapper-extras/src/test/java/org/opensearch/index/mapper/SearchAsYouTypeFieldMapperTests.java index 5e67aaa2ed246..551bd38b65f59 100644 --- a/modules/mapper-extras/src/test/java/org/opensearch/index/mapper/SearchAsYouTypeFieldMapperTests.java +++ b/modules/mapper-extras/src/test/java/org/opensearch/index/mapper/SearchAsYouTypeFieldMapperTests.java @@ -50,9 +50,9 @@ import org.apache.lucene.search.Query; import org.apache.lucene.search.SynonymQuery; import org.apache.lucene.search.TermQuery; -import org.opensearch.common.Strings; import org.opensearch.common.lucene.search.MultiPhrasePrefixQuery; import org.opensearch.common.xcontent.XContentType; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.index.IndexSettings; import org.opensearch.index.analysis.AnalyzerScope; diff --git a/modules/mapper-extras/src/test/java/org/opensearch/index/query/RankFeatureQueryBuilderTests.java b/modules/mapper-extras/src/test/java/org/opensearch/index/query/RankFeatureQueryBuilderTests.java index f57aac8a244b7..a8d672c025af0 100644 --- a/modules/mapper-extras/src/test/java/org/opensearch/index/query/RankFeatureQueryBuilderTests.java +++ b/modules/mapper-extras/src/test/java/org/opensearch/index/query/RankFeatureQueryBuilderTests.java @@ -36,7 +36,6 @@ import org.apache.lucene.search.MatchNoDocsQuery; import org.apache.lucene.search.Query; import org.opensearch.action.admin.indices.mapping.put.PutMappingRequest; -import org.opensearch.common.Strings; import org.opensearch.common.compress.CompressedXContent; import org.opensearch.index.mapper.MapperExtrasModulePlugin; import org.opensearch.index.mapper.MapperService; @@ -61,16 +60,14 @@ protected void initializeAdditionalMappings(MapperService mapperService) throws mapperService.merge( "_doc", new CompressedXContent( - Strings.toString( - PutMappingRequest.simpleMapping( - "my_feature_field", - "type=rank_feature", - "my_negative_feature_field", - "type=rank_feature,positive_score_impact=false", - "my_feature_vector_field", - "type=rank_features" - ) - ) + PutMappingRequest.simpleMapping( + "my_feature_field", + "type=rank_feature", + "my_negative_feature_field", + "type=rank_feature,positive_score_impact=false", + "my_feature_vector_field", + "type=rank_features" + ).toString() ), MapperService.MergeReason.MAPPING_UPDATE ); diff --git a/modules/parent-join/src/internalClusterTest/java/org/opensearch/join/query/ParentChildTestCase.java b/modules/parent-join/src/internalClusterTest/java/org/opensearch/join/query/ParentChildTestCase.java index 34a6af6ee3639..9b7ad4425bd5e 100644 --- a/modules/parent-join/src/internalClusterTest/java/org/opensearch/join/query/ParentChildTestCase.java +++ b/modules/parent-join/src/internalClusterTest/java/org/opensearch/join/query/ParentChildTestCase.java @@ -32,7 +32,6 @@ package org.opensearch.join.query; import org.opensearch.action.index.IndexRequestBuilder; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.common.xcontent.XContentHelper; @@ -85,7 +84,7 @@ protected IndexRequestBuilder createIndexRequest(String index, String type, Stri protected IndexRequestBuilder createIndexRequest(String index, String type, String id, String parentId, XContentBuilder builder) throws IOException { - Map source = XContentHelper.convertToMap(JsonXContent.jsonXContent, Strings.toString(builder), false); + Map source = XContentHelper.convertToMap(JsonXContent.jsonXContent, builder.toString(), false); return createIndexRequest(index, type, id, parentId, source); } diff --git a/modules/parent-join/src/test/java/org/opensearch/join/mapper/ParentJoinFieldMapperTests.java b/modules/parent-join/src/test/java/org/opensearch/join/mapper/ParentJoinFieldMapperTests.java index cd8f18b679750..213ba43ee34cd 100644 --- a/modules/parent-join/src/test/java/org/opensearch/join/mapper/ParentJoinFieldMapperTests.java +++ b/modules/parent-join/src/test/java/org/opensearch/join/mapper/ParentJoinFieldMapperTests.java @@ -32,11 +32,10 @@ package org.opensearch.join.mapper; -import org.opensearch.common.Strings; -import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.common.compress.CompressedXContent; import org.opensearch.common.xcontent.XContentFactory; import org.opensearch.common.xcontent.XContentType; +import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.index.IndexService; import org.opensearch.index.mapper.DocumentMapper; import org.opensearch.index.mapper.MapperException; @@ -60,19 +59,18 @@ protected Collection> getPlugins() { } public void testSingleLevel() throws Exception { - String mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject("join_field") - .field("type", "join") - .startObject("relations") - .field("parent", "child") - .endObject() - .endObject() - .endObject() - .endObject() - ); + String mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject("join_field") + .field("type", "join") + .startObject("relations") + .field("parent", "child") + .endObject() + .endObject() + .endObject() + .endObject() + .toString(); IndexService service = createIndex("test"); DocumentMapper docMapper = service.mapperService() .merge("type", new CompressedXContent(mapping), MapperService.MergeReason.MAPPING_UPDATE); @@ -133,19 +131,18 @@ public void testSingleLevel() throws Exception { } public void testParentIdSpecifiedAsNumber() throws Exception { - String mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject("join_field") - .field("type", "join") - .startObject("relations") - .field("parent", "child") - .endObject() - .endObject() - .endObject() - .endObject() - ); + String mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject("join_field") + .field("type", "join") + .startObject("relations") + .field("parent", "child") + .endObject() + .endObject() + .endObject() + .endObject() + .toString(); IndexService service = createIndex("test"); DocumentMapper docMapper = service.mapperService() .merge("type", new CompressedXContent(mapping), MapperService.MergeReason.MAPPING_UPDATE); @@ -190,20 +187,19 @@ public void testParentIdSpecifiedAsNumber() throws Exception { } public void testMultipleLevels() throws Exception { - String mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject("join_field") - .field("type", "join") - .startObject("relations") - .field("parent", "child") - .field("child", "grand_child") - .endObject() - .endObject() - .endObject() - .endObject() - ); + String mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject("join_field") + .field("type", "join") + .startObject("relations") + .field("parent", "child") + .field("child", "grand_child") + .endObject() + .endObject() + .endObject() + .endObject() + .toString(); IndexService service = createIndex("test"); DocumentMapper docMapper = service.mapperService() .merge("type", new CompressedXContent(mapping), MapperService.MergeReason.MAPPING_UPDATE); @@ -323,39 +319,37 @@ public void testMultipleLevels() throws Exception { } public void testUpdateRelations() throws Exception { - String mapping = Strings.toString( - XContentFactory.jsonBuilder() + String mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject("join_field") + .field("type", "join") + .startObject("relations") + .field("parent", "child") + .array("child", "grand_child1", "grand_child2") + .endObject() + .endObject() + .endObject() + .endObject() + .toString(); + IndexService indexService = createIndex("test"); + DocumentMapper docMapper = indexService.mapperService() + .merge("type", new CompressedXContent(mapping), MapperService.MergeReason.MAPPING_UPDATE); + assertTrue(docMapper.mappers().getMapper("join_field") == ParentJoinFieldMapper.getMapper(indexService.mapperService())); + + { + final String updateMapping = XContentFactory.jsonBuilder() .startObject() .startObject("properties") .startObject("join_field") .field("type", "join") .startObject("relations") - .field("parent", "child") .array("child", "grand_child1", "grand_child2") .endObject() .endObject() .endObject() .endObject() - ); - IndexService indexService = createIndex("test"); - DocumentMapper docMapper = indexService.mapperService() - .merge("type", new CompressedXContent(mapping), MapperService.MergeReason.MAPPING_UPDATE); - assertTrue(docMapper.mappers().getMapper("join_field") == ParentJoinFieldMapper.getMapper(indexService.mapperService())); - - { - final String updateMapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject("join_field") - .field("type", "join") - .startObject("relations") - .array("child", "grand_child1", "grand_child2") - .endObject() - .endObject() - .endObject() - .endObject() - ); + .toString(); IllegalArgumentException exc = expectThrows( IllegalArgumentException.class, () -> indexService.mapperService() @@ -365,20 +359,19 @@ public void testUpdateRelations() throws Exception { } { - final String updateMapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject("join_field") - .field("type", "join") - .startObject("relations") - .field("parent", "child") - .field("child", "grand_child1") - .endObject() - .endObject() - .endObject() - .endObject() - ); + final String updateMapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject("join_field") + .field("type", "join") + .startObject("relations") + .field("parent", "child") + .field("child", "grand_child1") + .endObject() + .endObject() + .endObject() + .endObject() + .toString(); IllegalArgumentException exc = expectThrows( IllegalArgumentException.class, () -> indexService.mapperService() @@ -388,21 +381,20 @@ public void testUpdateRelations() throws Exception { } { - final String updateMapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject("join_field") - .field("type", "join") - .startObject("relations") - .field("uber_parent", "parent") - .field("parent", "child") - .array("child", "grand_child1", "grand_child2") - .endObject() - .endObject() - .endObject() - .endObject() - ); + final String updateMapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject("join_field") + .field("type", "join") + .startObject("relations") + .field("uber_parent", "parent") + .field("parent", "child") + .array("child", "grand_child1", "grand_child2") + .endObject() + .endObject() + .endObject() + .endObject() + .toString(); IllegalArgumentException exc = expectThrows( IllegalArgumentException.class, () -> indexService.mapperService() @@ -412,21 +404,20 @@ public void testUpdateRelations() throws Exception { } { - final String updateMapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject("join_field") - .field("type", "join") - .startObject("relations") - .field("parent", "child") - .array("child", "grand_child1", "grand_child2") - .field("grand_child2", "grand_grand_child") - .endObject() - .endObject() - .endObject() - .endObject() - ); + final String updateMapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject("join_field") + .field("type", "join") + .startObject("relations") + .field("parent", "child") + .array("child", "grand_child1", "grand_child2") + .field("grand_child2", "grand_grand_child") + .endObject() + .endObject() + .endObject() + .endObject() + .toString(); IllegalArgumentException exc = expectThrows( IllegalArgumentException.class, () -> indexService.mapperService() @@ -436,20 +427,19 @@ public void testUpdateRelations() throws Exception { } { - final String updateMapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject("join_field") - .field("type", "join") - .startObject("relations") - .array("parent", "child", "child2") - .array("child", "grand_child1", "grand_child2") - .endObject() - .endObject() - .endObject() - .endObject() - ); + final String updateMapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject("join_field") + .field("type", "join") + .startObject("relations") + .array("parent", "child", "child2") + .array("child", "grand_child1", "grand_child2") + .endObject() + .endObject() + .endObject() + .endObject() + .toString(); docMapper = indexService.mapperService() .merge("type", new CompressedXContent(updateMapping), MapperService.MergeReason.MAPPING_UPDATE); ParentJoinFieldMapper mapper = ParentJoinFieldMapper.getMapper(indexService.mapperService()); @@ -462,21 +452,20 @@ public void testUpdateRelations() throws Exception { } { - final String updateMapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject("join_field") - .field("type", "join") - .startObject("relations") - .array("parent", "child", "child2") - .array("child", "grand_child1", "grand_child2") - .array("other", "child_other1", "child_other2") - .endObject() - .endObject() - .endObject() - .endObject() - ); + final String updateMapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject("join_field") + .field("type", "join") + .startObject("relations") + .array("parent", "child", "child2") + .array("child", "grand_child1", "grand_child2") + .array("other", "child_other1", "child_other2") + .endObject() + .endObject() + .endObject() + .endObject() + .toString(); docMapper = indexService.mapperService() .merge("type", new CompressedXContent(updateMapping), MapperService.MergeReason.MAPPING_UPDATE); ParentJoinFieldMapper mapper = ParentJoinFieldMapper.getMapper(indexService.mapperService()); @@ -492,23 +481,22 @@ public void testUpdateRelations() throws Exception { } public void testInvalidJoinFieldInsideObject() throws Exception { - String mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject("object") - .startObject("properties") - .startObject("join_field") - .field("type", "join") - .startObject("relations") - .field("parent", "child") - .endObject() - .endObject() - .endObject() - .endObject() - .endObject() - .endObject() - ); + String mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject("object") + .startObject("properties") + .startObject("join_field") + .field("type", "join") + .startObject("relations") + .field("parent", "child") + .endObject() + .endObject() + .endObject() + .endObject() + .endObject() + .endObject() + .toString(); IndexService indexService = createIndex("test"); MapperParsingException exc = expectThrows( MapperParsingException.class, @@ -521,24 +509,23 @@ public void testInvalidJoinFieldInsideObject() throws Exception { } public void testInvalidJoinFieldInsideMultiFields() throws Exception { - String mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject("number") - .field("type", "integer") - .startObject("fields") - .startObject("join_field") - .field("type", "join") - .startObject("relations") - .field("parent", "child") - .endObject() - .endObject() - .endObject() - .endObject() - .endObject() - .endObject() - ); + String mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject("number") + .field("type", "integer") + .startObject("fields") + .startObject("join_field") + .field("type", "join") + .startObject("relations") + .field("parent", "child") + .endObject() + .endObject() + .endObject() + .endObject() + .endObject() + .endObject() + .toString(); IndexService indexService = createIndex("test"); MapperParsingException exc = expectThrows( MapperParsingException.class, @@ -553,26 +540,25 @@ public void testInvalidJoinFieldInsideMultiFields() throws Exception { public void testMultipleJoinFields() throws Exception { IndexService indexService = createIndex("test"); { - String mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject("join_field") - .field("type", "join") - .startObject("relations") - .field("parent", "child") - .field("child", "grand_child") - .endObject() - .endObject() - .startObject("another_join_field") - .field("type", "join") - .startObject("relations") - .field("product", "item") - .endObject() - .endObject() - .endObject() - .endObject() - ); + String mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject("join_field") + .field("type", "join") + .startObject("relations") + .field("parent", "child") + .field("child", "grand_child") + .endObject() + .endObject() + .startObject("another_join_field") + .field("type", "join") + .startObject("relations") + .field("product", "item") + .endObject() + .endObject() + .endObject() + .endObject() + .toString(); MapperParsingException exc = expectThrows( MapperParsingException.class, () -> indexService.mapperService().merge("type", new CompressedXContent(mapping), MapperService.MergeReason.MAPPING_UPDATE) @@ -581,43 +567,7 @@ public void testMultipleJoinFields() throws Exception { } { - String mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject("join_field") - .field("type", "join") - .startObject("relations") - .field("parent", "child") - .field("child", "grand_child") - .endObject() - .endObject() - .endObject() - .endObject() - ); - indexService.mapperService().merge("type", new CompressedXContent(mapping), MapperService.MergeReason.MAPPING_UPDATE); - String updateMapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject("another_join_field") - .field("type", "join") - .endObject() - .endObject() - .endObject() - ); - MapperParsingException exc = expectThrows( - MapperParsingException.class, - () -> indexService.mapperService() - .merge("type", new CompressedXContent(updateMapping), MapperService.MergeReason.MAPPING_UPDATE) - ); - assertThat(exc.getMessage(), containsString("Field [_parent_join] is defined more than once")); - } - } - - public void testEagerGlobalOrdinals() throws Exception { - String mapping = Strings.toString( - XContentFactory.jsonBuilder() + String mapping = XContentFactory.jsonBuilder() .startObject() .startObject("properties") .startObject("join_field") @@ -629,7 +579,40 @@ public void testEagerGlobalOrdinals() throws Exception { .endObject() .endObject() .endObject() - ); + .toString(); + indexService.mapperService().merge("type", new CompressedXContent(mapping), MapperService.MergeReason.MAPPING_UPDATE); + String updateMapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject("another_join_field") + .field("type", "join") + .endObject() + .endObject() + .endObject() + .toString(); + MapperParsingException exc = expectThrows( + MapperParsingException.class, + () -> indexService.mapperService() + .merge("type", new CompressedXContent(updateMapping), MapperService.MergeReason.MAPPING_UPDATE) + ); + assertThat(exc.getMessage(), containsString("Field [_parent_join] is defined more than once")); + } + } + + public void testEagerGlobalOrdinals() throws Exception { + String mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject("join_field") + .field("type", "join") + .startObject("relations") + .field("parent", "child") + .field("child", "grand_child") + .endObject() + .endObject() + .endObject() + .endObject() + .toString(); IndexService service = createIndex("test"); DocumentMapper docMapper = service.mapperService() .merge("type", new CompressedXContent(mapping), MapperService.MergeReason.MAPPING_UPDATE); @@ -640,21 +623,20 @@ public void testEagerGlobalOrdinals() throws Exception { assertNotNull(service.mapperService().fieldType("join_field#child")); assertTrue(service.mapperService().fieldType("join_field#child").eagerGlobalOrdinals()); - mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject("join_field") - .field("type", "join") - .field("eager_global_ordinals", false) - .startObject("relations") - .field("parent", "child") - .field("child", "grand_child") - .endObject() - .endObject() - .endObject() - .endObject() - ); + mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject("join_field") + .field("type", "join") + .field("eager_global_ordinals", false) + .startObject("relations") + .field("parent", "child") + .field("child", "grand_child") + .endObject() + .endObject() + .endObject() + .endObject() + .toString(); service.mapperService().merge("type", new CompressedXContent(mapping), MapperService.MergeReason.MAPPING_UPDATE); assertFalse(service.mapperService().fieldType("join_field").eagerGlobalOrdinals()); assertNotNull(service.mapperService().fieldType("join_field#parent")); diff --git a/modules/parent-join/src/test/java/org/opensearch/join/query/HasChildQueryBuilderTests.java b/modules/parent-join/src/test/java/org/opensearch/join/query/HasChildQueryBuilderTests.java index 6610b103509b0..96220c247d909 100644 --- a/modules/parent-join/src/test/java/org/opensearch/join/query/HasChildQueryBuilderTests.java +++ b/modules/parent-join/src/test/java/org/opensearch/join/query/HasChildQueryBuilderTests.java @@ -46,7 +46,6 @@ import org.apache.lucene.search.similarities.Similarity; import org.opensearch.OpenSearchException; import org.opensearch.Version; -import org.opensearch.common.Strings; import org.opensearch.common.compress.CompressedXContent; import org.opensearch.common.settings.Settings; import org.opensearch.core.xcontent.XContentBuilder; @@ -141,7 +140,7 @@ protected void initializeAdditionalMappings(MapperService mapperService) throws .endObject() .endObject(); - mapperService.merge(TYPE, new CompressedXContent(Strings.toString(mapping)), MapperService.MergeReason.MAPPING_UPDATE); + mapperService.merge(TYPE, new CompressedXContent(mapping.toString()), MapperService.MergeReason.MAPPING_UPDATE); } /** diff --git a/modules/parent-join/src/test/java/org/opensearch/join/query/HasParentQueryBuilderTests.java b/modules/parent-join/src/test/java/org/opensearch/join/query/HasParentQueryBuilderTests.java index 9921b6b040901..63af6873e14af 100644 --- a/modules/parent-join/src/test/java/org/opensearch/join/query/HasParentQueryBuilderTests.java +++ b/modules/parent-join/src/test/java/org/opensearch/join/query/HasParentQueryBuilderTests.java @@ -37,7 +37,6 @@ import org.apache.lucene.search.join.ScoreMode; import org.opensearch.OpenSearchException; import org.opensearch.Version; -import org.opensearch.common.Strings; import org.opensearch.common.compress.CompressedXContent; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.index.mapper.MapperService; @@ -122,7 +121,7 @@ protected void initializeAdditionalMappings(MapperService mapperService) throws .endObject() .endObject(); - mapperService.merge(TYPE, new CompressedXContent(Strings.toString(mapping)), MapperService.MergeReason.MAPPING_UPDATE); + mapperService.merge(TYPE, new CompressedXContent(mapping.toString()), MapperService.MergeReason.MAPPING_UPDATE); } /** diff --git a/modules/parent-join/src/test/java/org/opensearch/join/query/ParentIdQueryBuilderTests.java b/modules/parent-join/src/test/java/org/opensearch/join/query/ParentIdQueryBuilderTests.java index 88da6a6953d1e..ec555448fd218 100644 --- a/modules/parent-join/src/test/java/org/opensearch/join/query/ParentIdQueryBuilderTests.java +++ b/modules/parent-join/src/test/java/org/opensearch/join/query/ParentIdQueryBuilderTests.java @@ -39,7 +39,6 @@ import org.apache.lucene.search.Query; import org.apache.lucene.search.TermQuery; import org.opensearch.OpenSearchException; -import org.opensearch.common.Strings; import org.opensearch.common.compress.CompressedXContent; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.index.mapper.MapperService; @@ -111,7 +110,7 @@ protected void initializeAdditionalMappings(MapperService mapperService) throws .endObject() .endObject(); - mapperService.merge(TYPE, new CompressedXContent(Strings.toString(mapping)), MapperService.MergeReason.MAPPING_UPDATE); + mapperService.merge(TYPE, new CompressedXContent(mapping.toString()), MapperService.MergeReason.MAPPING_UPDATE); } @Override diff --git a/modules/percolator/src/test/java/org/opensearch/percolator/CandidateQueryTests.java b/modules/percolator/src/test/java/org/opensearch/percolator/CandidateQueryTests.java index 1040d014483e1..9f80069b99c10 100644 --- a/modules/percolator/src/test/java/org/opensearch/percolator/CandidateQueryTests.java +++ b/modules/percolator/src/test/java/org/opensearch/percolator/CandidateQueryTests.java @@ -94,7 +94,6 @@ import org.opensearch.Version; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.CheckedFunction; -import org.opensearch.common.Strings; import org.opensearch.core.common.bytes.BytesArray; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.common.compress.CompressedXContent; @@ -163,51 +162,49 @@ public void init() throws Exception { indexService = createIndex(indexName, Settings.EMPTY); mapperService = indexService.mapperService(); - String mapper = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("type") - .startObject("properties") - .startObject("int_field") - .field("type", "integer") - .endObject() - .startObject("long_field") - .field("type", "long") - .endObject() - .startObject("half_float_field") - .field("type", "half_float") - .endObject() - .startObject("float_field") - .field("type", "float") - .endObject() - .startObject("double_field") - .field("type", "double") - .endObject() - .startObject("ip_field") - .field("type", "ip") - .endObject() - .startObject("field") - .field("type", "keyword") - .endObject() - .endObject() - .endObject() - .endObject() - ); + String mapper = XContentFactory.jsonBuilder() + .startObject() + .startObject("type") + .startObject("properties") + .startObject("int_field") + .field("type", "integer") + .endObject() + .startObject("long_field") + .field("type", "long") + .endObject() + .startObject("half_float_field") + .field("type", "half_float") + .endObject() + .startObject("float_field") + .field("type", "float") + .endObject() + .startObject("double_field") + .field("type", "double") + .endObject() + .startObject("ip_field") + .field("type", "ip") + .endObject() + .startObject("field") + .field("type", "keyword") + .endObject() + .endObject() + .endObject() + .endObject() + .toString(); documentMapper = mapperService.merge("type", new CompressedXContent(mapper), MapperService.MergeReason.MAPPING_UPDATE); String queryField = "query_field"; - String percolatorMapper = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("type") - .startObject("properties") - .startObject(queryField) - .field("type", "percolator") - .endObject() - .endObject() - .endObject() - .endObject() - ); + String percolatorMapper = XContentFactory.jsonBuilder() + .startObject() + .startObject("type") + .startObject("properties") + .startObject(queryField) + .field("type", "percolator") + .endObject() + .endObject() + .endObject() + .endObject() + .toString(); mapperService.merge("type", new CompressedXContent(percolatorMapper), MapperService.MergeReason.MAPPING_UPDATE); fieldMapper = (PercolatorFieldMapper) mapperService.documentMapper().mappers().getMapper(queryField); fieldType = (PercolatorFieldMapper.PercolatorFieldType) fieldMapper.fieldType(); diff --git a/modules/percolator/src/test/java/org/opensearch/percolator/PercolateQueryBuilderTests.java b/modules/percolator/src/test/java/org/opensearch/percolator/PercolateQueryBuilderTests.java index 34ed195cd0f23..93b78d67bf6f1 100644 --- a/modules/percolator/src/test/java/org/opensearch/percolator/PercolateQueryBuilderTests.java +++ b/modules/percolator/src/test/java/org/opensearch/percolator/PercolateQueryBuilderTests.java @@ -38,7 +38,6 @@ import org.opensearch.action.admin.indices.mapping.put.PutMappingRequest; import org.opensearch.action.get.GetRequest; import org.opensearch.action.get.GetResponse; -import org.opensearch.common.Strings; import org.opensearch.core.common.bytes.BytesArray; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.common.compress.CompressedXContent; @@ -109,15 +108,13 @@ protected void initializeAdditionalMappings(MapperService mapperService) throws mapperService.merge( docType, new CompressedXContent( - Strings.toString( - PutMappingRequest.simpleMapping(queryField, "type=percolator", aliasField, "type=alias,path=" + queryField) - ) + PutMappingRequest.simpleMapping(queryField, "type=percolator", aliasField, "type=alias,path=" + queryField).toString() ), MapperService.MergeReason.MAPPING_UPDATE ); mapperService.merge( docType, - new CompressedXContent(Strings.toString(PutMappingRequest.simpleMapping(TEXT_FIELD_NAME, "type=text"))), + new CompressedXContent(PutMappingRequest.simpleMapping(TEXT_FIELD_NAME, "type=text").toString()), MapperService.MergeReason.MAPPING_UPDATE ); } diff --git a/modules/percolator/src/test/java/org/opensearch/percolator/PercolateWithNestedQueryBuilderTests.java b/modules/percolator/src/test/java/org/opensearch/percolator/PercolateWithNestedQueryBuilderTests.java index 748b79d70af07..677d169162c74 100644 --- a/modules/percolator/src/test/java/org/opensearch/percolator/PercolateWithNestedQueryBuilderTests.java +++ b/modules/percolator/src/test/java/org/opensearch/percolator/PercolateWithNestedQueryBuilderTests.java @@ -33,7 +33,6 @@ package org.opensearch.percolator; import org.opensearch.action.admin.indices.mapping.put.PutMappingRequest; -import org.opensearch.common.Strings; import org.opensearch.core.common.bytes.BytesArray; import org.opensearch.common.compress.CompressedXContent; import org.opensearch.common.xcontent.XContentType; @@ -50,7 +49,7 @@ protected void initializeAdditionalMappings(MapperService mapperService) throws super.initializeAdditionalMappings(mapperService); mapperService.merge( "_doc", - new CompressedXContent(Strings.toString(PutMappingRequest.simpleMapping("some_nested_object", "type=nested"))), + new CompressedXContent(PutMappingRequest.simpleMapping("some_nested_object", "type=nested").toString()), MapperService.MergeReason.MAPPING_UPDATE ); } diff --git a/modules/percolator/src/test/java/org/opensearch/percolator/PercolatorFieldMapperTests.java b/modules/percolator/src/test/java/org/opensearch/percolator/PercolatorFieldMapperTests.java index 77a4718b1d755..c5e2a1f68de9c 100644 --- a/modules/percolator/src/test/java/org/opensearch/percolator/PercolatorFieldMapperTests.java +++ b/modules/percolator/src/test/java/org/opensearch/percolator/PercolatorFieldMapperTests.java @@ -56,7 +56,6 @@ import org.opensearch.Version; import org.opensearch.action.support.PlainActionFuture; import org.opensearch.cluster.metadata.IndexMetadata; -import org.opensearch.common.Strings; import org.opensearch.core.common.bytes.BytesArray; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.common.collect.Tuple; @@ -163,67 +162,65 @@ public void init() throws Exception { indexService = createIndex("test"); mapperService = indexService.mapperService(); - String mapper = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject("field") - .field("type", "text") - .endObject() - .startObject("field1") - .field("type", "text") - .endObject() - .startObject("field2") - .field("type", "text") - .endObject() - .startObject("_field3") - .field("type", "text") - .endObject() - .startObject("field4") - .field("type", "text") - .endObject() - .startObject("number_field1") - .field("type", "integer") - .endObject() - .startObject("number_field2") - .field("type", "long") - .endObject() - .startObject("number_field3") - .field("type", "long") - .endObject() - .startObject("number_field4") - .field("type", "half_float") - .endObject() - .startObject("number_field5") - .field("type", "float") - .endObject() - .startObject("number_field6") - .field("type", "double") - .endObject() - .startObject("number_field7") - .field("type", "ip") - .endObject() - .startObject("date_field") - .field("type", "date") - .endObject() - .endObject() - .endObject() - ); + String mapper = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject("field") + .field("type", "text") + .endObject() + .startObject("field1") + .field("type", "text") + .endObject() + .startObject("field2") + .field("type", "text") + .endObject() + .startObject("_field3") + .field("type", "text") + .endObject() + .startObject("field4") + .field("type", "text") + .endObject() + .startObject("number_field1") + .field("type", "integer") + .endObject() + .startObject("number_field2") + .field("type", "long") + .endObject() + .startObject("number_field3") + .field("type", "long") + .endObject() + .startObject("number_field4") + .field("type", "half_float") + .endObject() + .startObject("number_field5") + .field("type", "float") + .endObject() + .startObject("number_field6") + .field("type", "double") + .endObject() + .startObject("number_field7") + .field("type", "ip") + .endObject() + .startObject("date_field") + .field("type", "date") + .endObject() + .endObject() + .endObject() + .toString(); mapperService.merge(MapperService.SINGLE_MAPPING_NAME, new CompressedXContent(mapper), MapperService.MergeReason.MAPPING_UPDATE); } private void addQueryFieldMappings() throws Exception { fieldName = randomAlphaOfLength(4); - String percolatorMapper = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject(fieldName) - .field("type", "percolator") - .endObject() - .endObject() - .endObject() - ); + String percolatorMapper = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject(fieldName) + .field("type", "percolator") + .endObject() + .endObject() + .endObject() + .toString(); mapperService.merge( MapperService.SINGLE_MAPPING_NAME, new CompressedXContent(percolatorMapper), @@ -710,17 +707,16 @@ public void testAllowNoAdditionalSettings() throws Exception { IndexService indexService = createIndex("test1", Settings.EMPTY); MapperService mapperService = indexService.mapperService(); - String percolatorMapper = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject(fieldName) - .field("type", "percolator") - .field("index", "no") - .endObject() - .endObject() - .endObject() - ); + String percolatorMapper = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject(fieldName) + .field("type", "percolator") + .field("index", "no") + .endObject() + .endObject() + .endObject() + .toString(); MapperParsingException e = expectThrows( MapperParsingException.class, () -> mapperService.merge( @@ -735,21 +731,20 @@ public void testAllowNoAdditionalSettings() throws Exception { // multiple percolator fields are allowed in the mapping, but only one field can be used at index time. public void testMultiplePercolatorFields() throws Exception { String typeName = MapperService.SINGLE_MAPPING_NAME; - String percolatorMapper = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject(typeName) - .startObject("properties") - .startObject("query_field1") - .field("type", "percolator") - .endObject() - .startObject("query_field2") - .field("type", "percolator") - .endObject() - .endObject() - .endObject() - .endObject() - ); + String percolatorMapper = XContentFactory.jsonBuilder() + .startObject() + .startObject(typeName) + .startObject("properties") + .startObject("query_field1") + .field("type", "percolator") + .endObject() + .startObject("query_field2") + .field("type", "percolator") + .endObject() + .endObject() + .endObject() + .endObject() + .toString(); mapperService.merge(typeName, new CompressedXContent(percolatorMapper), MapperService.MergeReason.MAPPING_UPDATE); QueryBuilder queryBuilder = matchQuery("field", "value"); @@ -775,23 +770,22 @@ public void testMultiplePercolatorFields() throws Exception { // percolator field can be nested under an object field, but only one query can be specified per document public void testNestedPercolatorField() throws Exception { String typeName = MapperService.SINGLE_MAPPING_NAME; - String percolatorMapper = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject(typeName) - .startObject("properties") - .startObject("object_field") - .field("type", "object") - .startObject("properties") - .startObject("query_field") - .field("type", "percolator") - .endObject() - .endObject() - .endObject() - .endObject() - .endObject() - .endObject() - ); + String percolatorMapper = XContentFactory.jsonBuilder() + .startObject() + .startObject(typeName) + .startObject("properties") + .startObject("object_field") + .field("type", "object") + .startObject("properties") + .startObject("query_field") + .field("type", "percolator") + .endObject() + .endObject() + .endObject() + .endObject() + .endObject() + .endObject() + .toString(); mapperService.merge(typeName, new CompressedXContent(percolatorMapper), MapperService.MergeReason.MAPPING_UPDATE); QueryBuilder queryBuilder = matchQuery("field", "value"); @@ -907,18 +901,17 @@ private void assertQueryBuilder(BytesRef actual, QueryBuilder expected) throws I } public void testEmptyName() throws Exception { - String mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("type1") - .startObject("properties") - .startObject("") - .field("type", "percolator") - .endObject() - .endObject() - .endObject() - .endObject() - ); + String mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("type1") + .startObject("properties") + .startObject("") + .field("type", "percolator") + .endObject() + .endObject() + .endObject() + .endObject() + .toString(); DocumentMapperParser parser = mapperService.documentMapperParser(); IllegalArgumentException e = expectThrows( @@ -951,7 +944,7 @@ public void testImplicitlySetDefaultScriptLang() throws Exception { BytesReference.bytes( XContentFactory.jsonBuilder() .startObject() - .rawField(fieldName, new BytesArray(Strings.toString(query)).streamInput(), query.contentType()) + .rawField(fieldName, new BytesArray(query.toString()).streamInput(), query.contentType()) .endObject() ), XContentType.JSON @@ -998,7 +991,7 @@ public void testImplicitlySetDefaultScriptLang() throws Exception { BytesReference.bytes( XContentFactory.jsonBuilder() .startObject() - .rawField(fieldName, new BytesArray(Strings.toString(query)).streamInput(), query.contentType()) + .rawField(fieldName, new BytesArray(query.toString()).streamInput(), query.contentType()) .endObject() ), XContentType.JSON diff --git a/modules/rank-eval/src/main/java/org/opensearch/index/rankeval/RankEvalResponse.java b/modules/rank-eval/src/main/java/org/opensearch/index/rankeval/RankEvalResponse.java index 4dbc348fe458e..68c61183a4486 100644 --- a/modules/rank-eval/src/main/java/org/opensearch/index/rankeval/RankEvalResponse.java +++ b/modules/rank-eval/src/main/java/org/opensearch/index/rankeval/RankEvalResponse.java @@ -35,8 +35,8 @@ import org.opensearch.OpenSearchException; import org.opensearch.action.ActionResponse; import org.opensearch.core.ParseField; -import org.opensearch.common.Strings; import org.opensearch.common.collect.Tuple; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.xcontent.ConstructingObjectParser; diff --git a/modules/rank-eval/src/main/java/org/opensearch/index/rankeval/RankEvalSpec.java b/modules/rank-eval/src/main/java/org/opensearch/index/rankeval/RankEvalSpec.java index 9585e79e69cf1..44eeceb117794 100644 --- a/modules/rank-eval/src/main/java/org/opensearch/index/rankeval/RankEvalSpec.java +++ b/modules/rank-eval/src/main/java/org/opensearch/index/rankeval/RankEvalSpec.java @@ -34,7 +34,7 @@ import org.opensearch.core.ParseField; import org.opensearch.core.common.ParsingException; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; diff --git a/modules/rank-eval/src/main/java/org/opensearch/index/rankeval/RatedDocument.java b/modules/rank-eval/src/main/java/org/opensearch/index/rankeval/RatedDocument.java index 02ac9182c4f35..e91b8671d0804 100644 --- a/modules/rank-eval/src/main/java/org/opensearch/index/rankeval/RatedDocument.java +++ b/modules/rank-eval/src/main/java/org/opensearch/index/rankeval/RatedDocument.java @@ -129,7 +129,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws @Override public String toString() { - return org.opensearch.common.Strings.toString(XContentType.JSON, this); + return Strings.toString(XContentType.JSON, this); } @Override diff --git a/modules/rank-eval/src/main/java/org/opensearch/index/rankeval/RatedRequest.java b/modules/rank-eval/src/main/java/org/opensearch/index/rankeval/RatedRequest.java index c5f899cbefdf6..78c2dbd33182f 100644 --- a/modules/rank-eval/src/main/java/org/opensearch/index/rankeval/RatedRequest.java +++ b/modules/rank-eval/src/main/java/org/opensearch/index/rankeval/RatedRequest.java @@ -33,7 +33,7 @@ package org.opensearch.index.rankeval; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; diff --git a/modules/rank-eval/src/test/java/org/opensearch/index/rankeval/DiscountedCumulativeGainTests.java b/modules/rank-eval/src/test/java/org/opensearch/index/rankeval/DiscountedCumulativeGainTests.java index d97d5a3a7dcd5..edda52c57f280 100644 --- a/modules/rank-eval/src/test/java/org/opensearch/index/rankeval/DiscountedCumulativeGainTests.java +++ b/modules/rank-eval/src/test/java/org/opensearch/index/rankeval/DiscountedCumulativeGainTests.java @@ -33,7 +33,7 @@ package org.opensearch.index.rankeval; import org.opensearch.action.OriginalIndices; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.common.io.stream.NamedWriteableRegistry; import org.opensearch.core.xcontent.MediaTypeRegistry; diff --git a/modules/rank-eval/src/test/java/org/opensearch/index/rankeval/RankEvalSpecTests.java b/modules/rank-eval/src/test/java/org/opensearch/index/rankeval/RankEvalSpecTests.java index c42c0722e0fae..5601c972375f1 100644 --- a/modules/rank-eval/src/test/java/org/opensearch/index/rankeval/RankEvalSpecTests.java +++ b/modules/rank-eval/src/test/java/org/opensearch/index/rankeval/RankEvalSpecTests.java @@ -32,7 +32,6 @@ package org.opensearch.index.rankeval; -import org.opensearch.common.Strings; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.common.io.stream.NamedWriteableRegistry; import org.opensearch.core.xcontent.NamedXContentRegistry; @@ -103,7 +102,7 @@ static RankEvalSpec createTestItem() { builder.startObject(); builder.field("field", randomAlphaOfLengthBetween(1, 5)); builder.endObject(); - script = Strings.toString(builder); + script = builder.toString(); } catch (IOException e) { // this shouldn't happen in tests, re-throw just not to swallow it throw new RuntimeException(e); diff --git a/modules/reindex/src/main/java/org/opensearch/index/reindex/remote/RemoteRequestBuilders.java b/modules/reindex/src/main/java/org/opensearch/index/reindex/remote/RemoteRequestBuilders.java index 511c44ae3c2b6..68c30b616adfd 100644 --- a/modules/reindex/src/main/java/org/opensearch/index/reindex/remote/RemoteRequestBuilders.java +++ b/modules/reindex/src/main/java/org/opensearch/index/reindex/remote/RemoteRequestBuilders.java @@ -38,7 +38,6 @@ import org.opensearch.Version; import org.opensearch.action.search.SearchRequest; import org.opensearch.client.Request; -import org.opensearch.common.Strings; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.unit.TimeValue; @@ -181,7 +180,7 @@ static Request initialSearch(SearchRequest searchRequest, BytesReference query, } entity.endObject(); - request.setJsonEntity(Strings.toString(entity)); + request.setJsonEntity(entity.toString()); } catch (IOException e) { throw new OpenSearchException("unexpected error building entity", e); } @@ -246,7 +245,7 @@ static Request scroll(String scroll, TimeValue keepAlive, Version remoteVersion) try (XContentBuilder entity = JsonXContent.contentBuilder()) { entity.startObject().field("scroll_id", scroll).endObject(); - request.setJsonEntity(Strings.toString(entity)); + request.setJsonEntity(entity.toString()); } catch (IOException e) { throw new OpenSearchException("failed to build scroll entity", e); } @@ -263,7 +262,7 @@ static Request clearScroll(String scroll, Version remoteVersion) { } try (XContentBuilder entity = JsonXContent.contentBuilder()) { entity.startObject().array("scroll_id", scroll).endObject(); - request.setJsonEntity(Strings.toString(entity)); + request.setJsonEntity(entity.toString()); } catch (IOException e) { throw new OpenSearchException("failed to build clear scroll entity", e); } diff --git a/modules/repository-url/src/yamlRestTest/java/org/opensearch/repositories/url/RepositoryURLClientYamlTestSuiteIT.java b/modules/repository-url/src/yamlRestTest/java/org/opensearch/repositories/url/RepositoryURLClientYamlTestSuiteIT.java index 705cbafd1bd3a..16c4ddf6aaabf 100644 --- a/modules/repository-url/src/yamlRestTest/java/org/opensearch/repositories/url/RepositoryURLClientYamlTestSuiteIT.java +++ b/modules/repository-url/src/yamlRestTest/java/org/opensearch/repositories/url/RepositoryURLClientYamlTestSuiteIT.java @@ -36,7 +36,6 @@ import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; import org.opensearch.client.Request; import org.opensearch.client.Response; -import org.opensearch.common.Strings; import org.opensearch.common.io.PathUtils; import org.opensearch.common.settings.Settings; import org.opensearch.core.xcontent.ToXContent; @@ -144,7 +143,7 @@ private static HttpEntity buildRepositorySettings(final String type, final Setti builder.endObject(); } builder.endObject(); - return new StringEntity(Strings.toString(builder), ContentType.APPLICATION_JSON); + return new StringEntity(builder.toString(), ContentType.APPLICATION_JSON); } } } diff --git a/modules/transport-netty4/src/javaRestTest/java/org/opensearch/rest/Netty4HeadBodyIsEmptyIT.java b/modules/transport-netty4/src/javaRestTest/java/org/opensearch/rest/Netty4HeadBodyIsEmptyIT.java index 4b962401387b7..b4b15c22258de 100644 --- a/modules/transport-netty4/src/javaRestTest/java/org/opensearch/rest/Netty4HeadBodyIsEmptyIT.java +++ b/modules/transport-netty4/src/javaRestTest/java/org/opensearch/rest/Netty4HeadBodyIsEmptyIT.java @@ -34,7 +34,6 @@ import org.opensearch.client.Request; import org.opensearch.client.Response; -import org.opensearch.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.test.rest.OpenSearchRestTestCase; import org.hamcrest.Matcher; @@ -68,7 +67,7 @@ private void createTestDoc(final String indexName) throws IOException { } builder.endObject(); Request request = new Request("PUT", "/" + indexName + "/_doc/" + "1"); - request.setJsonEntity(Strings.toString(builder)); + request.setJsonEntity(builder.toString()); client().performRequest(request); } } @@ -109,7 +108,7 @@ public void testAliasExists() throws IOException { builder.endObject(); Request request = new Request("POST", "/_aliases"); - request.setJsonEntity(Strings.toString(builder)); + request.setJsonEntity(builder.toString()); client().performRequest(request); headTestCase("/_alias/test_alias", emptyMap(), greaterThan(0)); headTestCase("/test/_alias/test_alias", emptyMap(), greaterThan(0)); @@ -136,7 +135,7 @@ public void testTemplateExists() throws IOException { builder.endObject(); Request request = new Request("PUT", "/_template/template"); - request.setJsonEntity(Strings.toString(builder)); + request.setJsonEntity(builder.toString()); client().performRequest(request); headTestCase("/_template/template", emptyMap(), greaterThan(0)); } @@ -163,7 +162,7 @@ public void testGetSourceAction() throws IOException { builder.endObject(); Request request = new Request("PUT", "/test-no-source"); - request.setJsonEntity(Strings.toString(builder)); + request.setJsonEntity(builder.toString()); client().performRequest(request); createTestDoc("test-no-source"); headTestCase("/test-no-source/_source/1", emptyMap(), NOT_FOUND.getStatus(), greaterThan(0)); diff --git a/plugins/analysis-icu/src/test/java/org/opensearch/index/mapper/ICUCollationKeywordFieldMapperTests.java b/plugins/analysis-icu/src/test/java/org/opensearch/index/mapper/ICUCollationKeywordFieldMapperTests.java index 37cb73e21b5d4..0a2f48f4215cb 100644 --- a/plugins/analysis-icu/src/test/java/org/opensearch/index/mapper/ICUCollationKeywordFieldMapperTests.java +++ b/plugins/analysis-icu/src/test/java/org/opensearch/index/mapper/ICUCollationKeywordFieldMapperTests.java @@ -39,7 +39,6 @@ import org.apache.lucene.index.IndexableField; import org.apache.lucene.index.IndexableFieldType; import org.apache.lucene.util.BytesRef; -import org.opensearch.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.plugin.analysis.icu.AnalysisICUPlugin; import org.opensearch.plugins.Plugin; @@ -96,7 +95,7 @@ protected void writeFieldValue(XContentBuilder builder) throws IOException { public void testDefaults() throws Exception { DocumentMapper mapper = createDocumentMapper(fieldMapping(this::minimalMapping)); - assertEquals(Strings.toString(fieldMapping(this::minimalMapping)), mapper.mappingSource().toString()); + assertEquals(fieldMapping(this::minimalMapping).toString(), mapper.mappingSource().toString()); ParsedDocument doc = mapper.parse(source(b -> b.field("field", "1234"))); IndexableField[] fields = doc.rootDoc().getFields("field"); diff --git a/plugins/discovery-gce/qa/gce/src/yamlRestTest/java/org/opensearch/cloud/gce/GCEFixture.java b/plugins/discovery-gce/qa/gce/src/yamlRestTest/java/org/opensearch/cloud/gce/GCEFixture.java index f588316fdc32f..db23944b25369 100644 --- a/plugins/discovery-gce/qa/gce/src/yamlRestTest/java/org/opensearch/cloud/gce/GCEFixture.java +++ b/plugins/discovery-gce/qa/gce/src/yamlRestTest/java/org/opensearch/cloud/gce/GCEFixture.java @@ -33,7 +33,6 @@ import org.apache.http.client.methods.HttpGet; -import org.opensearch.common.Strings; import org.opensearch.common.SuppressForbidden; import org.opensearch.common.collect.MapBuilder; import org.opensearch.common.path.PathTrie; @@ -139,13 +138,12 @@ private PathTrie defaultHandlers() { handlers.insert( nonAuthPath(HttpGet.METHOD_NAME, "/computeMetadata/v1/instance/service-accounts/default/token"), request -> jsonValue.apply( - Strings.toString( - jsonBuilder().startObject() - .field("access_token", TOKEN) - .field("expires_in", TimeUnit.HOURS.toSeconds(1)) - .field("token_type", TOKEN_TYPE) - .endObject() - ) + jsonBuilder().startObject() + .field("access_token", TOKEN) + .field("expires_in", TimeUnit.HOURS.toSeconds(1)) + .field("token_type", TOKEN_TYPE) + .endObject() + .toString() ) ); @@ -179,9 +177,7 @@ private PathTrie defaultHandlers() { ); } - final String json = Strings.toString( - jsonBuilder().startObject().field("id", "test-instances").field("items", items).endObject() - ); + final String json = jsonBuilder().startObject().field("id", "test-instances").field("items", items).endObject().toString(); final byte[] responseAsBytes = json.getBytes(StandardCharsets.UTF_8); final Map headers = new HashMap<>(JSON_CONTENT_TYPE); @@ -213,29 +209,28 @@ protected Response handle(final Request request) throws IOException { } private static Response newError(final RestStatus status, final String code, final String message) throws IOException { - final String response = Strings.toString( - jsonBuilder().startObject() - .field( - "error", - MapBuilder.newMapBuilder() - .put( - "errors", - Collections.singletonList( - MapBuilder.newMapBuilder() - .put("domain", "global") - .put("reason", "required") - .put("message", message) - .put("locationType", "header") - .put("location", code) - .immutableMap() - ) + final String response = jsonBuilder().startObject() + .field( + "error", + MapBuilder.newMapBuilder() + .put( + "errors", + Collections.singletonList( + MapBuilder.newMapBuilder() + .put("domain", "global") + .put("reason", "required") + .put("message", message) + .put("locationType", "header") + .put("location", code) + .immutableMap() ) - .put("code", status.getStatus()) - .put("message", message) - .immutableMap() - ) - .endObject() - ); + ) + .put("code", status.getStatus()) + .put("message", message) + .immutableMap() + ) + .endObject() + .toString(); return new Response(status.getStatus(), JSON_CONTENT_TYPE, response.getBytes(UTF_8)); } diff --git a/plugins/events-correlation-engine/src/javaRestTest/java/org/opensearch/plugin/correlation/CorrelationVectorsEngineIT.java b/plugins/events-correlation-engine/src/javaRestTest/java/org/opensearch/plugin/correlation/CorrelationVectorsEngineIT.java index dd83dfa84dbdb..000e667f39eab 100644 --- a/plugins/events-correlation-engine/src/javaRestTest/java/org/opensearch/plugin/correlation/CorrelationVectorsEngineIT.java +++ b/plugins/events-correlation-engine/src/javaRestTest/java/org/opensearch/plugin/correlation/CorrelationVectorsEngineIT.java @@ -18,11 +18,11 @@ import org.opensearch.client.ResponseException; import org.opensearch.client.RestClient; import org.opensearch.client.WarningsHandler; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; import org.opensearch.common.xcontent.XContentFactory; -import org.opensearch.common.xcontent.XContentType; +import org.opensearch.core.common.Strings; import org.opensearch.core.rest.RestStatus; +import org.opensearch.core.xcontent.MediaTypeRegistry; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.index.IndexSettings; import org.opensearch.test.rest.OpenSearchRestTestCase; @@ -94,7 +94,7 @@ public void testQuery() throws IOException { .endObject() .endObject(); - String mapping = Strings.toString(builder); + String mapping = builder.toString(); createTestIndexWithMappingJson(client(), INDEX_NAME, mapping, getCorrelationDefaultIndexSettings()); for (int idx = 0; idx < TEST_VECTORS.length; ++idx) { @@ -209,7 +209,7 @@ public void testQueryWithWrongMapping() throws IOException { .endObject() .endObject(); - String mapping = Strings.toString(builder); + String mapping = builder.toString(); Exception ex = assertThrows(ResponseException.class, () -> { createTestIndexWithMappingJson(client(), INDEX_NAME, mapping, getCorrelationDefaultIndexSettings()); }); @@ -225,7 +225,7 @@ public void testQueryWithWrongMapping() throws IOException { private String createTestIndexWithMappingJson(RestClient client, String index, String mapping, Settings settings) throws IOException { Request request = new Request("PUT", "/" + index); - String entity = "{\"settings\": " + Strings.toString(XContentType.JSON, settings); + String entity = "{\"settings\": " + Strings.toString(MediaTypeRegistry.JSON, settings); if (mapping != null) { entity = entity + ",\"mappings\" : " + mapping; } @@ -253,7 +253,7 @@ private void addCorrelationDoc(String index, String docId, List fieldNam } builder.endObject(); - request.setJsonEntity(Strings.toString(builder)); + request.setJsonEntity(builder.toString()); Response response = client().performRequest(request); assertEquals(request.getEndpoint() + ": failed", RestStatus.CREATED, RestStatus.fromCode(response.getStatusLine().getStatusCode())); } diff --git a/plugins/events-correlation-engine/src/test/java/org/opensearch/plugin/correlation/core/index/query/CorrelationQueryBuilderTests.java b/plugins/events-correlation-engine/src/test/java/org/opensearch/plugin/correlation/core/index/query/CorrelationQueryBuilderTests.java index f23a4f25302b1..3489dfdcc4530 100644 --- a/plugins/events-correlation-engine/src/test/java/org/opensearch/plugin/correlation/core/index/query/CorrelationQueryBuilderTests.java +++ b/plugins/events-correlation-engine/src/test/java/org/opensearch/plugin/correlation/core/index/query/CorrelationQueryBuilderTests.java @@ -12,11 +12,11 @@ import org.junit.Assert; import org.opensearch.Version; import org.opensearch.cluster.ClusterModule; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.xcontent.XContentFactory; import org.opensearch.common.xcontent.XContentType; import org.opensearch.common.xcontent.json.JsonXContent; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.NamedWriteableAwareStreamInput; import org.opensearch.core.common.io.stream.NamedWriteableRegistry; import org.opensearch.core.common.io.stream.StreamInput; diff --git a/plugins/mapper-annotated-text/src/internalClusterTest/java/org/opensearch/index/mapper/annotatedtext/AnnotatedTextFieldMapperTests.java b/plugins/mapper-annotated-text/src/internalClusterTest/java/org/opensearch/index/mapper/annotatedtext/AnnotatedTextFieldMapperTests.java index d03fcd47fe991..1a8bd1ae1d2a8 100644 --- a/plugins/mapper-annotated-text/src/internalClusterTest/java/org/opensearch/index/mapper/annotatedtext/AnnotatedTextFieldMapperTests.java +++ b/plugins/mapper-annotated-text/src/internalClusterTest/java/org/opensearch/index/mapper/annotatedtext/AnnotatedTextFieldMapperTests.java @@ -47,7 +47,6 @@ import org.apache.lucene.index.Terms; import org.apache.lucene.index.TermsEnum; import org.apache.lucene.util.BytesRef; -import org.opensearch.common.Strings; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.common.xcontent.XContentFactory; @@ -396,92 +395,87 @@ public void testPositionIncrementGap() throws IOException { } public void testSearchAnalyzerSerialization() throws IOException { - String mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("_doc") - .startObject("properties") - .startObject("field") - .field("type", "annotated_text") - .field("analyzer", "standard") - .field("search_analyzer", "keyword") - .endObject() - .endObject() - .endObject() - .endObject() - ); + String mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("_doc") + .startObject("properties") + .startObject("field") + .field("type", "annotated_text") + .field("analyzer", "standard") + .field("search_analyzer", "keyword") + .endObject() + .endObject() + .endObject() + .endObject() + .toString(); DocumentMapper mapper = createDocumentMapper("_doc", mapping); assertEquals(mapping, mapper.mappingSource().toString()); // special case: default index analyzer - mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("_doc") - .startObject("properties") - .startObject("field") - .field("type", "annotated_text") - .field("analyzer", "default") - .field("search_analyzer", "keyword") - .endObject() - .endObject() - .endObject() - .endObject() - ); + mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("_doc") + .startObject("properties") + .startObject("field") + .field("type", "annotated_text") + .field("analyzer", "default") + .field("search_analyzer", "keyword") + .endObject() + .endObject() + .endObject() + .endObject() + .toString(); mapper = createDocumentMapper("_doc", mapping); assertEquals(mapping, mapper.mappingSource().toString()); - mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("_doc") - .startObject("properties") - .startObject("field") - .field("type", "annotated_text") - .field("analyzer", "keyword") - .endObject() - .endObject() - .endObject() - .endObject() - ); + mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("_doc") + .startObject("properties") + .startObject("field") + .field("type", "annotated_text") + .field("analyzer", "keyword") + .endObject() + .endObject() + .endObject() + .endObject() + .toString(); mapper = createDocumentMapper("_doc", mapping); assertEquals(mapping, mapper.mappingSource().toString()); // special case: default search analyzer - mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("_doc") - .startObject("properties") - .startObject("field") - .field("type", "annotated_text") - .field("analyzer", "keyword") - .field("search_analyzer", "default") - .endObject() - .endObject() - .endObject() - .endObject() - ); + mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("_doc") + .startObject("properties") + .startObject("field") + .field("type", "annotated_text") + .field("analyzer", "keyword") + .field("search_analyzer", "default") + .endObject() + .endObject() + .endObject() + .endObject() + .toString(); mapper = createDocumentMapper("_doc", mapping); assertEquals(mapping, mapper.mappingSource().toString()); - mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("_doc") - .startObject("properties") - .startObject("field") - .field("type", "annotated_text") - .field("analyzer", "keyword") - .endObject() - .endObject() - .endObject() - .endObject() - ); + mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("_doc") + .startObject("properties") + .startObject("field") + .field("type", "annotated_text") + .field("analyzer", "keyword") + .endObject() + .endObject() + .endObject() + .endObject() + .toString(); mapper = createDocumentMapper("_doc", mapping); XContentBuilder builder = XContentFactory.jsonBuilder(); @@ -489,48 +483,46 @@ public void testSearchAnalyzerSerialization() throws IOException { mapper.toXContent(builder, new ToXContent.MapParams(Collections.singletonMap("include_defaults", "true"))); builder.endObject(); - String mappingString = Strings.toString(builder); + String mappingString = builder.toString(); assertTrue(mappingString.contains("analyzer")); assertTrue(mappingString.contains("search_analyzer")); assertTrue(mappingString.contains("search_quote_analyzer")); } public void testSearchQuoteAnalyzerSerialization() throws IOException { - String mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("_doc") - .startObject("properties") - .startObject("field") - .field("type", "annotated_text") - .field("analyzer", "standard") - .field("search_analyzer", "standard") - .field("search_quote_analyzer", "keyword") - .endObject() - .endObject() - .endObject() - .endObject() - ); + String mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("_doc") + .startObject("properties") + .startObject("field") + .field("type", "annotated_text") + .field("analyzer", "standard") + .field("search_analyzer", "standard") + .field("search_quote_analyzer", "keyword") + .endObject() + .endObject() + .endObject() + .endObject() + .toString(); DocumentMapper mapper = createDocumentMapper("_doc", mapping); assertEquals(mapping, mapper.mappingSource().toString()); // special case: default index/search analyzer - mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("_doc") - .startObject("properties") - .startObject("field") - .field("type", "annotated_text") - .field("analyzer", "default") - .field("search_analyzer", "default") - .field("search_quote_analyzer", "keyword") - .endObject() - .endObject() - .endObject() - .endObject() - ); + mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("_doc") + .startObject("properties") + .startObject("field") + .field("type", "annotated_text") + .field("analyzer", "default") + .field("search_analyzer", "default") + .field("search_quote_analyzer", "keyword") + .endObject() + .endObject() + .endObject() + .endObject() + .toString(); mapper = createDocumentMapper("_doc", mapping); assertEquals(mapping, mapper.mappingSource().toString()); diff --git a/plugins/mapper-size/src/internalClusterTest/java/org/opensearch/index/mapper/size/SizeMappingTests.java b/plugins/mapper-size/src/internalClusterTest/java/org/opensearch/index/mapper/size/SizeMappingTests.java index 834c8a448d3d5..87b1624cbcd64 100644 --- a/plugins/mapper-size/src/internalClusterTest/java/org/opensearch/index/mapper/size/SizeMappingTests.java +++ b/plugins/mapper-size/src/internalClusterTest/java/org/opensearch/index/mapper/size/SizeMappingTests.java @@ -34,7 +34,6 @@ import java.util.Collection; -import org.opensearch.common.Strings; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.common.compress.CompressedXContent; import org.opensearch.common.settings.Settings; @@ -103,16 +102,15 @@ public void testThatDisablingWorksWhenMerging() throws Exception { DocumentMapper docMapper = service.mapperService().documentMapper(); assertThat(docMapper.metadataMapper(SizeFieldMapper.class).enabled(), is(true)); - String disabledMapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("type") - .startObject("_size") - .field("enabled", false) - .endObject() - .endObject() - .endObject() - ); + String disabledMapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("type") + .startObject("_size") + .field("enabled", false) + .endObject() + .endObject() + .endObject() + .toString(); docMapper = service.mapperService() .merge("type", new CompressedXContent(disabledMapping), MapperService.MergeReason.MAPPING_UPDATE); diff --git a/qa/ccs-unavailable-clusters/src/test/java/org/opensearch/search/CrossClusterSearchUnavailableClusterIT.java b/qa/ccs-unavailable-clusters/src/test/java/org/opensearch/search/CrossClusterSearchUnavailableClusterIT.java index 07cd901449a18..6b09d5477e8d2 100644 --- a/qa/ccs-unavailable-clusters/src/test/java/org/opensearch/search/CrossClusterSearchUnavailableClusterIT.java +++ b/qa/ccs-unavailable-clusters/src/test/java/org/opensearch/search/CrossClusterSearchUnavailableClusterIT.java @@ -61,7 +61,6 @@ import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodes; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.common.xcontent.json.JsonXContent; @@ -341,7 +340,7 @@ private static HttpEntity buildUpdateSettingsRequestBody(Map set builder.endObject(); } builder.endObject(); - requestBody = Strings.toString(builder); + requestBody = builder.toString(); } return new StringEntity(requestBody, ContentType.APPLICATION_JSON); } diff --git a/qa/full-cluster-restart/src/test/java/org/opensearch/upgrades/FullClusterRestartIT.java b/qa/full-cluster-restart/src/test/java/org/opensearch/upgrades/FullClusterRestartIT.java index 5fef24e75d8b7..95bc7f8dc404e 100644 --- a/qa/full-cluster-restart/src/test/java/org/opensearch/upgrades/FullClusterRestartIT.java +++ b/qa/full-cluster-restart/src/test/java/org/opensearch/upgrades/FullClusterRestartIT.java @@ -41,11 +41,11 @@ import org.opensearch.cluster.metadata.MetadataIndexStateService; import org.opensearch.common.Booleans; import org.opensearch.common.CheckedFunction; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; import org.opensearch.common.xcontent.XContentType; import org.opensearch.common.xcontent.json.JsonXContent; import org.opensearch.common.xcontent.support.XContentMapValues; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.index.IndexSettings; import org.opensearch.test.NotEqualMessageBuilder; @@ -149,7 +149,7 @@ public void testSearch() throws Exception { mappingsAndSettings.endObject(); Request createIndex = new Request("PUT", "/" + index); - createIndex.setJsonEntity(Strings.toString(mappingsAndSettings)); + createIndex.setJsonEntity(mappingsAndSettings.toString()); createIndex.setOptions(allowTypesRemovalWarnings()); client().performRequest(createIndex); @@ -208,7 +208,7 @@ public void testNewReplicasWork() throws Exception { mappingsAndSettings.endObject(); Request createIndex = new Request("PUT", "/" + index); - createIndex.setJsonEntity(Strings.toString(mappingsAndSettings)); + createIndex.setJsonEntity(mappingsAndSettings.toString()); client().performRequest(createIndex); int numDocs = randomIntBetween(2000, 3000); @@ -257,7 +257,7 @@ public void testClusterState() throws Exception { } mappingsAndSettings.endObject(); Request createTemplate = new Request("PUT", "/_template/template_1"); - createTemplate.setJsonEntity(Strings.toString(mappingsAndSettings)); + createTemplate.setJsonEntity(mappingsAndSettings.toString()); client().performRequest(createTemplate); client().performRequest(new Request("PUT", "/" + index)); } @@ -315,7 +315,7 @@ public void testShrink() throws IOException, NumberFormatException, ParseExcepti mappingsAndSettings.endObject(); Request createIndex = new Request("PUT", "/" + index); - createIndex.setJsonEntity(Strings.toString(mappingsAndSettings)); + createIndex.setJsonEntity(mappingsAndSettings.toString()); client().performRequest(createIndex); numDocs = randomIntBetween(512, 1024); @@ -384,7 +384,7 @@ public void testShrinkAfterUpgrade() throws IOException, ParseException { mappingsAndSettings.endObject(); Request createIndex = new Request("PUT", "/" + index); - createIndex.setJsonEntity(Strings.toString(mappingsAndSettings)); + createIndex.setJsonEntity(mappingsAndSettings.toString()); client().performRequest(createIndex); numDocs = randomIntBetween(512, 1024); @@ -854,7 +854,7 @@ public void testSnapshotRestore() throws IOException, ParseException { } templateBuilder.endObject().endObject(); Request createTemplateRequest = new Request("PUT", "/_template/test_template"); - createTemplateRequest.setJsonEntity(Strings.toString(templateBuilder)); + createTemplateRequest.setJsonEntity(templateBuilder.toString()); client().performRequest(createTemplateRequest); @@ -870,7 +870,7 @@ public void testSnapshotRestore() throws IOException, ParseException { } repoConfig.endObject(); Request createRepoRequest = new Request("PUT", "/_snapshot/repo"); - createRepoRequest.setJsonEntity(Strings.toString(repoConfig)); + createRepoRequest.setJsonEntity(repoConfig.toString()); client().performRequest(createRepoRequest); } @@ -897,7 +897,7 @@ public void testHistoryUUIDIsAdded() throws Exception { } mappingsAndSettings.endObject(); Request createIndex = new Request("PUT", "/" + index); - createIndex.setJsonEntity(Strings.toString(mappingsAndSettings)); + createIndex.setJsonEntity(mappingsAndSettings.toString()); client().performRequest(createIndex); } else { ensureGreenLongWait(index); @@ -940,11 +940,11 @@ public void testSoftDeletes() throws Exception { } mappingsAndSettings.endObject(); Request createIndex = new Request("PUT", "/" + index); - createIndex.setJsonEntity(Strings.toString(mappingsAndSettings)); + createIndex.setJsonEntity(mappingsAndSettings.toString()); client().performRequest(createIndex); int numDocs = between(10, 100); for (int i = 0; i < numDocs; i++) { - String doc = Strings.toString(JsonXContent.contentBuilder().startObject().field("field", "v1").endObject()); + String doc = JsonXContent.contentBuilder().startObject().field("field", "v1").endObject().toString(); Request request = new Request("POST", "/" + index + "/_doc/" + i); request.setJsonEntity(doc); client().performRequest(request); @@ -955,7 +955,7 @@ public void testSoftDeletes() throws Exception { assertTotalHits(liveDocs, entityAsMap(client().performRequest(new Request("GET", "/" + index + "/_search")))); for (int i = 0; i < numDocs; i++) { if (randomBoolean()) { - String doc = Strings.toString(JsonXContent.contentBuilder().startObject().field("field", "v2").endObject()); + String doc = JsonXContent.contentBuilder().startObject().field("field", "v2").endObject().toString(); Request request = new Request("POST", "/" + index + "/_doc/" + i); request.setJsonEntity(doc); client().performRequest(request); @@ -989,7 +989,7 @@ public void testClosedIndices() throws Exception { numDocs = between(1, 100); for (int i = 0; i < numDocs; i++) { final Request request = new Request("POST", "/" + index + "/" + type + "/" + i); - request.setJsonEntity(Strings.toString(JsonXContent.contentBuilder().startObject().field("field", "v1").endObject())); + request.setJsonEntity(JsonXContent.contentBuilder().startObject().field("field", "v1").endObject().toString()); assertOK(client().performRequest(request)); if (rarely()) { refresh(); @@ -1083,7 +1083,7 @@ private void checkSnapshot(final String snapshotName, final int count, final Ver restoreCommand.endObject(); Request restoreRequest = new Request("POST", "/_snapshot/repo/" + snapshotName + "/_restore"); restoreRequest.addParameter("wait_for_completion", "true"); - restoreRequest.setJsonEntity(Strings.toString(restoreCommand)); + restoreRequest.setJsonEntity(restoreCommand.toString()); client().performRequest(restoreRequest); // Make sure search finds all documents @@ -1158,7 +1158,7 @@ private void indexRandomDocuments( for (int i = 0; i < count; i++) { logger.debug("Indexing document [{}]", i); Request createDocument = new Request("POST", "/" + index + "/_doc/" + (specifyId ? i : "")); - createDocument.setJsonEntity(Strings.toString(docSupplier.apply(i))); + createDocument.setJsonEntity(docSupplier.apply(i).toString()); client().performRequest(createDocument); if (rarely()) { refreshAllIndices(); @@ -1175,7 +1175,7 @@ private void indexRandomDocuments( private void indexDocument(String id) throws IOException { final Request indexRequest = new Request("POST", "/" + index + "/" + type + "/" + id); - indexRequest.setJsonEntity(Strings.toString(JsonXContent.contentBuilder().startObject().field("f", "v").endObject())); + indexRequest.setJsonEntity(JsonXContent.contentBuilder().startObject().field("f", "v").endObject().toString()); assertOK(client().performRequest(indexRequest)); } @@ -1190,7 +1190,7 @@ private void saveInfoDocument(String id, String value) throws IOException { // Only create the first version so we know how many documents are created when the index is first created Request request = new Request("PUT", "/info/" + type + "/" + id); request.addParameter("op_type", "create"); - request.setJsonEntity(Strings.toString(infoDoc)); + request.setJsonEntity(infoDoc.toString()); client().performRequest(request); } @@ -1255,7 +1255,7 @@ public void testPeerRecoveryRetentionLeases() throws Exception { settings.endObject(); Request createIndex = new Request("PUT", "/" + index); - createIndex.setJsonEntity(Strings.toString(settings)); + createIndex.setJsonEntity(settings.toString()); client().performRequest(createIndex); } ensureGreen(index); @@ -1485,7 +1485,7 @@ public void testEnableSoftDeletesOnRestore() throws Exception { } repoConfig.endObject(); Request createRepoRequest = new Request("PUT", "/_snapshot/repo"); - createRepoRequest.setJsonEntity(Strings.toString(repoConfig)); + createRepoRequest.setJsonEntity(repoConfig.toString()); client().performRequest(createRepoRequest); // create snapshot Request createSnapshot = new Request("PUT", "/_snapshot/repo/" + snapshot); @@ -1507,7 +1507,7 @@ public void testEnableSoftDeletesOnRestore() throws Exception { restoreCommand.endObject(); Request restoreRequest = new Request("POST", "/_snapshot/repo/" + snapshot + "/_restore"); restoreRequest.addParameter("wait_for_completion", "true"); - restoreRequest.setJsonEntity(Strings.toString(restoreCommand)); + restoreRequest.setJsonEntity(restoreCommand.toString()); client().performRequest(restoreRequest); ensureGreen(restoredIndex); int numDocs = countOfIndexedRandomDocuments(); @@ -1539,7 +1539,7 @@ public void testForbidDisableSoftDeletesOnRestore() throws Exception { } repoConfig.endObject(); Request createRepoRequest = new Request("PUT", "/_snapshot/repo"); - createRepoRequest.setJsonEntity(Strings.toString(repoConfig)); + createRepoRequest.setJsonEntity(repoConfig.toString()); client().performRequest(createRepoRequest); // create snapshot Request createSnapshot = new Request("PUT", "/_snapshot/repo/" + snapshot); @@ -1560,7 +1560,7 @@ public void testForbidDisableSoftDeletesOnRestore() throws Exception { restoreCommand.endObject(); Request restoreRequest = new Request("POST", "/_snapshot/repo/" + snapshot + "/_restore"); restoreRequest.addParameter("wait_for_completion", "true"); - restoreRequest.setJsonEntity(Strings.toString(restoreCommand)); + restoreRequest.setJsonEntity(restoreCommand.toString()); final ResponseException error = expectThrows(ResponseException.class, () -> client().performRequest(restoreRequest)); assertThat(error.getMessage(), containsString("cannot disable setting [index.soft_deletes.enabled] on restore")); } diff --git a/qa/full-cluster-restart/src/test/java/org/opensearch/upgrades/QueryBuilderBWCIT.java b/qa/full-cluster-restart/src/test/java/org/opensearch/upgrades/QueryBuilderBWCIT.java index 724ac9883efaa..aabc3aee8887f 100644 --- a/qa/full-cluster-restart/src/test/java/org/opensearch/upgrades/QueryBuilderBWCIT.java +++ b/qa/full-cluster-restart/src/test/java/org/opensearch/upgrades/QueryBuilderBWCIT.java @@ -36,7 +36,6 @@ import org.apache.hc.core5.http.io.entity.EntityUtils; import org.opensearch.client.Request; import org.opensearch.client.Response; -import org.opensearch.common.Strings; import org.opensearch.core.common.io.stream.InputStreamStreamInput; import org.opensearch.core.common.io.stream.NamedWriteableAwareStreamInput; import org.opensearch.core.common.io.stream.NamedWriteableRegistry; @@ -199,7 +198,7 @@ public void testQueryBuilderBWC() throws Exception { mappingsAndSettings.endObject(); Request request = new Request("PUT", "/" + index); request.setOptions(allowTypesRemovalWarnings()); - request.setJsonEntity(Strings.toString(mappingsAndSettings)); + request.setJsonEntity(mappingsAndSettings.toString()); Response rsp = client().performRequest(request); assertEquals(200, rsp.getStatusLine().getStatusCode()); diff --git a/qa/mixed-cluster/src/test/java/org/opensearch/backwards/IndexingIT.java b/qa/mixed-cluster/src/test/java/org/opensearch/backwards/IndexingIT.java index f9810e027bb1e..75f7e00f499c6 100644 --- a/qa/mixed-cluster/src/test/java/org/opensearch/backwards/IndexingIT.java +++ b/qa/mixed-cluster/src/test/java/org/opensearch/backwards/IndexingIT.java @@ -42,7 +42,6 @@ import org.opensearch.client.ResponseException; import org.opensearch.client.RestClient; import org.opensearch.cluster.metadata.IndexMetadata; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; import org.opensearch.common.xcontent.json.JsonXContent; import org.opensearch.common.xcontent.support.XContentMapValues; @@ -365,15 +364,14 @@ public void testUpdateSnapshotStatus() throws Exception { // Create the repository before taking the snapshot. Request request = new Request("PUT", "/_snapshot/repo"); - request.setJsonEntity(Strings - .toString(JsonXContent.contentBuilder() + request.setJsonEntity(JsonXContent.contentBuilder() .startObject() .field("type", "fs") .startObject("settings") .field("compress", randomBoolean()) .field("location", System.getProperty("tests.path.repo")) .endObject() - .endObject())); + .endObject().toString()); assertOK(client().performRequest(request)); diff --git a/qa/multi-cluster-search/src/test/java/org/opensearch/search/CCSDuelIT.java b/qa/multi-cluster-search/src/test/java/org/opensearch/search/CCSDuelIT.java index 5f52250c7fb0c..1f2409741a878 100644 --- a/qa/multi-cluster-search/src/test/java/org/opensearch/search/CCSDuelIT.java +++ b/qa/multi-cluster-search/src/test/java/org/opensearch/search/CCSDuelIT.java @@ -53,12 +53,12 @@ import org.opensearch.client.RestHighLevelClient; import org.opensearch.client.indices.CreateIndexRequest; import org.opensearch.client.indices.CreateIndexResponse; -import org.opensearch.common.Strings; -import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.common.settings.Settings; import org.opensearch.common.xcontent.XContentHelper; import org.opensearch.common.xcontent.XContentType; import org.opensearch.common.util.io.IOUtils; +import org.opensearch.core.common.Strings; +import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.index.query.InnerHitBuilder; import org.opensearch.index.query.MatchQueryBuilder; import org.opensearch.index.query.QueryBuilders; diff --git a/qa/rolling-upgrade/src/test/java/org/opensearch/upgrades/RecoveryIT.java b/qa/rolling-upgrade/src/test/java/org/opensearch/upgrades/RecoveryIT.java index 3dd9f371f06fd..6d143d08452e9 100644 --- a/qa/rolling-upgrade/src/test/java/org/opensearch/upgrades/RecoveryIT.java +++ b/qa/rolling-upgrade/src/test/java/org/opensearch/upgrades/RecoveryIT.java @@ -40,11 +40,11 @@ import org.opensearch.cluster.metadata.MetadataIndexStateService; import org.opensearch.cluster.routing.allocation.decider.EnableAllocationDecider; import org.opensearch.common.Booleans; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.concurrent.AbstractRunnable; import org.opensearch.common.xcontent.XContentType; import org.opensearch.common.xcontent.support.XContentMapValues; +import org.opensearch.core.common.Strings; import org.opensearch.index.IndexSettings; import org.opensearch.index.mapper.MapperService; import org.opensearch.core.rest.RestStatus; diff --git a/qa/smoke-test-http/src/test/java/org/opensearch/http/SearchRestCancellationIT.java b/qa/smoke-test-http/src/test/java/org/opensearch/http/SearchRestCancellationIT.java index 4a898d816bbf4..b1143ad647327 100644 --- a/qa/smoke-test-http/src/test/java/org/opensearch/http/SearchRestCancellationIT.java +++ b/qa/smoke-test-http/src/test/java/org/opensearch/http/SearchRestCancellationIT.java @@ -48,8 +48,8 @@ import org.opensearch.client.Response; import org.opensearch.client.ResponseListener; import org.opensearch.common.SetOnce; -import org.opensearch.common.Strings; import org.opensearch.common.xcontent.XContentType; +import org.opensearch.core.common.Strings; import org.opensearch.plugins.Plugin; import org.opensearch.plugins.PluginsService; import org.opensearch.script.MockScriptPlugin; @@ -83,7 +83,6 @@ import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertNoFailures; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.instanceOf; -import static org.opensearch.test.hamcrest.OpenSearchAssertions.awaitLatch; public class SearchRestCancellationIT extends HttpSmokeTestCase { diff --git a/qa/translog-policy/src/test/java/org/opensearch/upgrades/TranslogPolicyIT.java b/qa/translog-policy/src/test/java/org/opensearch/upgrades/TranslogPolicyIT.java index 0dc62b160ff3f..5f0f468898c47 100644 --- a/qa/translog-policy/src/test/java/org/opensearch/upgrades/TranslogPolicyIT.java +++ b/qa/translog-policy/src/test/java/org/opensearch/upgrades/TranslogPolicyIT.java @@ -35,7 +35,7 @@ import org.opensearch.LegacyESVersion; import org.opensearch.client.Request; import org.opensearch.cluster.metadata.IndexMetadata; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.common.settings.Settings; import org.opensearch.common.xcontent.json.JsonXContent; import org.opensearch.index.IndexSettings; diff --git a/server/src/internalClusterTest/java/org/opensearch/action/admin/cluster/allocation/ClusterAllocationExplainIT.java b/server/src/internalClusterTest/java/org/opensearch/action/admin/cluster/allocation/ClusterAllocationExplainIT.java index 4cbcc5b9bb507..966c040d7877c 100644 --- a/server/src/internalClusterTest/java/org/opensearch/action/admin/cluster/allocation/ClusterAllocationExplainIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/action/admin/cluster/allocation/ClusterAllocationExplainIT.java @@ -49,7 +49,6 @@ import org.opensearch.cluster.routing.allocation.NodeAllocationResult; import org.opensearch.cluster.routing.allocation.decider.Decision; import org.opensearch.common.Priority; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.set.Sets; @@ -1275,7 +1274,7 @@ private ClusterAllocationExplanation runExplain(boolean primary, String nodeId, XContentBuilder builder = JsonXContent.contentBuilder(); builder.prettyPrint(); builder.humanReadable(true); - logger.debug("--> explain json output: \n{}", Strings.toString(explanation.toXContent(builder, ToXContent.EMPTY_PARAMS))); + logger.debug("--> explain json output: \n{}", explanation.toXContent(builder, ToXContent.EMPTY_PARAMS).toString()); } return explanation; } diff --git a/server/src/internalClusterTest/java/org/opensearch/action/bulk/BulkWithUpdatesIT.java b/server/src/internalClusterTest/java/org/opensearch/action/bulk/BulkWithUpdatesIT.java index 53afa53de92f3..139b2cb896ded 100644 --- a/server/src/internalClusterTest/java/org/opensearch/action/bulk/BulkWithUpdatesIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/action/bulk/BulkWithUpdatesIT.java @@ -46,9 +46,9 @@ import org.opensearch.action.update.UpdateResponse; import org.opensearch.client.Requests; import org.opensearch.cluster.metadata.IndexMetadata; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; import org.opensearch.common.xcontent.XContentType; +import org.opensearch.core.common.Strings; import org.opensearch.index.VersionType; import org.opensearch.indices.IndexClosedException; import org.opensearch.plugins.Plugin; diff --git a/server/src/internalClusterTest/java/org/opensearch/cluster/routing/PrimaryAllocationIT.java b/server/src/internalClusterTest/java/org/opensearch/cluster/routing/PrimaryAllocationIT.java index 637cc96bdfc44..14b6ffcd50825 100644 --- a/server/src/internalClusterTest/java/org/opensearch/cluster/routing/PrimaryAllocationIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/cluster/routing/PrimaryAllocationIT.java @@ -46,10 +46,10 @@ import org.opensearch.cluster.routing.allocation.command.AllocateEmptyPrimaryAllocationCommand; import org.opensearch.cluster.routing.allocation.command.AllocateStalePrimaryAllocationCommand; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.set.Sets; import org.opensearch.common.xcontent.XContentType; +import org.opensearch.core.common.Strings; import org.opensearch.gateway.GatewayAllocator; import org.opensearch.index.IndexNotFoundException; import org.opensearch.index.engine.Engine; diff --git a/server/src/internalClusterTest/java/org/opensearch/document/DocumentActionsIT.java b/server/src/internalClusterTest/java/org/opensearch/document/DocumentActionsIT.java index 581b352e917f0..10e6aa906ecc9 100644 --- a/server/src/internalClusterTest/java/org/opensearch/document/DocumentActionsIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/document/DocumentActionsIT.java @@ -43,7 +43,6 @@ import org.opensearch.action.search.SearchResponse; import org.opensearch.action.support.WriteRequest.RefreshPolicy; import org.opensearch.cluster.health.ClusterHealthStatus; -import org.opensearch.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.common.xcontent.XContentFactory; import org.opensearch.common.xcontent.XContentType; @@ -117,10 +116,10 @@ public void testIndexActions() throws Exception { for (int i = 0; i < 5; i++) { getResult = client().prepareGet("test", "1").execute().actionGet(); assertThat(getResult.getIndex(), equalTo(getConcreteIndexName())); - assertThat("cycle #" + i, getResult.getSourceAsString(), equalTo(Strings.toString(source("1", "test")))); + assertThat("cycle #" + i, getResult.getSourceAsString(), equalTo(source("1", "test").toString())); assertThat("cycle(map) #" + i, (String) getResult.getSourceAsMap().get("name"), equalTo("test")); getResult = client().get(getRequest("test").id("1")).actionGet(); - assertThat("cycle #" + i, getResult.getSourceAsString(), equalTo(Strings.toString(source("1", "test")))); + assertThat("cycle #" + i, getResult.getSourceAsString(), equalTo(source("1", "test").toString())); assertThat(getResult.getIndex(), equalTo(getConcreteIndexName())); } @@ -168,10 +167,10 @@ public void testIndexActions() throws Exception { for (int i = 0; i < 5; i++) { getResult = client().get(getRequest("test").id("1")).actionGet(); assertThat(getResult.getIndex(), equalTo(getConcreteIndexName())); - assertThat("cycle #" + i, getResult.getSourceAsString(), equalTo(Strings.toString(source("1", "test")))); + assertThat("cycle #" + i, getResult.getSourceAsString(), equalTo(source("1", "test").toString())); getResult = client().get(getRequest("test").id("2")).actionGet(); String ste1 = getResult.getSourceAsString(); - String ste2 = Strings.toString(source("2", "test2")); + String ste2 = source("2", "test2").toString(); assertThat("cycle #" + i, ste1, equalTo(ste2)); assertThat(getResult.getIndex(), equalTo(getConcreteIndexName())); } @@ -258,15 +257,15 @@ public void testBulk() throws Exception { assertThat("cycle #" + i, getResult.isExists(), equalTo(false)); getResult = client().get(getRequest("test").id("2")).actionGet(); - assertThat("cycle #" + i, getResult.getSourceAsString(), equalTo(Strings.toString(source("2", "test")))); + assertThat("cycle #" + i, getResult.getSourceAsString(), equalTo(source("2", "test").toString())); assertThat(getResult.getIndex(), equalTo(getConcreteIndexName())); getResult = client().get(getRequest("test").id(generatedId3)).actionGet(); - assertThat("cycle #" + i, getResult.getSourceAsString(), equalTo(Strings.toString(source("3", "test")))); + assertThat("cycle #" + i, getResult.getSourceAsString(), equalTo(source("3", "test").toString())); assertThat(getResult.getIndex(), equalTo(getConcreteIndexName())); getResult = client().get(getRequest("test").id(generatedId4)).actionGet(); - assertThat("cycle #" + i, getResult.getSourceAsString(), equalTo(Strings.toString(source("4", "test")))); + assertThat("cycle #" + i, getResult.getSourceAsString(), equalTo(source("4", "test").toString())); assertThat(getResult.getIndex(), equalTo(getConcreteIndexName())); } } diff --git a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java index 0b3a689e81b94..c88bf942fa8d0 100644 --- a/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/gateway/RecoveryFromGatewayIT.java @@ -46,7 +46,6 @@ import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.routing.UnassignedInfo; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; import org.opensearch.common.xcontent.XContentFactory; import org.opensearch.env.NodeEnvironment; @@ -110,16 +109,15 @@ public void testOneNodeRecoverFromGateway() throws Exception { internalCluster().startNode(); - String mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject("appAccountIds") - .field("type", "text") - .endObject() - .endObject() - .endObject() - ); + String mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject("appAccountIds") + .field("type", "text") + .endObject() + .endObject() + .endObject() + .toString(); assertAcked(prepareCreate("test").setMapping(mapping)); client().prepareIndex("test") @@ -204,19 +202,18 @@ private Map assertAndCapturePrimaryTerms(Map pre public void testSingleNodeNoFlush() throws Exception { internalCluster().startNode(); - String mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject("field") - .field("type", "text") - .endObject() - .startObject("num") - .field("type", "integer") - .endObject() - .endObject() - .endObject() - ); + String mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject("field") + .field("type", "text") + .endObject() + .startObject("num") + .field("type", "integer") + .endObject() + .endObject() + .endObject() + .toString(); // note: default replica settings are tied to #data nodes-1 which is 0 here. We can do with 1 in this test. int numberOfShards = numberOfShards(); assertAcked( diff --git a/server/src/internalClusterTest/java/org/opensearch/get/GetActionIT.java b/server/src/internalClusterTest/java/org/opensearch/get/GetActionIT.java index 997e8e9d5258b..2375c62342533 100644 --- a/server/src/internalClusterTest/java/org/opensearch/get/GetActionIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/get/GetActionIT.java @@ -45,12 +45,12 @@ import org.opensearch.action.index.IndexResponse; import org.opensearch.core.action.support.DefaultShardOperationFailedException; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; -import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.common.lucene.uid.Versions; import org.opensearch.common.settings.Settings; import org.opensearch.common.xcontent.XContentFactory; import org.opensearch.common.xcontent.XContentType; +import org.opensearch.core.common.Strings; +import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.index.engine.VersionConflictEngineException; import org.opensearch.plugins.Plugin; import org.opensearch.core.rest.RestStatus; @@ -288,17 +288,16 @@ public void testSimpleMultiGet() throws Exception { } public void testGetDocWithMultivaluedFields() throws Exception { - String mapping1 = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject("field") - .field("type", "text") - .field("store", true) - .endObject() - .endObject() - .endObject() - ); + String mapping1 = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject("field") + .field("type", "text") + .field("store", true) + .endObject() + .endObject() + .endObject() + .toString(); assertAcked(prepareCreate("test").setMapping(mapping1)); ensureGreen(); diff --git a/server/src/internalClusterTest/java/org/opensearch/index/mapper/CopyToMapperIntegrationIT.java b/server/src/internalClusterTest/java/org/opensearch/index/mapper/CopyToMapperIntegrationIT.java index f4ccea40e6e3f..a25b6049e92a0 100644 --- a/server/src/internalClusterTest/java/org/opensearch/index/mapper/CopyToMapperIntegrationIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/index/mapper/CopyToMapperIntegrationIT.java @@ -33,7 +33,6 @@ package org.opensearch.index.mapper; import org.opensearch.action.search.SearchResponse; -import org.opensearch.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.common.xcontent.XContentFactory; import org.opensearch.index.query.QueryBuilders; @@ -78,16 +77,15 @@ public void testDynamicTemplateCopyTo() throws Exception { } public void testDynamicObjectCopyTo() throws Exception { - String mapping = Strings.toString( - jsonBuilder().startObject() - .startObject("properties") - .startObject("foo") - .field("type", "text") - .field("copy_to", "root.top.child") - .endObject() - .endObject() - .endObject() - ); + String mapping = jsonBuilder().startObject() + .startObject("properties") + .startObject("foo") + .field("type", "text") + .field("copy_to", "root.top.child") + .endObject() + .endObject() + .endObject() + .toString(); assertAcked(client().admin().indices().prepareCreate("test-idx").setMapping(mapping)); client().prepareIndex("test-idx").setId("1").setSource("foo", "bar").get(); client().admin().indices().prepareRefresh("test-idx").execute().actionGet(); diff --git a/server/src/internalClusterTest/java/org/opensearch/index/shard/IndexShardIT.java b/server/src/internalClusterTest/java/org/opensearch/index/shard/IndexShardIT.java index 4b95c2fc4bce5..d9eeb3f7f8f42 100644 --- a/server/src/internalClusterTest/java/org/opensearch/index/shard/IndexShardIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/index/shard/IndexShardIT.java @@ -53,8 +53,8 @@ import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.CheckedFunction; import org.opensearch.common.CheckedRunnable; -import org.opensearch.common.Strings; import org.opensearch.common.UUIDs; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.bytes.BytesArray; import org.opensearch.common.lucene.uid.Versions; import org.opensearch.common.settings.Settings; diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/mapping/SimpleGetFieldMappingsIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/mapping/SimpleGetFieldMappingsIT.java index 30c6a0dc068e5..1525c7bada9ac 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/mapping/SimpleGetFieldMappingsIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/mapping/SimpleGetFieldMappingsIT.java @@ -34,7 +34,6 @@ import org.opensearch.action.admin.indices.mapping.get.GetFieldMappingsResponse; import org.opensearch.action.admin.indices.mapping.get.GetFieldMappingsResponse.FieldMappingMetadata; -import org.opensearch.common.Strings; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.common.xcontent.XContentFactory; @@ -187,22 +186,22 @@ public void testSimpleGetFieldMappingsWithPretty() throws Exception { .get(); XContentBuilder responseBuilder = XContentFactory.jsonBuilder().prettyPrint(); response.toXContent(responseBuilder, new ToXContent.MapParams(params)); - String responseStrings = Strings.toString(responseBuilder); + String responseStrings = responseBuilder.toString(); XContentBuilder prettyJsonBuilder = XContentFactory.jsonBuilder().prettyPrint(); prettyJsonBuilder.copyCurrentStructure(createParser(JsonXContent.jsonXContent, responseStrings)); - assertThat(responseStrings, equalTo(Strings.toString(prettyJsonBuilder))); + assertThat(responseStrings, equalTo(prettyJsonBuilder.toString())); params.put("pretty", "false"); response = client().admin().indices().prepareGetFieldMappings("index").setFields("field1", "obj.subfield").get(); responseBuilder = XContentFactory.jsonBuilder().prettyPrint().lfAtEnd(); response.toXContent(responseBuilder, new ToXContent.MapParams(params)); - responseStrings = Strings.toString(responseBuilder); + responseStrings = responseBuilder.toString(); prettyJsonBuilder = XContentFactory.jsonBuilder().prettyPrint(); prettyJsonBuilder.copyCurrentStructure(createParser(JsonXContent.jsonXContent, responseStrings)); - assertThat(responseStrings, not(equalTo(Strings.toString(prettyJsonBuilder)))); + assertThat(responseStrings, not(equalTo(prettyJsonBuilder).toString())); } diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/memory/breaker/RandomExceptionCircuitBreakerIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/memory/breaker/RandomExceptionCircuitBreakerIT.java index 5ce9de7d0eae0..62efdc6a722ee 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/memory/breaker/RandomExceptionCircuitBreakerIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/memory/breaker/RandomExceptionCircuitBreakerIT.java @@ -43,7 +43,6 @@ import org.opensearch.action.admin.indices.refresh.RefreshResponse; import org.opensearch.action.search.SearchPhaseExecutionException; import org.opensearch.action.search.SearchRequestBuilder; -import org.opensearch.common.Strings; import org.opensearch.core.common.breaker.CircuitBreaker; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Setting.Property; @@ -99,22 +98,20 @@ public void testBreakerWithRandomExceptions() throws IOException, InterruptedExc assertThat("Breaker is not set to 0", node.getBreaker().getStats(CircuitBreaker.FIELDDATA).getEstimated(), equalTo(0L)); } - String mapping = Strings // {} - .toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject("test-str") - .field("type", "keyword") - .field("doc_values", randomBoolean()) - .endObject() // test-str - .startObject("test-num") - // I don't use randomNumericType() here because I don't want "byte", and I want "float" and "double" - .field("type", randomFrom(Arrays.asList("float", "long", "double", "short", "integer"))) - .endObject() // test-num - .endObject() // properties - .endObject() - ); + String mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject("test-str") + .field("type", "keyword") + .field("doc_values", randomBoolean()) + .endObject() // test-str + .startObject("test-num") + // I don't use randomNumericType() here because I don't want "byte", and I want "float" and "double" + .field("type", randomFrom(Arrays.asList("float", "long", "double", "short", "integer"))) + .endObject() // test-num + .endObject() // properties + .endObject() + .toString(); final double topLevelRate; final double lowLevelRate; if (frequently()) { diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/recovery/IndexRecoveryIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/recovery/IndexRecoveryIT.java index 850f08b8136c1..efd43ec5ad82d 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/recovery/IndexRecoveryIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/recovery/IndexRecoveryIT.java @@ -73,7 +73,7 @@ import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.Priority; import org.opensearch.common.SetOnce; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.breaker.CircuitBreaker; import org.opensearch.core.common.breaker.CircuitBreakingException; import org.opensearch.common.concurrent.GatedCloseable; diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/state/OpenCloseIndexIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/state/OpenCloseIndexIT.java index fde30f35d1b6d..3bbd4f83d2b3c 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/state/OpenCloseIndexIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/state/OpenCloseIndexIT.java @@ -44,7 +44,6 @@ import org.opensearch.action.support.master.AcknowledgedResponse; import org.opensearch.client.Client; import org.opensearch.cluster.metadata.IndexMetadata; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; import org.opensearch.common.xcontent.XContentFactory; import org.opensearch.index.IndexNotFoundException; @@ -301,16 +300,15 @@ public void testOpenWaitingForActiveShardsFailed() throws Exception { } public void testOpenCloseWithDocs() throws IOException, ExecutionException, InterruptedException { - String mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject("test") - .field("type", "keyword") - .endObject() - .endObject() - .endObject() - ); + String mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject("test") + .field("type", "keyword") + .endObject() + .endObject() + .endObject() + .toString(); assertAcked(client().admin().indices().prepareCreate("test").setMapping(mapping)); ensureGreen(); diff --git a/server/src/internalClusterTest/java/org/opensearch/search/SearchCancellationIT.java b/server/src/internalClusterTest/java/org/opensearch/search/SearchCancellationIT.java index 9db0ac4590efa..eedd9328826a5 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/SearchCancellationIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/SearchCancellationIT.java @@ -48,10 +48,10 @@ import org.opensearch.action.search.SearchScrollAction; import org.opensearch.action.search.ShardSearchFailure; import org.opensearch.action.support.WriteRequest; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.xcontent.XContentType; +import org.opensearch.core.common.Strings; import org.opensearch.plugins.Plugin; import org.opensearch.plugins.PluginsService; import org.opensearch.script.MockScriptPlugin; diff --git a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/DateHistogramIT.java b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/DateHistogramIT.java index 617c5745c9bba..271492e47d2ef 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/DateHistogramIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/DateHistogramIT.java @@ -35,7 +35,6 @@ import org.opensearch.action.index.IndexRequestBuilder; import org.opensearch.action.search.SearchPhaseExecutionException; import org.opensearch.action.search.SearchResponse; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; import org.opensearch.common.time.DateFormatter; import org.opensearch.common.time.DateFormatters; @@ -1309,16 +1308,15 @@ public void testSingleValueFieldWithExtendedBoundsOffset() throws Exception { } public void testSingleValueWithMultipleDateFormatsFromMapping() throws Exception { - String mappingJson = Strings.toString( - jsonBuilder().startObject() - .startObject("properties") - .startObject("date") - .field("type", "date") - .field("format", "strict_date_optional_time||dd-MM-yyyy") - .endObject() - .endObject() - .endObject() - ); + String mappingJson = jsonBuilder().startObject() + .startObject("properties") + .startObject("date") + .field("type", "date") + .field("format", "strict_date_optional_time||dd-MM-yyyy") + .endObject() + .endObject() + .endObject() + .toString(); prepareCreate("idx2").setMapping(mappingJson).get(); IndexRequestBuilder[] reqs = new IndexRequestBuilder[5]; for (int i = 0; i < reqs.length; i++) { diff --git a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/SignificantTermsSignificanceScoreIT.java b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/SignificantTermsSignificanceScoreIT.java index 43d49dc0bfd60..0854faf6c515c 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/SignificantTermsSignificanceScoreIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/SignificantTermsSignificanceScoreIT.java @@ -35,7 +35,6 @@ import org.opensearch.action.index.IndexRequestBuilder; import org.opensearch.action.search.SearchRequestBuilder; import org.opensearch.action.search.SearchResponse; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.XContentBuilder; @@ -208,7 +207,7 @@ public void testXContentResponse() throws Exception { + "\"score\":0.75," + "\"bg_count\":4" + "}]}}]}}"; - assertThat(Strings.toString(responseBuilder), equalTo(result)); + assertThat(responseBuilder.toString(), equalTo(result)); } diff --git a/server/src/internalClusterTest/java/org/opensearch/search/basic/SearchWithRandomExceptionsIT.java b/server/src/internalClusterTest/java/org/opensearch/search/basic/SearchWithRandomExceptionsIT.java index 0b55ea9119d89..b33adea494d17 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/basic/SearchWithRandomExceptionsIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/basic/SearchWithRandomExceptionsIT.java @@ -43,7 +43,6 @@ import org.opensearch.action.index.IndexResponse; import org.opensearch.action.search.SearchPhaseExecutionException; import org.opensearch.action.search.SearchResponse; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Setting.Property; import org.opensearch.common.settings.Settings; @@ -81,16 +80,15 @@ protected boolean addMockInternalEngine() { } public void testRandomExceptions() throws IOException, InterruptedException, ExecutionException { - String mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject("test") - .field("type", "keyword") - .endObject() - .endObject() - .endObject() - ); + String mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject("test") + .field("type", "keyword") + .endObject() + .endObject() + .endObject() + .toString(); final double lowLevelRate; final double topLevelRate; if (frequently()) { diff --git a/server/src/internalClusterTest/java/org/opensearch/search/basic/SearchWithRandomIOExceptionsIT.java b/server/src/internalClusterTest/java/org/opensearch/search/basic/SearchWithRandomIOExceptionsIT.java index b0adc00f37fee..7cd389e6274dc 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/basic/SearchWithRandomIOExceptionsIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/basic/SearchWithRandomIOExceptionsIT.java @@ -42,7 +42,6 @@ import org.opensearch.action.search.SearchPhaseExecutionException; import org.opensearch.action.search.SearchResponse; import org.opensearch.client.Requests; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.xcontent.XContentFactory; @@ -69,16 +68,15 @@ protected Collection> nodePlugins() { } public void testRandomDirectoryIOExceptions() throws IOException, InterruptedException, ExecutionException { - String mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject("test") - .field("type", "keyword") - .endObject() - .endObject() - .endObject() - ); + String mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject("test") + .field("type", "keyword") + .endObject() + .endObject() + .endObject() + .toString(); final double exceptionRate; final double exceptionOnOpenRate; if (frequently()) { diff --git a/server/src/internalClusterTest/java/org/opensearch/search/fetch/subphase/highlight/HighlighterSearchIT.java b/server/src/internalClusterTest/java/org/opensearch/search/fetch/subphase/highlight/HighlighterSearchIT.java index f2a22b99a86a3..bf1ca409eee92 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/fetch/subphase/highlight/HighlighterSearchIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/fetch/subphase/highlight/HighlighterSearchIT.java @@ -44,7 +44,6 @@ import org.opensearch.action.search.SearchRequestBuilder; import org.opensearch.action.search.SearchResponse; import org.opensearch.action.support.WriteRequest; -import org.opensearch.common.Strings; import org.opensearch.common.geo.GeoPoint; import org.opensearch.common.settings.Settings; import org.opensearch.common.settings.Settings.Builder; @@ -3231,26 +3230,25 @@ public void testCopyToFields() throws Exception { } public void testACopyFieldWithNestedQuery() throws Exception { - String mapping = Strings.toString( - jsonBuilder().startObject() - .startObject("properties") - .startObject("foo") - .field("type", "nested") - .startObject("properties") - .startObject("text") - .field("type", "text") - .field("copy_to", "foo_text") - .endObject() - .endObject() - .endObject() - .startObject("foo_text") - .field("type", "text") - .field("term_vector", "with_positions_offsets") - .field("store", true) - .endObject() - .endObject() - .endObject() - ); + String mapping = jsonBuilder().startObject() + .startObject("properties") + .startObject("foo") + .field("type", "nested") + .startObject("properties") + .startObject("text") + .field("type", "text") + .field("copy_to", "foo_text") + .endObject() + .endObject() + .endObject() + .startObject("foo_text") + .field("type", "text") + .field("term_vector", "with_positions_offsets") + .field("store", true) + .endObject() + .endObject() + .endObject() + .toString(); prepareCreate("test").setMapping(mapping).get(); client().prepareIndex("test") @@ -3361,25 +3359,24 @@ public void testHighlightQueryRewriteDatesWithNow() throws Exception { } public void testWithNestedQuery() throws Exception { - String mapping = Strings.toString( - jsonBuilder().startObject() - .startObject("properties") - .startObject("text") - .field("type", "text") - .field("index_options", "offsets") - .field("term_vector", "with_positions_offsets") - .endObject() - .startObject("foo") - .field("type", "nested") - .startObject("properties") - .startObject("text") - .field("type", "text") - .endObject() - .endObject() - .endObject() - .endObject() - .endObject() - ); + String mapping = jsonBuilder().startObject() + .startObject("properties") + .startObject("text") + .field("type", "text") + .field("index_options", "offsets") + .field("term_vector", "with_positions_offsets") + .endObject() + .startObject("foo") + .field("type", "nested") + .startObject("properties") + .startObject("text") + .field("type", "text") + .endObject() + .endObject() + .endObject() + .endObject() + .endObject() + .toString(); prepareCreate("test").setMapping(mapping).get(); client().prepareIndex("test") diff --git a/server/src/internalClusterTest/java/org/opensearch/search/fields/SearchFieldsIT.java b/server/src/internalClusterTest/java/org/opensearch/search/fields/SearchFieldsIT.java index 90d0a59f7b58d..53eb290e1edbf 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/fields/SearchFieldsIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/fields/SearchFieldsIT.java @@ -36,7 +36,6 @@ import org.opensearch.action.search.SearchRequestBuilder; import org.opensearch.action.search.SearchResponse; import org.opensearch.common.Numbers; -import org.opensearch.common.Strings; import org.opensearch.core.common.bytes.BytesArray; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.common.collect.MapBuilder; @@ -201,27 +200,26 @@ static Object docScript(Map vars, String fieldName) { public void testStoredFields() throws Exception { createIndex("test"); - String mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject(MapperService.SINGLE_MAPPING_NAME) - .startObject("properties") - .startObject("field1") - .field("type", "text") - .field("store", true) - .endObject() - .startObject("field2") - .field("type", "text") - .field("store", false) - .endObject() - .startObject("field3") - .field("type", "text") - .field("store", true) - .endObject() - .endObject() - .endObject() - .endObject() - ); + String mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject(MapperService.SINGLE_MAPPING_NAME) + .startObject("properties") + .startObject("field1") + .field("type", "text") + .field("store", true) + .endObject() + .startObject("field2") + .field("type", "text") + .field("store", false) + .endObject() + .startObject("field3") + .field("type", "text") + .field("store", true) + .endObject() + .endObject() + .endObject() + .endObject() + .toString(); client().admin().indices().preparePutMapping().setSource(mapping, XContentType.JSON).get(); @@ -304,19 +302,18 @@ public void testStoredFields() throws Exception { public void testScriptDocAndFields() throws Exception { createIndex("test"); - String mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject(MapperService.SINGLE_MAPPING_NAME) - .startObject("properties") - .startObject("num1") - .field("type", "double") - .field("store", true) - .endObject() - .endObject() - .endObject() - .endObject() - ); + String mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject(MapperService.SINGLE_MAPPING_NAME) + .startObject("properties") + .startObject("num1") + .field("type", "double") + .field("store", true) + .endObject() + .endObject() + .endObject() + .endObject() + .toString(); client().admin().indices().preparePutMapping().setSource(mapping, XContentType.JSON).get(); @@ -406,19 +403,18 @@ public void testScriptDocAndFields() throws Exception { public void testScriptWithUnsignedLong() throws Exception { createIndex("test"); - String mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject(MapperService.SINGLE_MAPPING_NAME) - .startObject("properties") - .startObject("unsigned_num1") - .field("type", "unsigned_long") - .field("store", true) - .endObject() - .endObject() - .endObject() - .endObject() - ); + String mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject(MapperService.SINGLE_MAPPING_NAME) + .startObject("properties") + .startObject("unsigned_num1") + .field("type", "unsigned_long") + .field("store", true) + .endObject() + .endObject() + .endObject() + .endObject() + .toString(); client().admin().indices().preparePutMapping().setSource(mapping, XContentType.JSON).get(); @@ -509,18 +505,17 @@ public void testScriptWithUnsignedLong() throws Exception { public void testScriptFieldWithNanos() throws Exception { createIndex("test"); - String mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject(MapperService.SINGLE_MAPPING_NAME) - .startObject("properties") - .startObject("date") - .field("type", "date_nanos") - .endObject() - .endObject() - .endObject() - .endObject() - ); + String mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject(MapperService.SINGLE_MAPPING_NAME) + .startObject("properties") + .startObject("date") + .field("type", "date_nanos") + .endObject() + .endObject() + .endObject() + .endObject() + .toString(); client().admin().indices().preparePutMapping().setSource(mapping, XContentType.JSON).get(); String date = "2019-01-31T10:00:00.123456789Z"; @@ -706,58 +701,57 @@ public void testPartialFields() throws Exception { public void testStoredFieldsWithoutSource() throws Exception { createIndex("test"); - String mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject(MapperService.SINGLE_MAPPING_NAME) - .startObject("_source") - .field("enabled", false) - .endObject() - .startObject("properties") - .startObject("byte_field") - .field("type", "byte") - .field("store", true) - .endObject() - .startObject("short_field") - .field("type", "short") - .field("store", true) - .endObject() - .startObject("integer_field") - .field("type", "integer") - .field("store", true) - .endObject() - .startObject("long_field") - .field("type", "long") - .field("store", true) - .endObject() - .startObject("float_field") - .field("type", "float") - .field("store", true) - .endObject() - .startObject("double_field") - .field("type", "double") - .field("store", true) - .endObject() - .startObject("date_field") - .field("type", "date") - .field("store", true) - .endObject() - .startObject("boolean_field") - .field("type", "boolean") - .field("store", true) - .endObject() - .startObject("binary_field") - .field("type", "binary") - .field("store", true) - .endObject() - .startObject("unsigned_long_field") - .field("type", "unsigned_long") - .field("store", true) - .endObject() - .endObject() - .endObject() - .endObject() - ); + String mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject(MapperService.SINGLE_MAPPING_NAME) + .startObject("_source") + .field("enabled", false) + .endObject() + .startObject("properties") + .startObject("byte_field") + .field("type", "byte") + .field("store", true) + .endObject() + .startObject("short_field") + .field("type", "short") + .field("store", true) + .endObject() + .startObject("integer_field") + .field("type", "integer") + .field("store", true) + .endObject() + .startObject("long_field") + .field("type", "long") + .field("store", true) + .endObject() + .startObject("float_field") + .field("type", "float") + .field("store", true) + .endObject() + .startObject("double_field") + .field("type", "double") + .field("store", true) + .endObject() + .startObject("date_field") + .field("type", "date") + .field("store", true) + .endObject() + .startObject("boolean_field") + .field("type", "boolean") + .field("store", true) + .endObject() + .startObject("binary_field") + .field("type", "binary") + .field("store", true) + .endObject() + .startObject("unsigned_long_field") + .field("type", "unsigned_long") + .field("store", true) + .endObject() + .endObject() + .endObject() + .endObject() + .toString(); client().admin().indices().preparePutMapping().setSource(mapping, XContentType.JSON).get(); @@ -944,59 +938,58 @@ public void testSingleValueFieldDatatField() throws ExecutionException, Interrup public void testDocValueFields() throws Exception { createIndex("test"); - String mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject(MapperService.SINGLE_MAPPING_NAME) - .startObject("_source") - .field("enabled", false) - .endObject() - .startObject("properties") - .startObject("text_field") - .field("type", "text") - .field("fielddata", true) - .endObject() - .startObject("keyword_field") - .field("type", "keyword") - .endObject() - .startObject("byte_field") - .field("type", "byte") - .endObject() - .startObject("short_field") - .field("type", "short") - .endObject() - .startObject("integer_field") - .field("type", "integer") - .endObject() - .startObject("long_field") - .field("type", "long") - .endObject() - .startObject("float_field") - .field("type", "float") - .endObject() - .startObject("double_field") - .field("type", "double") - .endObject() - .startObject("date_field") - .field("type", "date") - .endObject() - .startObject("boolean_field") - .field("type", "boolean") - .endObject() - .startObject("binary_field") - .field("type", "binary") - .field("doc_values", true) // off by default on binary fields - .endObject() - .startObject("ip_field") - .field("type", "ip") - .endObject() - .startObject("flat_object_field") - .field("type", "flat_object") - .endObject() - .endObject() - .endObject() - .endObject() - ); + String mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject(MapperService.SINGLE_MAPPING_NAME) + .startObject("_source") + .field("enabled", false) + .endObject() + .startObject("properties") + .startObject("text_field") + .field("type", "text") + .field("fielddata", true) + .endObject() + .startObject("keyword_field") + .field("type", "keyword") + .endObject() + .startObject("byte_field") + .field("type", "byte") + .endObject() + .startObject("short_field") + .field("type", "short") + .endObject() + .startObject("integer_field") + .field("type", "integer") + .endObject() + .startObject("long_field") + .field("type", "long") + .endObject() + .startObject("float_field") + .field("type", "float") + .endObject() + .startObject("double_field") + .field("type", "double") + .endObject() + .startObject("date_field") + .field("type", "date") + .endObject() + .startObject("boolean_field") + .field("type", "boolean") + .endObject() + .startObject("binary_field") + .field("type", "binary") + .field("doc_values", true) // off by default on binary fields + .endObject() + .startObject("ip_field") + .field("type", "ip") + .endObject() + .startObject("flat_object_field") + .field("type", "flat_object") + .endObject() + .endObject() + .endObject() + .endObject() + .toString(); client().admin().indices().preparePutMapping().setSource(mapping, XContentType.JSON).get(); @@ -1067,9 +1060,6 @@ public void testDocValueFields() throws Exception { ) ) ); - String json = Strings.toString( - XContentFactory.jsonBuilder().startObject().startObject("flat_object_field").field("foo", "bar").endObject().endObject() - ); assertThat(searchResponse.getHits().getAt(0).getFields().get("byte_field").getValue().toString(), equalTo("1")); assertThat(searchResponse.getHits().getAt(0).getFields().get("short_field").getValue().toString(), equalTo("2")); assertThat(searchResponse.getHits().getAt(0).getFields().get("integer_field").getValue(), equalTo((Object) 3L)); diff --git a/server/src/internalClusterTest/java/org/opensearch/search/geo/GeoFilterIT.java b/server/src/internalClusterTest/java/org/opensearch/search/geo/GeoFilterIT.java index bb14ed1ea5578..d4467b49d1c18 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/geo/GeoFilterIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/geo/GeoFilterIT.java @@ -47,7 +47,6 @@ import org.opensearch.action.search.SearchResponse; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.Priority; -import org.opensearch.common.Strings; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.common.geo.GeoPoint; import org.opensearch.common.geo.GeoUtils; @@ -211,17 +210,16 @@ public void testShapeRelations() throws Exception { assertTrue("Disjoint relation is not supported", disjointSupport); assertTrue("within relation is not supported", withinSupport); - String mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject("area") - .field("type", "geo_shape") - .field("tree", "geohash") - .endObject() - .endObject() - .endObject() - ); + String mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject("area") + .field("type", "geo_shape") + .field("tree", "geohash") + .endObject() + .endObject() + .endObject() + .toString(); CreateIndexRequestBuilder mappingRequest = client().admin().indices().prepareCreate("shapes").setMapping(mapping); mappingRequest.get(); diff --git a/server/src/internalClusterTest/java/org/opensearch/search/geo/GeoShapeIntegrationIT.java b/server/src/internalClusterTest/java/org/opensearch/search/geo/GeoShapeIntegrationIT.java index 89eb6038d8110..cf1a1f82d7200 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/geo/GeoShapeIntegrationIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/geo/GeoShapeIntegrationIT.java @@ -36,7 +36,6 @@ import org.opensearch.action.search.SearchResponse; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.routing.IndexShardRoutingTable; -import org.opensearch.common.Strings; import org.opensearch.common.geo.builders.PointBuilder; import org.opensearch.common.geo.builders.ShapeBuilder; import org.opensearch.common.settings.Settings; @@ -73,32 +72,30 @@ protected Settings nodeSettings(int nodeOrdinal) { */ public void testOrientationPersistence() throws Exception { String idxName = "orientation"; - String mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject("location") - .field("type", "geo_shape") - .field("orientation", "left") - .endObject() - .endObject() - .endObject() - ); + String mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject("location") + .field("type", "geo_shape") + .field("orientation", "left") + .endObject() + .endObject() + .endObject() + .toString(); // create index assertAcked(prepareCreate(idxName).setMapping(mapping)); - mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject("location") - .field("type", "geo_shape") - .field("orientation", "right") - .endObject() - .endObject() - .endObject() - ); + mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject("location") + .field("type", "geo_shape") + .field("orientation", "right") + .endObject() + .endObject() + .endObject() + .toString(); assertAcked(prepareCreate(idxName + "2").setMapping(mapping)); ensureGreen(idxName, idxName + "2"); @@ -140,44 +137,43 @@ public void testIgnoreMalformed() throws Exception { ensureGreen(); // test self crossing ccw poly not crossing dateline - String polygonGeoJson = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .field("type", "Polygon") - .startArray("coordinates") - .startArray() - .startArray() - .value(176.0) - .value(15.0) - .endArray() - .startArray() - .value(-177.0) - .value(10.0) - .endArray() - .startArray() - .value(-177.0) - .value(-10.0) - .endArray() - .startArray() - .value(176.0) - .value(-15.0) - .endArray() - .startArray() - .value(-177.0) - .value(15.0) - .endArray() - .startArray() - .value(172.0) - .value(0.0) - .endArray() - .startArray() - .value(176.0) - .value(15.0) - .endArray() - .endArray() - .endArray() - .endObject() - ); + String polygonGeoJson = XContentFactory.jsonBuilder() + .startObject() + .field("type", "Polygon") + .startArray("coordinates") + .startArray() + .startArray() + .value(176.0) + .value(15.0) + .endArray() + .startArray() + .value(-177.0) + .value(10.0) + .endArray() + .startArray() + .value(-177.0) + .value(-10.0) + .endArray() + .startArray() + .value(176.0) + .value(-15.0) + .endArray() + .startArray() + .value(-177.0) + .value(15.0) + .endArray() + .startArray() + .value(172.0) + .value(0.0) + .endArray() + .startArray() + .value(176.0) + .value(15.0) + .endArray() + .endArray() + .endArray() + .endObject() + .toString(); indexRandom(true, client().prepareIndex("test").setId("0").setSource("shape", polygonGeoJson)); SearchResponse searchResponse = client().prepareSearch("test").setQuery(matchAllQuery()).get(); diff --git a/server/src/internalClusterTest/java/org/opensearch/search/geo/LegacyGeoShapeIntegrationIT.java b/server/src/internalClusterTest/java/org/opensearch/search/geo/LegacyGeoShapeIntegrationIT.java index 11f2132bb29de..6332e2b94750d 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/geo/LegacyGeoShapeIntegrationIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/geo/LegacyGeoShapeIntegrationIT.java @@ -37,7 +37,6 @@ import org.opensearch.action.search.SearchResponse; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.routing.IndexShardRoutingTable; -import org.opensearch.common.Strings; import org.opensearch.common.geo.builders.ShapeBuilder; import org.opensearch.common.settings.Settings; import org.opensearch.core.xcontent.ToXContent; @@ -65,34 +64,32 @@ public class LegacyGeoShapeIntegrationIT extends OpenSearchIntegTestCase { */ public void testOrientationPersistence() throws Exception { String idxName = "orientation"; - String mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject("location") - .field("type", "geo_shape") - .field("tree", "quadtree") - .field("orientation", "left") - .endObject() - .endObject() - .endObject() - ); + String mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject("location") + .field("type", "geo_shape") + .field("tree", "quadtree") + .field("orientation", "left") + .endObject() + .endObject() + .endObject() + .toString(); // create index assertAcked(prepareCreate(idxName).setMapping(mapping)); - mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject("location") - .field("type", "geo_shape") - .field("tree", "quadtree") - .field("orientation", "right") - .endObject() - .endObject() - .endObject() - ); + mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject("location") + .field("type", "geo_shape") + .field("tree", "quadtree") + .field("orientation", "right") + .endObject() + .endObject() + .endObject() + .toString(); assertAcked(prepareCreate(idxName + "2").setMapping(mapping)); ensureGreen(idxName, idxName + "2"); @@ -136,44 +133,43 @@ public void testIgnoreMalformed() throws Exception { ensureGreen(); // test self crossing ccw poly not crossing dateline - String polygonGeoJson = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .field("type", "Polygon") - .startArray("coordinates") - .startArray() - .startArray() - .value(176.0) - .value(15.0) - .endArray() - .startArray() - .value(-177.0) - .value(10.0) - .endArray() - .startArray() - .value(-177.0) - .value(-10.0) - .endArray() - .startArray() - .value(176.0) - .value(-15.0) - .endArray() - .startArray() - .value(-177.0) - .value(15.0) - .endArray() - .startArray() - .value(172.0) - .value(0.0) - .endArray() - .startArray() - .value(176.0) - .value(15.0) - .endArray() - .endArray() - .endArray() - .endObject() - ); + String polygonGeoJson = XContentFactory.jsonBuilder() + .startObject() + .field("type", "Polygon") + .startArray("coordinates") + .startArray() + .startArray() + .value(176.0) + .value(15.0) + .endArray() + .startArray() + .value(-177.0) + .value(10.0) + .endArray() + .startArray() + .value(-177.0) + .value(-10.0) + .endArray() + .startArray() + .value(176.0) + .value(-15.0) + .endArray() + .startArray() + .value(-177.0) + .value(15.0) + .endArray() + .startArray() + .value(172.0) + .value(0.0) + .endArray() + .startArray() + .value(176.0) + .value(15.0) + .endArray() + .endArray() + .endArray() + .endObject() + .toString(); indexRandom(true, client().prepareIndex("test").setId("0").setSource("shape", polygonGeoJson)); SearchResponse searchResponse = client().prepareSearch("test").setQuery(matchAllQuery()).get(); diff --git a/server/src/internalClusterTest/java/org/opensearch/search/morelikethis/MoreLikeThisIT.java b/server/src/internalClusterTest/java/org/opensearch/search/morelikethis/MoreLikeThisIT.java index 04d193aaea71a..2bf3394762621 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/morelikethis/MoreLikeThisIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/morelikethis/MoreLikeThisIT.java @@ -38,7 +38,6 @@ import org.opensearch.action.search.SearchPhaseExecutionException; import org.opensearch.action.search.SearchResponse; import org.opensearch.cluster.health.ClusterHealthStatus; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.common.xcontent.XContentFactory; @@ -270,7 +269,7 @@ public void testMoreLikeThisWithAliasesInLikeDocuments() throws Exception { String indexName = "foo"; String aliasName = "foo_name"; - String mapping = Strings.toString(XContentFactory.jsonBuilder().startObject().startObject("properties").endObject().endObject()); + String mapping = XContentFactory.jsonBuilder().startObject().startObject("properties").endObject().endObject().toString(); client().admin().indices().prepareCreate(indexName).setMapping(mapping).get(); client().admin().indices().prepareAliases().addAlias(indexName, aliasName).get(); @@ -292,7 +291,7 @@ public void testMoreLikeThisWithAliasesInLikeDocuments() throws Exception { } public void testMoreLikeThisIssue2197() throws Exception { - String mapping = Strings.toString(XContentFactory.jsonBuilder().startObject().startObject("properties").endObject().endObject()); + String mapping = XContentFactory.jsonBuilder().startObject().startObject("properties").endObject().endObject().toString(); client().admin().indices().prepareCreate("foo").setMapping(mapping).get(); client().prepareIndex("foo") .setId("1") @@ -313,7 +312,7 @@ public void testMoreLikeThisIssue2197() throws Exception { // Issue #2489 public void testMoreLikeWithCustomRouting() throws Exception { - String mapping = Strings.toString(XContentFactory.jsonBuilder().startObject().startObject("properties").endObject().endObject()); + String mapping = XContentFactory.jsonBuilder().startObject().startObject("properties").endObject().endObject().toString(); client().admin().indices().prepareCreate("foo").setMapping(mapping).get(); ensureGreen(); @@ -333,7 +332,7 @@ public void testMoreLikeWithCustomRouting() throws Exception { // Issue #3039 public void testMoreLikeThisIssueRoutingNotSerialized() throws Exception { - String mapping = Strings.toString(XContentFactory.jsonBuilder().startObject().startObject("properties").endObject().endObject()); + String mapping = XContentFactory.jsonBuilder().startObject().startObject("properties").endObject().endObject().toString(); assertAcked( prepareCreate("foo", 2, Settings.builder().put(SETTING_NUMBER_OF_SHARDS, 2).put(SETTING_NUMBER_OF_REPLICAS, 0)).setMapping( mapping diff --git a/server/src/internalClusterTest/java/org/opensearch/search/query/ExistsIT.java b/server/src/internalClusterTest/java/org/opensearch/search/query/ExistsIT.java index e1724d496fa91..a531fd0eeb5a9 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/query/ExistsIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/query/ExistsIT.java @@ -35,7 +35,6 @@ import org.opensearch.action.explain.ExplainResponse; import org.opensearch.action.index.IndexRequestBuilder; import org.opensearch.action.search.SearchResponse; -import org.opensearch.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.common.xcontent.XContentFactory; import org.opensearch.common.xcontent.json.JsonXContent; @@ -140,14 +139,7 @@ public void testExists() throws Exception { assertSearchResponse(resp); try { assertEquals( - String.format( - Locale.ROOT, - "exists(%s, %d) mapping: %s response: %s", - fieldName, - count, - Strings.toString(mapping), - resp - ), + String.format(Locale.ROOT, "exists(%s, %d) mapping: %s response: %s", fieldName, count, mapping.toString(), resp), count, resp.getHits().getTotalHits().value ); diff --git a/server/src/internalClusterTest/java/org/opensearch/search/query/SimpleQueryStringIT.java b/server/src/internalClusterTest/java/org/opensearch/search/query/SimpleQueryStringIT.java index 46b81ae2e750d..e2491600a9261 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/query/SimpleQueryStringIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/query/SimpleQueryStringIT.java @@ -41,7 +41,6 @@ import org.opensearch.action.index.IndexRequestBuilder; import org.opensearch.action.search.SearchPhaseExecutionException; import org.opensearch.action.search.SearchResponse; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.common.xcontent.XContentFactory; @@ -373,17 +372,16 @@ public void testLenientFlagBeingTooLenient() throws Exception { } public void testSimpleQueryStringAnalyzeWildcard() throws ExecutionException, InterruptedException, IOException { - String mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject("location") - .field("type", "text") - .field("analyzer", "standard") - .endObject() - .endObject() - .endObject() - ); + String mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject("location") + .field("type", "text") + .field("analyzer", "standard") + .endObject() + .endObject() + .endObject() + .toString(); CreateIndexRequestBuilder mappingRequest = client().admin().indices().prepareCreate("test1").setMapping(mapping); mappingRequest.get(); @@ -420,17 +418,16 @@ public void testSimpleQueryStringOnIndexMetaField() throws Exception { public void testEmptySimpleQueryStringWithAnalysis() throws Exception { // https://github.com/elastic/elasticsearch/issues/18202 - String mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject("body") - .field("type", "text") - .field("analyzer", "stop") - .endObject() - .endObject() - .endObject() - ); + String mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject("body") + .field("type", "text") + .field("analyzer", "stop") + .endObject() + .endObject() + .endObject() + .toString(); CreateIndexRequestBuilder mappingRequest = client().admin().indices().prepareCreate("test1").setMapping(mapping); mappingRequest.get(); diff --git a/server/src/internalClusterTest/java/org/opensearch/search/slice/SearchSliceIT.java b/server/src/internalClusterTest/java/org/opensearch/search/slice/SearchSliceIT.java index cc7e620f33216..dcc1136a6d267 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/slice/SearchSliceIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/slice/SearchSliceIT.java @@ -42,7 +42,6 @@ import org.opensearch.action.search.SearchPhaseExecutionException; import org.opensearch.action.search.SearchRequestBuilder; import org.opensearch.action.search.SearchResponse; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.core.xcontent.XContentBuilder; @@ -68,25 +67,24 @@ public class SearchSliceIT extends OpenSearchIntegTestCase { private void setupIndex(int numDocs, int numberOfShards) throws IOException, ExecutionException, InterruptedException { - String mapping = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("properties") - .startObject("invalid_random_kw") - .field("type", "keyword") - .field("doc_values", "false") - .endObject() - .startObject("random_int") - .field("type", "integer") - .field("doc_values", "true") - .endObject() - .startObject("invalid_random_int") - .field("type", "integer") - .field("doc_values", "false") - .endObject() - .endObject() - .endObject() - ); + String mapping = XContentFactory.jsonBuilder() + .startObject() + .startObject("properties") + .startObject("invalid_random_kw") + .field("type", "keyword") + .field("doc_values", "false") + .endObject() + .startObject("random_int") + .field("type", "integer") + .field("doc_values", "true") + .endObject() + .startObject("invalid_random_int") + .field("type", "integer") + .field("doc_values", "false") + .endObject() + .endObject() + .endObject() + .toString(); assertAcked( client().admin() .indices() diff --git a/server/src/internalClusterTest/java/org/opensearch/search/sort/SimpleSortIT.java b/server/src/internalClusterTest/java/org/opensearch/search/sort/SimpleSortIT.java index 8ff0790e7cb48..2905ef97d521e 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/sort/SimpleSortIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/sort/SimpleSortIT.java @@ -35,7 +35,6 @@ import org.opensearch.action.index.IndexRequestBuilder; import org.opensearch.action.search.SearchResponse; import org.opensearch.action.search.ShardSearchFailure; -import org.opensearch.common.Strings; import org.opensearch.common.geo.GeoPoint; import org.opensearch.common.geo.GeoUtils; import org.opensearch.index.fielddata.ScriptDocValues; @@ -237,24 +236,23 @@ public void testSimpleSorts() throws Exception { } public void testSortMinValueScript() throws IOException { - String mapping = Strings.toString( - jsonBuilder().startObject() - .startObject("properties") - .startObject("lvalue") - .field("type", "long") - .endObject() - .startObject("dvalue") - .field("type", "double") - .endObject() - .startObject("svalue") - .field("type", "keyword") - .endObject() - .startObject("gvalue") - .field("type", "geo_point") - .endObject() - .endObject() - .endObject() - ); + String mapping = jsonBuilder().startObject() + .startObject("properties") + .startObject("lvalue") + .field("type", "long") + .endObject() + .startObject("dvalue") + .field("type", "double") + .endObject() + .startObject("svalue") + .field("type", "keyword") + .endObject() + .startObject("gvalue") + .field("type", "geo_point") + .endObject() + .endObject() + .endObject() + .toString(); assertAcked(prepareCreate("test").setMapping(mapping)); ensureGreen(); @@ -351,18 +349,17 @@ public void testDocumentsWithNullValue() throws Exception { // TODO: sort shouldn't fail when sort field is mapped dynamically // We have to specify mapping explicitly because by the time search is performed dynamic mapping might not // be propagated to all nodes yet and sort operation fail when the sort field is not defined - String mapping = Strings.toString( - jsonBuilder().startObject() - .startObject("properties") - .startObject("id") - .field("type", "keyword") - .endObject() - .startObject("svalue") - .field("type", "keyword") - .endObject() - .endObject() - .endObject() - ); + String mapping = jsonBuilder().startObject() + .startObject("properties") + .startObject("id") + .field("type", "keyword") + .endObject() + .startObject("svalue") + .field("type", "keyword") + .endObject() + .endObject() + .endObject() + .toString(); assertAcked(prepareCreate("test").setMapping(mapping)); ensureGreen(); diff --git a/server/src/internalClusterTest/java/org/opensearch/search/suggest/SuggestSearchIT.java b/server/src/internalClusterTest/java/org/opensearch/search/suggest/SuggestSearchIT.java index 3ffd6ce66831e..4aaa5bf5af852 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/suggest/SuggestSearchIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/suggest/SuggestSearchIT.java @@ -38,7 +38,6 @@ import org.opensearch.action.search.SearchPhaseExecutionException; import org.opensearch.action.search.SearchRequestBuilder; import org.opensearch.action.search.SearchResponse; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.common.xcontent.XContentFactory; @@ -1305,14 +1304,13 @@ public void testPhraseSuggesterCollate() throws InterruptedException, ExecutionE assertSuggestionSize(searchSuggest, 0, 10, "title"); // suggest with collate - String filterString = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("match_phrase") - .field("{{field}}", "{{suggestion}}") - .endObject() - .endObject() - ); + String filterString = XContentFactory.jsonBuilder() + .startObject() + .startObject("match_phrase") + .field("{{field}}", "{{suggestion}}") + .endObject() + .endObject() + .toString(); PhraseSuggestionBuilder filteredQuerySuggest = suggest.collateQuery(filterString); filteredQuerySuggest.collateParams(Collections.singletonMap("field", "title")); searchSuggest = searchSuggest("united states house of representatives elections in washington 2006", "title", filteredQuerySuggest); @@ -1325,9 +1323,13 @@ public void testPhraseSuggesterCollate() throws InterruptedException, ExecutionE NumShards numShards = getNumShards("test"); // collate suggest with bad query - String incorrectFilterString = Strings.toString( - XContentFactory.jsonBuilder().startObject().startObject("test").field("title", "{{suggestion}}").endObject().endObject() - ); + String incorrectFilterString = XContentFactory.jsonBuilder() + .startObject() + .startObject("test") + .field("title", "{{suggestion}}") + .endObject() + .endObject() + .toString(); PhraseSuggestionBuilder incorrectFilteredSuggest = suggest.collateQuery(incorrectFilterString); Map> namedSuggestion = new HashMap<>(); namedSuggestion.put("my_title_suggestion", incorrectFilteredSuggest); @@ -1339,9 +1341,13 @@ public void testPhraseSuggesterCollate() throws InterruptedException, ExecutionE } // suggest with collation - String filterStringAsFilter = Strings.toString( - XContentFactory.jsonBuilder().startObject().startObject("match_phrase").field("title", "{{suggestion}}").endObject().endObject() - ); + String filterStringAsFilter = XContentFactory.jsonBuilder() + .startObject() + .startObject("match_phrase") + .field("title", "{{suggestion}}") + .endObject() + .endObject() + .toString(); PhraseSuggestionBuilder filteredFilterSuggest = suggest.collateQuery(filterStringAsFilter); searchSuggest = searchSuggest( @@ -1352,9 +1358,13 @@ public void testPhraseSuggesterCollate() throws InterruptedException, ExecutionE assertSuggestionSize(searchSuggest, 0, 2, "title"); // collate suggest with bad query - String filterStr = Strings.toString( - XContentFactory.jsonBuilder().startObject().startObject("pprefix").field("title", "{{suggestion}}").endObject().endObject() - ); + String filterStr = XContentFactory.jsonBuilder() + .startObject() + .startObject("pprefix") + .field("title", "{{suggestion}}") + .endObject() + .endObject() + .toString(); suggest.collateQuery(filterStr); try { @@ -1365,14 +1375,13 @@ public void testPhraseSuggesterCollate() throws InterruptedException, ExecutionE } // collate script failure due to no additional params - String collateWithParams = Strings.toString( - XContentFactory.jsonBuilder() - .startObject() - .startObject("{{query_type}}") - .field("{{query_field}}", "{{suggestion}}") - .endObject() - .endObject() - ); + String collateWithParams = XContentFactory.jsonBuilder() + .startObject() + .startObject("{{query_type}}") + .field("{{query_field}}", "{{suggestion}}") + .endObject() + .endObject() + .toString(); try { searchSuggest("united states house of representatives elections in washington 2006", numShards.numPrimaries, namedSuggestion); diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/health/ClusterHealthResponse.java b/server/src/main/java/org/opensearch/action/admin/cluster/health/ClusterHealthResponse.java index 714abe86df71e..019a1e1417510 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/health/ClusterHealthResponse.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/health/ClusterHealthResponse.java @@ -39,7 +39,7 @@ import org.opensearch.cluster.health.ClusterHealthStatus; import org.opensearch.cluster.health.ClusterIndexHealth; import org.opensearch.cluster.health.ClusterStateHealth; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.common.settings.ClusterSettings; diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/node/info/NodesInfoResponse.java b/server/src/main/java/org/opensearch/action/admin/cluster/node/info/NodesInfoResponse.java index 3dcb30a091a94..566ae06649168 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/node/info/NodesInfoResponse.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/node/info/NodesInfoResponse.java @@ -36,7 +36,6 @@ import org.opensearch.action.support.nodes.BaseNodesResponse; import org.opensearch.cluster.ClusterName; import org.opensearch.cluster.node.DiscoveryNodeRole; -import org.opensearch.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.common.settings.Settings; @@ -165,7 +164,7 @@ public String toString() { builder.startObject(); toXContent(builder, EMPTY_PARAMS); builder.endObject(); - return Strings.toString(builder); + return builder.toString(); } catch (IOException e) { return "{ \"error\" : \"" + e.getMessage() + "\"}"; } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/node/reload/NodesReloadSecureSettingsResponse.java b/server/src/main/java/org/opensearch/action/admin/cluster/node/reload/NodesReloadSecureSettingsResponse.java index 22044f0c69c48..a4a69cd301b41 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/node/reload/NodesReloadSecureSettingsResponse.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/node/reload/NodesReloadSecureSettingsResponse.java @@ -38,7 +38,6 @@ import org.opensearch.action.support.nodes.BaseNodesResponse; import org.opensearch.cluster.ClusterName; import org.opensearch.cluster.node.DiscoveryNode; -import org.opensearch.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.xcontent.ToXContentFragment; @@ -99,7 +98,7 @@ public String toString() { builder.startObject(); toXContent(builder, EMPTY_PARAMS); builder.endObject(); - return Strings.toString(builder); + return builder.toString(); } catch (final IOException e) { return "{ \"error\" : \"" + e.getMessage() + "\"}"; } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodesStatsResponse.java b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodesStatsResponse.java index 0037d1ea27873..539bae311ac98 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodesStatsResponse.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodesStatsResponse.java @@ -35,7 +35,6 @@ import org.opensearch.action.FailedNodeException; import org.opensearch.action.support.nodes.BaseNodesResponse; import org.opensearch.cluster.ClusterName; -import org.opensearch.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.xcontent.ToXContentFragment; @@ -92,7 +91,7 @@ public String toString() { builder.startObject(); toXContent(builder, EMPTY_PARAMS); builder.endObject(); - return Strings.toString(builder); + return builder.toString(); } catch (IOException e) { return "{ \"error\" : \"" + e.getMessage() + "\"}"; } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/node/tasks/cancel/CancelTasksResponse.java b/server/src/main/java/org/opensearch/action/admin/cluster/node/tasks/cancel/CancelTasksResponse.java index 4741504ddd035..f2ebde642d2be 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/node/tasks/cancel/CancelTasksResponse.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/node/tasks/cancel/CancelTasksResponse.java @@ -35,7 +35,7 @@ import org.opensearch.OpenSearchException; import org.opensearch.action.TaskOperationFailure; import org.opensearch.action.admin.cluster.node.tasks.list.ListTasksResponse; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.common.xcontent.XContentType; import org.opensearch.core.xcontent.ConstructingObjectParser; diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/node/tasks/get/GetTaskResponse.java b/server/src/main/java/org/opensearch/action/admin/cluster/node/tasks/get/GetTaskResponse.java index b32e59fc77794..36bec88109cf1 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/node/tasks/get/GetTaskResponse.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/node/tasks/get/GetTaskResponse.java @@ -33,7 +33,7 @@ package org.opensearch.action.admin.cluster.node.tasks.get; import org.opensearch.action.ActionResponse; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.common.xcontent.XContentType; diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/node/tasks/list/ListTasksResponse.java b/server/src/main/java/org/opensearch/action/admin/cluster/node/tasks/list/ListTasksResponse.java index 85b165df68cfa..1e2e432882623 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/node/tasks/list/ListTasksResponse.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/node/tasks/list/ListTasksResponse.java @@ -39,8 +39,8 @@ import org.opensearch.cluster.node.DiscoveryNodeRole; import org.opensearch.cluster.node.DiscoveryNodes; import org.opensearch.core.ParseField; -import org.opensearch.common.Strings; import org.opensearch.common.TriFunction; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.common.xcontent.XContentType; diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/node/usage/NodesUsageResponse.java b/server/src/main/java/org/opensearch/action/admin/cluster/node/usage/NodesUsageResponse.java index 59b1f99429cfe..e0c8e6e8e269e 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/node/usage/NodesUsageResponse.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/node/usage/NodesUsageResponse.java @@ -35,7 +35,6 @@ import org.opensearch.action.FailedNodeException; import org.opensearch.action.support.nodes.BaseNodesResponse; import org.opensearch.cluster.ClusterName; -import org.opensearch.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.xcontent.ToXContentFragment; @@ -93,7 +92,7 @@ public String toString() { builder.startObject(); toXContent(builder, EMPTY_PARAMS); builder.endObject(); - return Strings.toString(builder); + return builder.toString(); } catch (IOException e) { return "{ \"error\" : \"" + e.getMessage() + "\"}"; } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/remotestore/restore/RestoreRemoteStoreRequest.java b/server/src/main/java/org/opensearch/action/admin/cluster/remotestore/restore/RestoreRemoteStoreRequest.java index eb1935158c231..fd29c324d51a1 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/remotestore/restore/RestoreRemoteStoreRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/remotestore/restore/RestoreRemoteStoreRequest.java @@ -205,7 +205,7 @@ public int hashCode() { @Override public String toString() { - return org.opensearch.common.Strings.toString(XContentType.JSON, this); + return Strings.toString(XContentType.JSON, this); } } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/remotestore/stats/RemoteStoreStatsResponse.java b/server/src/main/java/org/opensearch/action/admin/cluster/remotestore/stats/RemoteStoreStatsResponse.java index f6613c1d2ac50..4f0832816fd8a 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/remotestore/stats/RemoteStoreStatsResponse.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/remotestore/stats/RemoteStoreStatsResponse.java @@ -10,7 +10,7 @@ import org.opensearch.core.action.support.DefaultShardOperationFailedException; import org.opensearch.action.support.broadcast.BroadcastResponse; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.common.xcontent.XContentType; diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/repositories/verify/VerifyRepositoryResponse.java b/server/src/main/java/org/opensearch/action/admin/cluster/repositories/verify/VerifyRepositoryResponse.java index 8b9142a39e9c9..9a97b67e1c2b7 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/repositories/verify/VerifyRepositoryResponse.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/repositories/verify/VerifyRepositoryResponse.java @@ -35,7 +35,7 @@ import org.opensearch.action.ActionResponse; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.core.ParseField; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/settings/ClusterGetSettingsResponse.java b/server/src/main/java/org/opensearch/action/admin/cluster/settings/ClusterGetSettingsResponse.java index 8d29baa82562a..9a8206a4cfdba 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/settings/ClusterGetSettingsResponse.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/settings/ClusterGetSettingsResponse.java @@ -34,7 +34,7 @@ import org.opensearch.action.ActionResponse; import org.opensearch.core.ParseField; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.common.settings.Settings; import org.opensearch.core.xcontent.ConstructingObjectParser; diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/clone/CloneSnapshotRequest.java b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/clone/CloneSnapshotRequest.java index fedcfa1f5d9ff..5729386259df1 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/clone/CloneSnapshotRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/clone/CloneSnapshotRequest.java @@ -36,7 +36,7 @@ import org.opensearch.action.IndicesRequest; import org.opensearch.action.support.IndicesOptions; import org.opensearch.action.support.clustermanager.ClusterManagerNodeRequest; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.common.xcontent.XContentType; diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/create/CreateSnapshotRequest.java b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/create/CreateSnapshotRequest.java index ce41066868202..9736d99b9f886 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/create/CreateSnapshotRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/create/CreateSnapshotRequest.java @@ -38,7 +38,7 @@ import org.opensearch.action.IndicesRequest; import org.opensearch.action.support.IndicesOptions; import org.opensearch.action.support.clustermanager.ClusterManagerNodeRequest; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; @@ -74,7 +74,7 @@ *
  • must not contain hash sign ('#')
  • *
  • must not start with underscore ('_')
  • *
  • must be lowercase
  • - *
  • must not contain invalid file name characters {@link org.opensearch.core.common.Strings#INVALID_FILENAME_CHARS}
  • + *
  • must not contain invalid file name characters {@link Strings#INVALID_FILENAME_CHARS}
  • * * * @opensearch.internal @@ -390,7 +390,7 @@ public CreateSnapshotRequest settings(Map source) { try { XContentBuilder builder = MediaTypeRegistry.contentBuilder(XContentType.JSON); builder.map(source); - settings(Strings.toString(builder), builder.contentType()); + settings(builder.toString(), builder.contentType()); } catch (IOException e) { throw new OpenSearchGenerationException("Failed to generate [" + source + "]", e); } @@ -447,7 +447,7 @@ public CreateSnapshotRequest source(Map source) { String name = entry.getKey(); if (name.equals("indices")) { if (entry.getValue() instanceof String) { - indices(org.opensearch.core.common.Strings.splitStringByCommaToArray((String) entry.getValue())); + indices(Strings.splitStringByCommaToArray((String) entry.getValue())); } else if (entry.getValue() instanceof List) { indices((List) entry.getValue()); } else { diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/get/GetSnapshotsResponse.java b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/get/GetSnapshotsResponse.java index 3c96c67f78fd4..b8b4d972c95f7 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/get/GetSnapshotsResponse.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/get/GetSnapshotsResponse.java @@ -33,7 +33,7 @@ package org.opensearch.action.admin.cluster.snapshots.get; import org.opensearch.action.ActionResponse; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.common.xcontent.XContentType; diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/restore/RestoreSnapshotRequest.java b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/restore/RestoreSnapshotRequest.java index 7a142e70305ae..0f9aa65afe3c2 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/restore/RestoreSnapshotRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/restore/RestoreSnapshotRequest.java @@ -749,6 +749,6 @@ public int hashCode() { @Override public String toString() { - return org.opensearch.common.Strings.toString(XContentType.JSON, this); + return Strings.toString(XContentType.JSON, this); } } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/SnapshotStats.java b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/SnapshotStats.java index 8b718aeaf70c7..c7690ea0d7817 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/SnapshotStats.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/SnapshotStats.java @@ -32,7 +32,7 @@ package org.opensearch.action.admin.cluster.snapshots.status; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/SnapshotStatus.java b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/SnapshotStatus.java index a3b401980b109..d1e25c1f1bdc4 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/SnapshotStatus.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/SnapshotStatus.java @@ -35,7 +35,7 @@ import org.opensearch.cluster.SnapshotsInProgress; import org.opensearch.cluster.SnapshotsInProgress.State; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/stats/AnalysisStats.java b/server/src/main/java/org/opensearch/action/admin/cluster/stats/AnalysisStats.java index 9cdd5bf244ecb..84b093f9bb238 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/stats/AnalysisStats.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/stats/AnalysisStats.java @@ -35,7 +35,7 @@ import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.MappingMetadata; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/stats/ClusterStatsResponse.java b/server/src/main/java/org/opensearch/action/admin/cluster/stats/ClusterStatsResponse.java index 1dda39a17babc..8926f41777809 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/stats/ClusterStatsResponse.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/stats/ClusterStatsResponse.java @@ -37,7 +37,6 @@ import org.opensearch.cluster.ClusterName; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.health.ClusterHealthStatus; -import org.opensearch.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.xcontent.ToXContentFragment; @@ -168,7 +167,7 @@ public String toString() { builder.startObject(); toXContent(builder, EMPTY_PARAMS); builder.endObject(); - return Strings.toString(builder); + return builder.toString(); } catch (IOException e) { return "{ \"error\" : \"" + e.getMessage() + "\"}"; } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/stats/MappingStats.java b/server/src/main/java/org/opensearch/action/admin/cluster/stats/MappingStats.java index 90c78f30ea78d..66d1fc6a52295 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/stats/MappingStats.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/stats/MappingStats.java @@ -35,7 +35,7 @@ import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.MappingMetadata; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/alias/Alias.java b/server/src/main/java/org/opensearch/action/admin/indices/alias/Alias.java index 2f304668df01c..94dbb5ff46a02 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/alias/Alias.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/alias/Alias.java @@ -35,7 +35,7 @@ import org.opensearch.OpenSearchGenerationException; import org.opensearch.common.Nullable; import org.opensearch.core.ParseField; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.bytes.BytesArray; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; @@ -44,9 +44,7 @@ import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.XContentBuilder; -import org.opensearch.common.xcontent.XContentFactory; import org.opensearch.core.xcontent.XContentParser; -import org.opensearch.common.xcontent.XContentType; import org.opensearch.index.query.QueryBuilder; import java.io.IOException; @@ -128,9 +126,9 @@ public Alias filter(Map filter) { return this; } try { - XContentBuilder builder = MediaTypeRegistry.contentBuilder(XContentType.JSON); + XContentBuilder builder = MediaTypeRegistry.contentBuilder(MediaTypeRegistry.JSON); builder.map(filter); - this.filter = Strings.toString(builder); + this.filter = builder.toString(); return this; } catch (IOException e) { throw new OpenSearchGenerationException("Failed to generate [" + filter + "]", e); @@ -146,10 +144,10 @@ public Alias filter(QueryBuilder filterBuilder) { return this; } try { - XContentBuilder builder = XContentFactory.jsonBuilder(); + XContentBuilder builder = MediaTypeRegistry.JSON.contentBuilder(); filterBuilder.toXContent(builder, ToXContent.EMPTY_PARAMS); builder.close(); - this.filter = Strings.toString(builder); + this.filter = builder.toString(); return this; } catch (IOException e) { throw new OpenSearchGenerationException("Failed to build json for alias request", e); @@ -279,7 +277,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (filter != null) { try (InputStream stream = new BytesArray(filter).streamInput()) { - builder.rawField(FILTER.getPreferredName(), stream, XContentType.JSON); + builder.rawField(FILTER.getPreferredName(), stream, MediaTypeRegistry.JSON); } } @@ -306,7 +304,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws @Override public String toString() { - return Strings.toString(XContentType.JSON, this); + return Strings.toString(MediaTypeRegistry.JSON, this); } @Override diff --git a/server/src/main/java/org/opensearch/action/admin/indices/alias/IndicesAliasesRequest.java b/server/src/main/java/org/opensearch/action/admin/indices/alias/IndicesAliasesRequest.java index 6ed9648502ee3..cd99a1067a8a4 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/alias/IndicesAliasesRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/alias/IndicesAliasesRequest.java @@ -431,7 +431,7 @@ public AliasActions filter(Map filter) { try { XContentBuilder builder = MediaTypeRegistry.contentBuilder(XContentType.JSON); builder.map(filter); - this.filter = org.opensearch.common.Strings.toString(builder); + this.filter = builder.toString(); return this; } catch (IOException e) { throw new OpenSearchGenerationException("Failed to generate [" + filter + "]", e); @@ -447,7 +447,7 @@ public AliasActions filter(QueryBuilder filter) { XContentBuilder builder = XContentFactory.jsonBuilder(); filter.toXContent(builder, ToXContent.EMPTY_PARAMS); builder.close(); - this.filter = org.opensearch.common.Strings.toString(builder); + this.filter = builder.toString(); return this; } catch (IOException e) { throw new OpenSearchGenerationException("Failed to build json for alias request", e); diff --git a/server/src/main/java/org/opensearch/action/admin/indices/analyze/AnalyzeAction.java b/server/src/main/java/org/opensearch/action/admin/indices/analyze/AnalyzeAction.java index 37e05151c8179..00144eedc438f 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/analyze/AnalyzeAction.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/analyze/AnalyzeAction.java @@ -36,7 +36,7 @@ import org.opensearch.action.ActionResponse; import org.opensearch.action.ActionType; import org.opensearch.action.support.single.shard.SingleShardRequest; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/close/CloseIndexResponse.java b/server/src/main/java/org/opensearch/action/admin/indices/close/CloseIndexResponse.java index a3c599a54a10c..c069cd17b8c51 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/close/CloseIndexResponse.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/close/CloseIndexResponse.java @@ -35,7 +35,7 @@ import org.opensearch.core.action.support.DefaultShardOperationFailedException; import org.opensearch.action.support.master.ShardsAcknowledgedResponse; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/create/CreateIndexRequest.java b/server/src/main/java/org/opensearch/action/admin/indices/create/CreateIndexRequest.java index 53840b7697e45..001b466fc47e5 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/create/CreateIndexRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/create/CreateIndexRequest.java @@ -42,7 +42,6 @@ import org.opensearch.action.support.ActiveShardCount; import org.opensearch.action.support.IndicesOptions; import org.opensearch.action.support.master.AcknowledgedRequest; -import org.opensearch.common.Strings; import org.opensearch.core.common.bytes.BytesArray; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.common.io.stream.StreamInput; @@ -230,7 +229,7 @@ private CreateIndexRequest settings(String source, MediaType mediaType) { * Allows to set the settings using a json builder. */ public CreateIndexRequest settings(XContentBuilder builder) { - settings(Strings.toString(builder), builder.contentType()); + settings(builder.toString(), builder.contentType()); return this; } @@ -346,7 +345,7 @@ private CreateIndexRequest mapping(String type, Map source) { try { XContentBuilder builder = XContentFactory.jsonBuilder(); builder.map(source); - return mapping(Strings.toString(builder)); + return mapping(builder.toString()); } catch (IOException e) { throw new OpenSearchGenerationException("Failed to generate [" + source + "]", e); } diff --git a/server/src/main/java/org/opensearch/action/admin/indices/get/GetIndexResponse.java b/server/src/main/java/org/opensearch/action/admin/indices/get/GetIndexResponse.java index df05c3dd665d2..b0fc6856eb43c 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/get/GetIndexResponse.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/get/GetIndexResponse.java @@ -36,7 +36,7 @@ import org.opensearch.action.ActionResponse; import org.opensearch.cluster.metadata.AliasMetadata; import org.opensearch.cluster.metadata.MappingMetadata; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.common.settings.Settings; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/mapping/get/GetMappingsResponse.java b/server/src/main/java/org/opensearch/action/admin/indices/mapping/get/GetMappingsResponse.java index c4c9094e276d6..6b3fff19d532f 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/mapping/get/GetMappingsResponse.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/mapping/get/GetMappingsResponse.java @@ -35,7 +35,7 @@ import org.opensearch.Version; import org.opensearch.action.ActionResponse; import org.opensearch.cluster.metadata.MappingMetadata; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.common.xcontent.XContentType; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/readonly/AddIndexBlockResponse.java b/server/src/main/java/org/opensearch/action/admin/indices/readonly/AddIndexBlockResponse.java index f90bdaca510ae..42dacfdb3ca2d 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/readonly/AddIndexBlockResponse.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/readonly/AddIndexBlockResponse.java @@ -35,7 +35,7 @@ import org.opensearch.core.action.support.DefaultShardOperationFailedException; import org.opensearch.action.support.master.ShardsAcknowledgedResponse; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/recovery/RecoveryResponse.java b/server/src/main/java/org/opensearch/action/admin/indices/recovery/RecoveryResponse.java index 7664a73c27fc8..a7015a9d580df 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/recovery/RecoveryResponse.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/recovery/RecoveryResponse.java @@ -34,7 +34,7 @@ import org.opensearch.core.action.support.DefaultShardOperationFailedException; import org.opensearch.action.support.broadcast.BroadcastResponse; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.common.xcontent.XContentType; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/replication/SegmentReplicationStatsResponse.java b/server/src/main/java/org/opensearch/action/admin/indices/replication/SegmentReplicationStatsResponse.java index 63899668badca..99ff501c1eed8 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/replication/SegmentReplicationStatsResponse.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/replication/SegmentReplicationStatsResponse.java @@ -10,7 +10,7 @@ import org.opensearch.core.action.support.DefaultShardOperationFailedException; import org.opensearch.action.support.broadcast.BroadcastResponse; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.common.xcontent.XContentType; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/rollover/RolloverInfo.java b/server/src/main/java/org/opensearch/action/admin/indices/rollover/RolloverInfo.java index f58c842be374b..8503c9b882c93 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/rollover/RolloverInfo.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/rollover/RolloverInfo.java @@ -34,7 +34,7 @@ import org.opensearch.cluster.AbstractDiffable; import org.opensearch.cluster.Diff; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/settings/get/GetSettingsResponse.java b/server/src/main/java/org/opensearch/action/admin/indices/settings/get/GetSettingsResponse.java index 05b06fc2b62c7..a1178fbe21f3c 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/settings/get/GetSettingsResponse.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/settings/get/GetSettingsResponse.java @@ -33,7 +33,6 @@ package org.opensearch.action.admin.indices.settings.get; import org.opensearch.action.ActionResponse; -import org.opensearch.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.common.settings.Settings; @@ -193,7 +192,7 @@ public String toString() { ByteArrayOutputStream baos = new ByteArrayOutputStream(); XContentBuilder builder = new XContentBuilder(JsonXContent.jsonXContent, baos); toXContent(builder, ToXContent.EMPTY_PARAMS, false); - return Strings.toString(builder); + return builder.toString(); } catch (IOException e) { throw new IllegalStateException(e); // should not be possible here } diff --git a/server/src/main/java/org/opensearch/action/admin/indices/settings/put/UpdateSettingsRequest.java b/server/src/main/java/org/opensearch/action/admin/indices/settings/put/UpdateSettingsRequest.java index a7b7e005bce90..3c12f3eb8b728 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/settings/put/UpdateSettingsRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/settings/put/UpdateSettingsRequest.java @@ -36,7 +36,7 @@ import org.opensearch.action.IndicesRequest; import org.opensearch.action.support.IndicesOptions; import org.opensearch.action.support.master.AcknowledgedRequest; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.common.settings.Settings; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/stats/IndicesStatsResponse.java b/server/src/main/java/org/opensearch/action/admin/indices/stats/IndicesStatsResponse.java index 4014bad06ff9a..b262835dc2f2a 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/stats/IndicesStatsResponse.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/stats/IndicesStatsResponse.java @@ -36,7 +36,7 @@ import org.opensearch.core.action.support.DefaultShardOperationFailedException; import org.opensearch.action.support.broadcast.BroadcastResponse; import org.opensearch.cluster.routing.ShardRouting; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.common.xcontent.XContentType; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/template/put/PutIndexTemplateRequest.java b/server/src/main/java/org/opensearch/action/admin/indices/template/put/PutIndexTemplateRequest.java index 099f7c34ff818..011f10bfaf6d6 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/template/put/PutIndexTemplateRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/template/put/PutIndexTemplateRequest.java @@ -41,7 +41,6 @@ import org.opensearch.action.support.IndicesOptions; import org.opensearch.action.support.clustermanager.ClusterManagerNodeRequest; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.core.common.bytes.BytesArray; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.common.io.stream.StreamInput; @@ -298,7 +297,7 @@ public PutIndexTemplateRequest mapping(Map source) { try { XContentBuilder builder = XContentFactory.jsonBuilder(); builder.map(source); - mappings = Strings.toString(builder); + mappings = builder.toString(); return this; } catch (IOException e) { throw new OpenSearchGenerationException("Failed to generate [" + source + "]", e); diff --git a/server/src/main/java/org/opensearch/action/bulk/BulkItemRequest.java b/server/src/main/java/org/opensearch/action/bulk/BulkItemRequest.java index 2a85b7abb741a..bbf887b71cbb2 100644 --- a/server/src/main/java/org/opensearch/action/bulk/BulkItemRequest.java +++ b/server/src/main/java/org/opensearch/action/bulk/BulkItemRequest.java @@ -36,7 +36,7 @@ import org.apache.lucene.util.RamUsageEstimator; import org.opensearch.action.DocWriteRequest; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; diff --git a/server/src/main/java/org/opensearch/action/bulk/BulkItemResponse.java b/server/src/main/java/org/opensearch/action/bulk/BulkItemResponse.java index 26cd318e7a280..08a8e7b6d7865 100644 --- a/server/src/main/java/org/opensearch/action/bulk/BulkItemResponse.java +++ b/server/src/main/java/org/opensearch/action/bulk/BulkItemResponse.java @@ -41,7 +41,7 @@ import org.opensearch.action.index.IndexResponse; import org.opensearch.action.update.UpdateResponse; import org.opensearch.common.CheckedConsumer; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; diff --git a/server/src/main/java/org/opensearch/action/fieldcaps/FieldCapabilities.java b/server/src/main/java/org/opensearch/action/fieldcaps/FieldCapabilities.java index 8053e1f8521cf..c4298e75f8302 100644 --- a/server/src/main/java/org/opensearch/action/fieldcaps/FieldCapabilities.java +++ b/server/src/main/java/org/opensearch/action/fieldcaps/FieldCapabilities.java @@ -32,7 +32,7 @@ package org.opensearch.action.fieldcaps; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; diff --git a/server/src/main/java/org/opensearch/action/fieldcaps/FieldCapabilitiesResponse.java b/server/src/main/java/org/opensearch/action/fieldcaps/FieldCapabilitiesResponse.java index 550ef2d412ca4..5b14a0d5a40b4 100644 --- a/server/src/main/java/org/opensearch/action/fieldcaps/FieldCapabilitiesResponse.java +++ b/server/src/main/java/org/opensearch/action/fieldcaps/FieldCapabilitiesResponse.java @@ -34,7 +34,7 @@ import org.opensearch.action.ActionResponse; import org.opensearch.core.ParseField; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.common.collect.Tuple; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; diff --git a/server/src/main/java/org/opensearch/action/get/GetResponse.java b/server/src/main/java/org/opensearch/action/get/GetResponse.java index b713dc8a507d1..abb1ddfe041c9 100644 --- a/server/src/main/java/org/opensearch/action/get/GetResponse.java +++ b/server/src/main/java/org/opensearch/action/get/GetResponse.java @@ -35,7 +35,7 @@ import org.opensearch.OpenSearchParseException; import org.opensearch.action.ActionResponse; import org.opensearch.core.common.ParsingException; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.common.document.DocumentField; import org.opensearch.core.common.io.stream.StreamInput; diff --git a/server/src/main/java/org/opensearch/action/get/MultiGetRequest.java b/server/src/main/java/org/opensearch/action/get/MultiGetRequest.java index e8d0c1b9d320f..3a28b123b6539 100644 --- a/server/src/main/java/org/opensearch/action/get/MultiGetRequest.java +++ b/server/src/main/java/org/opensearch/action/get/MultiGetRequest.java @@ -43,7 +43,7 @@ import org.opensearch.action.support.IndicesOptions; import org.opensearch.common.Nullable; import org.opensearch.core.common.ParsingException; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; diff --git a/server/src/main/java/org/opensearch/action/index/IndexResponse.java b/server/src/main/java/org/opensearch/action/index/IndexResponse.java index 12d788323b497..c7c2138a63b4e 100644 --- a/server/src/main/java/org/opensearch/action/index/IndexResponse.java +++ b/server/src/main/java/org/opensearch/action/index/IndexResponse.java @@ -33,7 +33,7 @@ package org.opensearch.action.index; import org.opensearch.action.DocWriteResponse; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.common.xcontent.XContentType; import org.opensearch.core.xcontent.XContentParser; diff --git a/server/src/main/java/org/opensearch/action/ingest/GetPipelineResponse.java b/server/src/main/java/org/opensearch/action/ingest/GetPipelineResponse.java index a26fa413b2f5b..a22f499c4add4 100644 --- a/server/src/main/java/org/opensearch/action/ingest/GetPipelineResponse.java +++ b/server/src/main/java/org/opensearch/action/ingest/GetPipelineResponse.java @@ -33,7 +33,7 @@ package org.opensearch.action.ingest; import org.opensearch.action.ActionResponse; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; diff --git a/server/src/main/java/org/opensearch/action/search/GetSearchPipelineResponse.java b/server/src/main/java/org/opensearch/action/search/GetSearchPipelineResponse.java index 4211839ce6569..cf8d9cec779c8 100644 --- a/server/src/main/java/org/opensearch/action/search/GetSearchPipelineResponse.java +++ b/server/src/main/java/org/opensearch/action/search/GetSearchPipelineResponse.java @@ -9,7 +9,7 @@ package org.opensearch.action.search; import org.opensearch.action.ActionResponse; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; diff --git a/server/src/main/java/org/opensearch/action/search/MultiSearchResponse.java b/server/src/main/java/org/opensearch/action/search/MultiSearchResponse.java index c576d87d85b0f..7facd62f90bad 100644 --- a/server/src/main/java/org/opensearch/action/search/MultiSearchResponse.java +++ b/server/src/main/java/org/opensearch/action/search/MultiSearchResponse.java @@ -36,7 +36,7 @@ import org.opensearch.OpenSearchException; import org.opensearch.action.ActionResponse; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; diff --git a/server/src/main/java/org/opensearch/action/search/SearchResponse.java b/server/src/main/java/org/opensearch/action/search/SearchResponse.java index c7ab8f0858e7b..d4ebc0a2363af 100644 --- a/server/src/main/java/org/opensearch/action/search/SearchResponse.java +++ b/server/src/main/java/org/opensearch/action/search/SearchResponse.java @@ -35,7 +35,7 @@ import org.apache.lucene.search.TotalHits; import org.opensearch.action.ActionResponse; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; diff --git a/server/src/main/java/org/opensearch/action/support/replication/ReplicationTask.java b/server/src/main/java/org/opensearch/action/support/replication/ReplicationTask.java index 5aa37ec65ff0d..da00183515ae5 100644 --- a/server/src/main/java/org/opensearch/action/support/replication/ReplicationTask.java +++ b/server/src/main/java/org/opensearch/action/support/replication/ReplicationTask.java @@ -32,7 +32,7 @@ package org.opensearch.action.support.replication; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.common.xcontent.XContentType; diff --git a/server/src/main/java/org/opensearch/cluster/RepositoryCleanupInProgress.java b/server/src/main/java/org/opensearch/cluster/RepositoryCleanupInProgress.java index c399bd59dbbe1..be471ab6a68ec 100644 --- a/server/src/main/java/org/opensearch/cluster/RepositoryCleanupInProgress.java +++ b/server/src/main/java/org/opensearch/cluster/RepositoryCleanupInProgress.java @@ -33,7 +33,7 @@ import org.opensearch.LegacyESVersion; import org.opensearch.Version; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; diff --git a/server/src/main/java/org/opensearch/cluster/SnapshotsInProgress.java b/server/src/main/java/org/opensearch/cluster/SnapshotsInProgress.java index 855f7755419d8..5f9c93ff254c9 100644 --- a/server/src/main/java/org/opensearch/cluster/SnapshotsInProgress.java +++ b/server/src/main/java/org/opensearch/cluster/SnapshotsInProgress.java @@ -35,7 +35,7 @@ import org.opensearch.Version; import org.opensearch.cluster.ClusterState.Custom; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; diff --git a/server/src/main/java/org/opensearch/cluster/awarenesshealth/ClusterAwarenessAttributeValueHealth.java b/server/src/main/java/org/opensearch/cluster/awarenesshealth/ClusterAwarenessAttributeValueHealth.java index 75b73be8fa12e..1520a293d2741 100644 --- a/server/src/main/java/org/opensearch/cluster/awarenesshealth/ClusterAwarenessAttributeValueHealth.java +++ b/server/src/main/java/org/opensearch/cluster/awarenesshealth/ClusterAwarenessAttributeValueHealth.java @@ -14,7 +14,7 @@ import org.opensearch.cluster.routing.RoutingNode; import org.opensearch.cluster.routing.ShardRoutingState; import org.opensearch.cluster.routing.WeightedRouting; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; diff --git a/server/src/main/java/org/opensearch/cluster/awarenesshealth/ClusterAwarenessAttributesHealth.java b/server/src/main/java/org/opensearch/cluster/awarenesshealth/ClusterAwarenessAttributesHealth.java index 340fcfe0d0d31..08832cb1e8807 100644 --- a/server/src/main/java/org/opensearch/cluster/awarenesshealth/ClusterAwarenessAttributesHealth.java +++ b/server/src/main/java/org/opensearch/cluster/awarenesshealth/ClusterAwarenessAttributesHealth.java @@ -11,7 +11,7 @@ import org.opensearch.OpenSearchParseException; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.node.DiscoveryNode; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; diff --git a/server/src/main/java/org/opensearch/cluster/coordination/Coordinator.java b/server/src/main/java/org/opensearch/cluster/coordination/Coordinator.java index b57da128ce852..4429136525534 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/Coordinator.java @@ -61,8 +61,8 @@ import org.opensearch.common.Nullable; import org.opensearch.common.Priority; import org.opensearch.common.SetOnce; -import org.opensearch.common.Strings; import org.opensearch.common.lifecycle.AbstractLifecycleComponent; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.NamedWriteableRegistry; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Setting; diff --git a/server/src/main/java/org/opensearch/cluster/decommission/DecommissionAttributeMetadata.java b/server/src/main/java/org/opensearch/cluster/decommission/DecommissionAttributeMetadata.java index 8af783bbdc52e..5a9c82a3849e9 100644 --- a/server/src/main/java/org/opensearch/cluster/decommission/DecommissionAttributeMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/decommission/DecommissionAttributeMetadata.java @@ -14,7 +14,7 @@ import org.opensearch.cluster.NamedDiff; import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.metadata.Metadata.Custom; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.common.xcontent.XContentType; diff --git a/server/src/main/java/org/opensearch/cluster/health/ClusterShardHealth.java b/server/src/main/java/org/opensearch/cluster/health/ClusterShardHealth.java index 00a83c85c17be..32b54ac947ebd 100644 --- a/server/src/main/java/org/opensearch/cluster/health/ClusterShardHealth.java +++ b/server/src/main/java/org/opensearch/cluster/health/ClusterShardHealth.java @@ -37,7 +37,7 @@ import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.UnassignedInfo; import org.opensearch.cluster.routing.UnassignedInfo.AllocationStatus; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; diff --git a/server/src/main/java/org/opensearch/cluster/metadata/AliasMetadata.java b/server/src/main/java/org/opensearch/cluster/metadata/AliasMetadata.java index 27ecca0358bd8..5e4de1be71214 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/AliasMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/AliasMetadata.java @@ -260,7 +260,7 @@ public static Diff readDiffFrom(StreamInput in) throws IOExceptio @Override public String toString() { - return org.opensearch.common.Strings.toString(XContentType.JSON, this, true, true); + return Strings.toString(XContentType.JSON, this, true, true); } @Override diff --git a/server/src/main/java/org/opensearch/cluster/metadata/ComponentTemplate.java b/server/src/main/java/org/opensearch/cluster/metadata/ComponentTemplate.java index 52096422248a5..7c5e0f664df4e 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/ComponentTemplate.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/ComponentTemplate.java @@ -35,7 +35,7 @@ import org.opensearch.cluster.AbstractDiffable; import org.opensearch.cluster.Diff; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.common.xcontent.XContentType; diff --git a/server/src/main/java/org/opensearch/cluster/metadata/ComponentTemplateMetadata.java b/server/src/main/java/org/opensearch/cluster/metadata/ComponentTemplateMetadata.java index ce806b2aa1f12..2e700389e4fc9 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/ComponentTemplateMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/ComponentTemplateMetadata.java @@ -36,7 +36,7 @@ import org.opensearch.cluster.Diff; import org.opensearch.cluster.DiffableUtils; import org.opensearch.cluster.NamedDiff; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.common.xcontent.XContentType; diff --git a/server/src/main/java/org/opensearch/cluster/metadata/ComposableIndexTemplate.java b/server/src/main/java/org/opensearch/cluster/metadata/ComposableIndexTemplate.java index 15e5cb5873719..b9b7c132ba2cf 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/ComposableIndexTemplate.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/ComposableIndexTemplate.java @@ -36,7 +36,7 @@ import org.opensearch.cluster.Diff; import org.opensearch.cluster.metadata.DataStream.TimestampField; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; diff --git a/server/src/main/java/org/opensearch/cluster/metadata/ComposableIndexTemplateMetadata.java b/server/src/main/java/org/opensearch/cluster/metadata/ComposableIndexTemplateMetadata.java index d34416c70dc16..b72c0fdf81e41 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/ComposableIndexTemplateMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/ComposableIndexTemplateMetadata.java @@ -36,7 +36,7 @@ import org.opensearch.cluster.Diff; import org.opensearch.cluster.DiffableUtils; import org.opensearch.cluster.NamedDiff; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.ParseField; diff --git a/server/src/main/java/org/opensearch/cluster/metadata/DataStreamMetadata.java b/server/src/main/java/org/opensearch/cluster/metadata/DataStreamMetadata.java index 89fe6e9be2320..c7854355ea5cc 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/DataStreamMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/DataStreamMetadata.java @@ -36,7 +36,7 @@ import org.opensearch.cluster.Diff; import org.opensearch.cluster.DiffableUtils; import org.opensearch.cluster.NamedDiff; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.common.xcontent.XContentType; diff --git a/server/src/main/java/org/opensearch/cluster/metadata/IndexTemplateMetadata.java b/server/src/main/java/org/opensearch/cluster/metadata/IndexTemplateMetadata.java index e430ba5465499..4a89298aa0429 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/IndexTemplateMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/IndexTemplateMetadata.java @@ -35,7 +35,6 @@ import org.opensearch.cluster.AbstractDiffable; import org.opensearch.cluster.Diff; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.collect.MapBuilder; import org.opensearch.common.compress.CompressedXContent; import org.opensearch.core.common.io.stream.StreamInput; @@ -263,7 +262,7 @@ public String toString() { builder.startObject(); Builder.toXContentWithTypes(this, builder, ToXContent.EMPTY_PARAMS); builder.endObject(); - return Strings.toString(builder); + return builder.toString(); } catch (IOException e) { throw new UncheckedIOException(e); } @@ -480,7 +479,7 @@ public static IndexTemplateMetadata fromXContent(XContentParser parser, String t Map mappingSource = MapBuilder.newMapBuilder() .put(mappingType, parser.mapOrdered()) .map(); - builder.putMapping(mappingType, Strings.toString(XContentFactory.jsonBuilder().map(mappingSource))); + builder.putMapping(mappingType, XContentFactory.jsonBuilder().map(mappingSource).toString()); } } } else if ("aliases".equals(currentFieldName)) { @@ -496,7 +495,7 @@ public static IndexTemplateMetadata fromXContent(XContentParser parser, String t Map mapping = parser.mapOrdered(); if (mapping.size() == 1) { String mappingType = mapping.keySet().iterator().next(); - String mappingSource = Strings.toString(XContentFactory.jsonBuilder().map(mapping)); + String mappingSource = XContentFactory.jsonBuilder().map(mapping).toString(); if (mappingSource == null) { // crap, no mapping source, warn? diff --git a/server/src/main/java/org/opensearch/cluster/metadata/MetadataIndexTemplateService.java b/server/src/main/java/org/opensearch/cluster/metadata/MetadataIndexTemplateService.java index 2c4e6bdec95b0..0779f61d97bf0 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataIndexTemplateService.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataIndexTemplateService.java @@ -302,9 +302,11 @@ ClusterState addComponentTemplate( if (stringMappings != null) { Map parsedMappings = MapperService.parseMapping(xContentRegistry, stringMappings); if (parsedMappings.size() > 0) { - stringMappings = org.opensearch.common.Strings.toString( - XContentFactory.jsonBuilder().startObject().field(MapperService.SINGLE_MAPPING_NAME, parsedMappings).endObject() - ); + stringMappings = XContentFactory.jsonBuilder() + .startObject() + .field(MapperService.SINGLE_MAPPING_NAME, parsedMappings) + .endObject() + .toString(); } } @@ -591,9 +593,11 @@ public ClusterState addIndexTemplateV2( if (stringMappings != null) { Map parsedMappings = MapperService.parseMapping(xContentRegistry, stringMappings); if (parsedMappings.size() > 0) { - stringMappings = org.opensearch.common.Strings.toString( - XContentFactory.jsonBuilder().startObject().field(MapperService.SINGLE_MAPPING_NAME, parsedMappings).endObject() - ); + stringMappings = XContentFactory.jsonBuilder() + .startObject() + .field(MapperService.SINGLE_MAPPING_NAME, parsedMappings) + .endObject() + .toString(); } } final Template finalTemplate = new Template( diff --git a/server/src/main/java/org/opensearch/cluster/metadata/RepositoriesMetadata.java b/server/src/main/java/org/opensearch/cluster/metadata/RepositoriesMetadata.java index e654745e8ccdb..9123e4a8d2de3 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/RepositoriesMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/RepositoriesMetadata.java @@ -38,7 +38,7 @@ import org.opensearch.cluster.NamedDiff; import org.opensearch.cluster.metadata.Metadata.Custom; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.common.settings.Settings; diff --git a/server/src/main/java/org/opensearch/cluster/metadata/Template.java b/server/src/main/java/org/opensearch/cluster/metadata/Template.java index 45d11dd9250e8..8e367c71ed166 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/Template.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/Template.java @@ -34,19 +34,18 @@ import org.opensearch.cluster.AbstractDiffable; import org.opensearch.common.Nullable; -import org.opensearch.core.ParseField; -import org.opensearch.common.Strings; import org.opensearch.common.compress.CompressedXContent; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.xcontent.XContentHelper; +import org.opensearch.core.ParseField; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; -import org.opensearch.common.settings.Settings; import org.opensearch.core.xcontent.ConstructingObjectParser; +import org.opensearch.core.xcontent.MediaTypeRegistry; import org.opensearch.core.xcontent.ToXContentObject; import org.opensearch.core.xcontent.XContentBuilder; -import org.opensearch.common.xcontent.XContentFactory; -import org.opensearch.common.xcontent.XContentHelper; import org.opensearch.core.xcontent.XContentParser; -import org.opensearch.common.xcontent.XContentType; import org.opensearch.index.mapper.MapperService; import java.io.IOException; @@ -77,7 +76,7 @@ public class Template extends AbstractDiffable