From 68a4d36891887414684f264185e7247da26dcee4 Mon Sep 17 00:00:00 2001 From: Nick Knize Date: Wed, 5 Jul 2023 19:51:51 -0500 Subject: [PATCH] [Refactor] OpenSearchException and ExceptionsHelper foundation to base class (#7508) (#8460) Creates new BaseOpenSearchException and BaseExceptionsHelper in core library as a step to moving OpenSearch exception mechanisms from the server module to core library. This is a move to support serverless and cloud native capabilities without requiring the server module for core opensearch indexing, search, and compute capabilities. Signed-off-by: Nicholas Walter Knize --- CHANGELOG.md | 1 + .../client/ClusterRequestConverters.java | 2 +- .../client/IndicesRequestConverters.java | 2 +- .../opensearch/client/RequestConverters.java | 2 +- .../client/SnapshotRequestConverters.java | 2 +- .../opensearch/client/core/CountRequest.java | 2 +- .../client/indices/AnalyzeRequest.java | 2 +- .../ComponentTemplatesExistRequest.java | 2 +- .../ComposableIndexTemplateExistRequest.java | 2 +- .../client/indices/DetailAnalyzeResponse.java | 2 +- .../indices/GetFieldMappingsRequest.java | 2 +- .../indices/GetIndexTemplatesRequest.java | 2 +- .../client/indices/GetMappingsRequest.java | 2 +- .../indices/PutComponentTemplateRequest.java | 2 +- .../PutComposableIndexTemplateRequest.java | 2 +- .../indices/SimulateIndexTemplateRequest.java | 2 +- .../forbidden/rest-high-level-signatures.txt | 2 +- .../java/org/opensearch/client/CrudIT.java | 2 +- .../opensearch/client/IndicesClientIT.java | 2 +- .../client/IndicesRequestConvertersTests.java | 2 +- .../client/RequestConvertersTests.java | 8 +- .../indices/CloseIndexRequestTests.java | 2 +- .../opensearch/bootstrap/JarHellTests.java | 2 +- .../org/opensearch/BaseExceptionsHelper.java | 283 +++++++++ .../opensearch/BaseOpenSearchException.java | 437 ++++++++++++++ .../OpenSearchWrapperException.java | 0 .../org/opensearch/core/common/Strings.java | 518 ++++++++++++++++ .../common/logging/LoggerMessageFormat.java | 2 +- .../core/common/logging/package-info.java | 18 + .../opensearch/core/common/package-info.java | 10 + .../opensearch/core/common/StringsTests.java | 28 + .../common/MultiplexerTokenFilterFactory.java | 2 +- .../PatternReplaceCharFilterFactory.java | 2 +- .../common/SnowballTokenFilterFactory.java | 2 +- .../common/StemmerTokenFilterFactory.java | 2 +- .../geogrid/GeoTileGridParserTests.java | 4 +- .../UserAgentProcessorFactoryTests.java | 2 +- .../mustache/MultiSearchTemplateResponse.java | 4 +- .../MultiSearchTemplateResponseTests.java | 2 +- .../index/rankeval/RankEvalRequestIT.java | 4 +- .../index/rankeval/RankEvalRequest.java | 2 +- .../index/rankeval/RankEvalResponse.java | 3 +- .../index/rankeval/RatedDocument.java | 4 +- .../index/rankeval/RestRankEvalAction.java | 2 +- .../index/reindex/ReindexValidator.java | 2 +- .../opensearch/index/reindex/Reindexer.java | 2 +- .../remote/RemoteScrollableHitSource.java | 2 +- .../reindex/AsyncBulkByScrollActionTests.java | 3 +- .../management/AzureComputeServiceImpl.java | 2 +- .../azure/classic/AzureSeedHostsProvider.java | 2 +- .../discovery/ec2/AbstractAwsTestCase.java | 2 +- .../discovery/ec2/AwsEc2ServiceImpl.java | 2 +- .../discovery/ec2/Ec2NetworkTests.java | 2 +- .../cloud/gce/network/GceNameResolver.java | 2 +- .../discovery/gce/GceSeedHostsProvider.java | 2 +- .../discovery/gce/GceMockUtils.java | 2 +- .../discovery/gce/GceNetworkTests.java | 2 +- .../attachment/AttachmentProcessor.java | 2 +- .../AnnotatedTextHighlighterTests.java | 2 +- .../AzureStorageCleanupThirdPartyTests.java | 2 +- .../repositories/azure/AzureRepository.java | 2 +- .../azure/AzureStorageSettings.java | 3 +- .../repositories/azure/ProxySettings.java | 2 +- .../azure/AzureBlobContainerRetriesTests.java | 2 +- .../azure/AzureStorageServiceTests.java | 2 +- .../GoogleCloudStorageThirdPartyTests.java | 2 +- .../gcs/GoogleCloudStorageClientSettings.java | 2 +- .../gcs/GoogleCloudStorageRepository.java | 2 +- .../gcs/GoogleCloudStorageService.java | 2 +- .../repositories/gcs/ProxySettings.java | 2 +- ...CloudStorageBlobContainerRetriesTests.java | 2 +- .../repositories/hdfs/HdfsRepository.java | 2 +- .../repositories/s3/ProxySettings.java | 2 +- .../repositories/s3/S3BlobContainer.java | 2 +- .../repositories/s3/S3ClientSettings.java | 2 +- .../repositories/s3/S3Repository.java | 2 +- .../opensearch/repositories/s3/S3Service.java | 2 +- .../qa/die_with_dignity/DieWithDignityIT.java | 2 +- .../org/opensearch/backwards/SearchingIT.java | 2 +- .../opensearch/action/RejectionActionIT.java | 4 +- .../action/admin/ReloadSecureSettingsIT.java | 2 +- .../cluster/node/tasks/AbstractTasksIT.java | 2 +- .../action/search/TransportSearchIT.java | 10 +- .../action/termvectors/GetTermVectorsIT.java | 2 +- .../org/opensearch/blocks/SimpleBlocksIT.java | 6 +- .../cluster/ClusterInfoServiceIT.java | 2 +- .../cluster/SimpleClusterStateIT.java | 2 +- .../allocation/FilteringAllocationIT.java | 2 +- .../StableClusterManagerDisruptionIT.java | 2 +- .../indices/IndicesLifecycleListenerIT.java | 2 +- .../indices/IndicesOptionsIntegrationIT.java | 2 +- .../breaker/CircuitBreakerServiceIT.java | 4 +- .../indices/state/CloseIndexIT.java | 4 +- .../indices/state/ReopenWhileClosingIT.java | 2 +- .../AggregationsIntegrationIT.java | 4 +- .../aggregations/bucket/DateHistogramIT.java | 6 +- .../aggregations/bucket/HistogramIT.java | 6 +- .../bucket/terms/StringTermsIT.java | 5 +- .../aggregations/pipeline/DerivativeIT.java | 4 +- .../pipeline/ExtendedStatsBucketIT.java | 4 +- .../pipeline/PercentilesBucketIT.java | 6 +- .../search/preference/SearchPreferenceIT.java | 2 +- .../snapshots/ConcurrentSnapshotsIT.java | 2 +- .../DedicatedClusterSnapshotRestoreIT.java | 2 +- .../SharedClusterSnapshotRestoreIT.java | 4 +- .../snapshots/SnapshotStatusApisIT.java | 2 +- .../java/org/opensearch/ExceptionsHelper.java | 79 +-- .../org/opensearch/OpenSearchException.java | 461 +------------- .../action/TaskOperationFailure.java | 3 +- .../AddVotingConfigExclusionsRequest.java | 2 +- .../get/GetDecommissionStateRequest.java | 2 +- .../awareness/put/DecommissionRequest.java | 2 +- .../health/TransportClusterHealthAction.java | 2 +- .../NodesReloadSecureSettingsResponse.java | 4 +- .../restore/RestoreRemoteStoreRequest.java | 4 +- .../get/GetRepositoriesRequest.java | 2 +- .../TransportClusterRerouteAction.java | 2 +- .../shards/ClusterSearchShardsRequest.java | 2 +- .../clone/CloneSnapshotRequestBuilder.java | 2 +- .../create/CreateSnapshotRequest.java | 4 +- .../snapshots/get/GetSnapshotsRequest.java | 2 +- .../restore/RestoreSnapshotRequest.java | 4 +- .../status/SnapshotsStatusRequest.java | 2 +- .../TransportSnapshotsStatusAction.java | 2 +- .../cluster/state/ClusterStateRequest.java | 2 +- .../cluster/stats/ClusterStatsNodes.java | 2 +- .../storedscripts/PutStoredScriptRequest.java | 3 +- .../indices/alias/IndicesAliasesRequest.java | 10 +- .../indices/alias/get/GetAliasesRequest.java | 2 +- .../cache/clear/ClearIndicesCacheRequest.java | 2 +- .../indices/close/CloseIndexResponse.java | 4 +- .../find/FindDanglingIndexRequest.java | 2 +- .../list/ListDanglingIndicesRequest.java | 2 +- .../datastream/CreateDataStreamAction.java | 2 +- .../datastream/DeleteDataStreamAction.java | 2 +- .../exists/indices/IndicesExistsRequest.java | 2 +- .../get/GetFieldMappingsIndexRequest.java | 2 +- .../mapping/get/GetFieldMappingsRequest.java | 2 +- .../mapping/put/PutMappingRequest.java | 6 +- .../readonly/AddIndexBlockResponse.java | 4 +- .../indices/recovery/RecoveryRequest.java | 2 +- .../SegmentReplicationStatsRequest.java | 2 +- .../indices/resolve/ResolveIndexAction.java | 4 +- .../rollover/MetadataRolloverService.java | 2 +- .../segments/IndicesSegmentsRequest.java | 2 +- .../indices/segments/PitSegmentsRequest.java | 2 +- .../segments/TransportPitSegmentsAction.java | 2 +- .../settings/get/GetSettingsRequest.java | 2 +- .../shards/IndicesShardStoresRequest.java | 2 +- .../shards/IndicesShardStoresResponse.java | 4 +- .../admin/indices/stats/CommonStatsFlags.java | 2 +- .../get/GetIndexTemplatesRequest.java | 2 +- .../post/SimulateIndexTemplateRequest.java | 2 +- .../put/PutComponentTemplateAction.java | 2 +- .../put/PutComposableIndexTemplateAction.java | 2 +- .../upgrade/get/UpgradeStatusRequest.java | 2 +- .../validate/query/ValidateQueryRequest.java | 2 +- .../action/bulk/BulkItemResponse.java | 5 +- .../opensearch/action/bulk/BulkRequest.java | 4 +- .../action/bulk/TransportBulkAction.java | 4 +- .../action/bulk/TransportShardBulkAction.java | 4 +- .../action/delete/DeleteRequest.java | 2 +- .../action/explain/ExplainRequest.java | 2 +- .../action/explain/ExplainRequestBuilder.java | 2 +- .../fieldcaps/FieldCapabilitiesRequest.java | 2 +- .../TransportFieldCapabilitiesAction.java | 2 +- ...TransportFieldCapabilitiesIndexAction.java | 2 +- .../org/opensearch/action/get/GetRequest.java | 2 +- .../action/get/GetRequestBuilder.java | 2 +- .../action/get/MultiGetResponse.java | 3 +- .../opensearch/action/index/IndexRequest.java | 3 +- .../action/ingest/GetPipelineRequest.java | 2 +- .../ingest/SimulateDocumentBaseResult.java | 3 +- .../ingest/SimulateProcessorResult.java | 5 +- .../search/AbstractSearchAsyncAction.java | 6 +- .../action/search/ClearScrollController.java | 2 +- .../action/search/CreatePitController.java | 2 +- .../action/search/CreatePitRequest.java | 2 +- .../search/GetSearchPipelineRequest.java | 2 +- .../action/search/MultiSearchResponse.java | 3 +- .../opensearch/action/search/PitService.java | 2 +- .../action/search/SearchContextId.java | 2 +- .../search/SearchPhaseExecutionException.java | 27 +- .../action/search/SearchRequest.java | 2 +- .../action/search/ShardSearchFailure.java | 20 +- .../action/search/TransportSearchAction.java | 2 +- .../action/support/AutoCreateIndex.java | 2 +- .../DefaultShardOperationFailedException.java | 3 +- .../action/support/TransportActions.java | 4 +- .../info/ClusterInfoRequest.java | 2 +- .../replication/ReplicationOperation.java | 5 +- .../replication/ReplicationResponse.java | 6 +- .../shard/TransportSingleShardAction.java | 2 +- .../support/tasks/BaseTasksRequest.java | 2 +- .../termvectors/MultiTermVectorsResponse.java | 4 +- .../action/update/UpdateRequest.java | 4 +- .../org/opensearch/cluster/ClusterState.java | 4 +- .../action/shard/ShardStateAction.java | 3 +- .../decommission/DecommissionHelper.java | 2 +- .../cluster/metadata/AliasAction.java | 2 +- .../cluster/metadata/AliasMetadata.java | 6 +- .../cluster/metadata/AliasValidator.java | 4 +- .../cluster/metadata/IndexAbstraction.java | 2 +- .../cluster/metadata/IndexMetadata.java | 3 +- .../metadata/IndexNameExpressionResolver.java | 2 +- .../opensearch/cluster/metadata/Metadata.java | 6 +- .../metadata/MetadataCreateIndexService.java | 13 +- .../metadata/MetadataIndexAliasesService.java | 2 +- .../metadata/MetadataIndexStateService.java | 2 +- .../MetadataIndexTemplateService.java | 17 +- .../metadata/MetadataMappingService.java | 2 +- .../cluster/node/DiscoveryNodeFilters.java | 2 +- .../cluster/node/DiscoveryNodes.java | 2 +- .../cluster/routing/OperationRouting.java | 2 +- .../cluster/routing/UnassignedInfo.java | 4 +- .../allocation/DiskThresholdMonitor.java | 2 +- .../routing/allocation/FailedShard.java | 4 +- .../allocation/NodeAllocationResult.java | 4 +- .../decider/AwarenessAllocationDecider.java | 2 +- .../decider/SameShardAllocationDecider.java | 2 +- .../java/org/opensearch/common/Strings.java | 567 +----------------- .../java/org/opensearch/common/Table.java | 1 + .../NotSerializableExceptionWrapper.java | 5 +- .../common/io/stream/StreamInput.java | 2 +- .../java/org/opensearch/common/joda/Joda.java | 2 +- .../common/logging/DeprecatedMessage.java | 2 +- .../common/logging/HeaderWarning.java | 1 + .../common/logging/OpenSearchJsonLayout.java | 2 +- .../OpenSearchMessageFieldConverter.java | 2 +- .../org/opensearch/common/lucene/Lucene.java | 2 +- .../lucene/search/MoreLikeThisQuery.java | 2 +- .../common/lucene/search/Queries.java | 4 +- .../lucene/search/function/Functions.java | 6 +- .../search/function/WeightFactorFunction.java | 2 +- .../org/opensearch/common/regex/Regex.java | 6 +- .../opensearch/common/settings/Settings.java | 6 +- .../common/settings/SettingsFilter.java | 2 +- .../opensearch/common/time/DateFormatter.java | 2 +- .../common/time/DateFormatters.java | 2 +- .../common/time/JavaDateFormatter.java | 2 +- .../common/time/JavaDateMathParser.java | 2 +- .../common/util/CollectionUtils.java | 2 +- .../common/xcontent/XContentParserUtils.java | 2 +- .../xcontent/support/XContentMapValues.java | 2 +- .../extensions/ExtensionsManager.java | 2 +- .../rest/RestInitializeExtensionAction.java | 2 +- .../opensearch/gateway/AsyncShardFetch.java | 4 +- .../http/AbstractHttpServerTransport.java | 2 +- .../java/org/opensearch/http/CorsHandler.java | 4 +- .../java/org/opensearch/http/HttpTracer.java | 2 +- .../main/java/org/opensearch/index/Index.java | 4 +- .../org/opensearch/index/IndexSettings.java | 4 +- .../org/opensearch/index/SearchSlowLog.java | 2 +- .../opensearch/index/analysis/Analysis.java | 2 +- .../index/engine/InternalEngine.java | 6 +- .../index/mapper/DocumentParser.java | 2 +- .../index/mapper/SourceFieldMapper.java | 4 +- .../index/query/BaseTermQueryBuilder.java | 2 +- .../index/query/CommonTermsQueryBuilder.java | 2 +- .../index/query/ExistsQueryBuilder.java | 2 +- .../query/FieldMaskingSpanQueryBuilder.java | 2 +- .../index/query/FuzzyQueryBuilder.java | 2 +- .../index/query/GeoDistanceQueryBuilder.java | 2 +- .../index/query/GeoPolygonQueryBuilder.java | 2 +- .../index/query/IdsQueryBuilder.java | 4 +- .../query/MatchBoolPrefixQueryBuilder.java | 2 +- .../index/query/MatchPhraseQueryBuilder.java | 2 +- .../index/query/MoreLikeThisQueryBuilder.java | 3 +- .../index/query/MultiMatchQueryBuilder.java | 2 +- .../index/query/PrefixQueryBuilder.java | 2 +- .../index/query/RangeQueryBuilder.java | 2 +- .../opensearch/index/query/RegexpFlag.java | 2 +- .../index/query/RegexpQueryBuilder.java | 2 +- .../index/query/SimpleQueryStringBuilder.java | 2 +- .../index/query/SimpleQueryStringFlag.java | 4 +- .../index/query/SpanNearQueryBuilder.java | 2 +- .../index/query/TermsQueryBuilder.java | 2 +- .../index/query/WildcardQueryBuilder.java | 2 +- .../index/query/WrapperQueryBuilder.java | 2 +- .../index/reindex/BulkByScrollTask.java | 3 +- .../index/reindex/ReindexRequest.java | 2 +- .../index/reindex/ScrollableHitSource.java | 6 +- .../opensearch/index/shard/StoreRecovery.java | 4 +- .../index/termvectors/TermVectorsService.java | 2 +- .../opensearch/index/translog/Translog.java | 2 +- .../indices/recovery/MultiFileWriter.java | 2 +- .../recovery/PeerRecoveryTargetService.java | 6 +- .../recovery/RetryableTransportClient.java | 6 +- .../SegmentReplicationTargetService.java | 4 +- .../replication/common/ReplicationTarget.java | 4 +- .../org/opensearch/ingest/IngestDocument.java | 2 +- .../org/opensearch/plugins/ActionPlugin.java | 2 +- .../org/opensearch/plugins/PluginInfo.java | 2 +- .../opensearch/plugins/PluginsService.java | 2 +- .../repositories/RepositoriesService.java | 2 +- .../repositories/fs/FsRepository.java | 2 +- .../opensearch/rest/AbstractRestChannel.java | 2 +- .../opensearch/rest/BytesRestResponse.java | 19 +- .../rest/DeprecationRestHandler.java | 2 +- .../org/opensearch/rest/RestController.java | 2 +- .../java/org/opensearch/rest/RestRequest.java | 2 +- .../opensearch/rest/RestRequestFilter.java | 4 +- .../java/org/opensearch/rest/RestUtils.java | 2 +- .../action/RestFieldCapabilitiesAction.java | 2 +- .../RestAddVotingConfigExclusionAction.java | 2 +- .../admin/cluster/RestCancelTasksAction.java | 2 +- .../cluster/RestClusterHealthAction.java | 2 +- .../cluster/RestClusterRerouteAction.java | 2 +- .../RestClusterSearchShardsAction.java | 2 +- .../admin/cluster/RestClusterStateAction.java | 2 +- .../cluster/RestDeleteSnapshotAction.java | 2 +- .../cluster/RestGetRepositoriesAction.java | 2 +- .../admin/cluster/RestGetSnapshotsAction.java | 2 +- .../admin/cluster/RestListTasksAction.java | 2 +- .../cluster/RestNodesHotThreadsAction.java | 2 +- .../admin/cluster/RestNodesInfoAction.java | 2 +- .../admin/cluster/RestNodesStatsAction.java | 2 +- .../admin/cluster/RestNodesUsageAction.java | 2 +- .../RestReloadSecureSettingsAction.java | 2 +- .../cluster/RestSnapshotsStatusAction.java | 2 +- .../indices/RestAddIndexBlockAction.java | 2 +- .../indices/RestClearIndicesCacheAction.java | 2 +- .../admin/indices/RestCloseIndexAction.java | 2 +- .../indices/RestDataStreamsStatsAction.java | 2 +- .../indices/RestDeleteDataStreamAction.java | 2 +- .../admin/indices/RestDeleteIndexAction.java | 2 +- .../action/admin/indices/RestFlushAction.java | 2 +- .../admin/indices/RestForceMergeAction.java | 2 +- .../admin/indices/RestGetAliasesAction.java | 2 +- .../indices/RestGetDataStreamsAction.java | 2 +- .../indices/RestGetFieldMappingAction.java | 2 +- .../indices/RestGetIndexTemplateAction.java | 2 +- .../admin/indices/RestGetIndicesAction.java | 2 +- .../admin/indices/RestGetMappingAction.java | 2 +- .../admin/indices/RestGetSettingsAction.java | 2 +- .../indices/RestIndexDeleteAliasesAction.java | 2 +- .../indices/RestIndexPutAliasAction.java | 2 +- .../indices/RestIndicesSegmentsAction.java | 2 +- .../indices/RestIndicesShardStoresAction.java | 2 +- .../admin/indices/RestIndicesStatsAction.java | 2 +- .../admin/indices/RestOpenIndexAction.java | 2 +- .../indices/RestPutIndexTemplateAction.java | 2 +- .../admin/indices/RestPutMappingAction.java | 2 +- .../admin/indices/RestRecoveryAction.java | 2 +- .../admin/indices/RestRefreshAction.java | 2 +- .../admin/indices/RestResolveIndexAction.java | 2 +- .../admin/indices/RestSyncedFlushAction.java | 2 +- .../indices/RestUpdateSettingsAction.java | 2 +- .../admin/indices/RestUpgradeAction.java | 2 +- .../indices/RestUpgradeStatusAction.java | 2 +- .../indices/RestValidateQueryAction.java | 2 +- .../rest/action/cat/RestAliasAction.java | 2 +- .../rest/action/cat/RestAllocationAction.java | 2 +- .../action/cat/RestCatRecoveryAction.java | 2 +- .../cat/RestCatSegmentReplicationAction.java | 2 +- .../rest/action/cat/RestCountAction.java | 2 +- .../rest/action/cat/RestIndicesAction.java | 2 +- .../rest/action/cat/RestSegmentsAction.java | 2 +- .../rest/action/cat/RestShardsAction.java | 2 +- .../opensearch/rest/action/cat/RestTable.java | 2 +- .../rest/action/document/RestGetAction.java | 2 +- .../action/document/RestMultiGetAction.java | 2 +- .../document/RestMultiTermVectorsAction.java | 2 +- .../document/RestTermVectorsAction.java | 2 +- .../action/ingest/RestGetPipelineAction.java | 2 +- .../action/search/RestClearScrollAction.java | 2 +- .../rest/action/search/RestCountAction.java | 2 +- .../action/search/RestCreatePitAction.java | 2 +- .../rest/action/search/RestExplainAction.java | 2 +- .../search/RestGetSearchPipelineAction.java | 2 +- .../action/search/RestMultiSearchAction.java | 2 +- .../rest/action/search/RestSearchAction.java | 2 +- .../org/opensearch/script/ScriptService.java | 2 +- .../histogram/DateHistogramInterval.java | 2 +- .../bucket/histogram/DateIntervalWrapper.java | 2 +- .../metrics/TopHitsAggregationBuilder.java | 2 +- .../MovFnPipelineAggregationBuilder.java | 2 +- .../aggregations/support/AggregationPath.java | 2 +- .../support/MultiTermsValuesSourceConfig.java | 2 +- .../support/MultiValuesSourceFieldConfig.java | 2 +- .../search/builder/SearchSourceBuilder.java | 8 +- .../search/collapse/CollapseBuilder.java | 2 +- .../search/fetch/StoredFieldsContext.java | 2 +- .../fetch/subphase/FetchSourceContext.java | 2 +- .../highlight/UnifiedHighlighter.java | 2 +- .../search/internal/AliasFilter.java | 2 +- .../search/internal/ShardSearchRequest.java | 2 +- .../search/query/QuerySearchRequest.java | 2 +- .../opensearch/search/slice/SliceBuilder.java | 2 +- .../DirectCandidateGeneratorBuilder.java | 4 +- .../snapshots/SnapshotsService.java | 6 +- .../tasks/TaskCancellationService.java | 6 +- .../java/org/opensearch/tasks/TaskId.java | 2 +- .../org/opensearch/tasks/TaskManager.java | 5 +- .../java/org/opensearch/tasks/TaskResult.java | 4 +- .../opensearch/tasks/TaskResultsService.java | 4 +- .../transport/ProxyConnectionStrategy.java | 2 +- .../transport/RemoteConnectionStrategy.java | 2 +- .../transport/SniffConnectionStrategy.java | 2 +- .../opensearch/transport/TcpTransport.java | 2 +- .../transport/TransportService.java | 4 +- .../org/opensearch/ExceptionsHelperTests.java | 6 +- .../opensearch/OpenSearchExceptionTests.java | 54 +- ...AddVotingConfigExclusionsRequestTests.java | 2 +- ...tAddVotingConfigExclusionsActionTests.java | 2 +- .../VotingConfigExclusionsHelperTests.java | 2 +- .../health/ClusterHealthRequestTests.java | 2 +- .../ClusterSearchShardsResponseTests.java | 2 +- .../DeleteDataStreamRequestTests.java | 2 +- .../resolve/ResolveIndexResponseTests.java | 2 +- .../indices/resolve/ResolveIndexTests.java | 2 +- .../bulk/TransportBulkActionTookTests.java | 2 +- .../CanMatchPreFilterSearchPhaseTests.java | 2 +- .../search/MultiSearchResponseTests.java | 2 +- .../action/search/SearchAsyncActionTests.java | 2 +- .../search/SearchPhaseControllerTests.java | 2 +- .../SearchQueryThenFetchAsyncActionTests.java | 2 +- .../action/search/SearchRequestTests.java | 2 +- .../search/TransportSearchActionTests.java | 2 +- .../client/AbstractClientHeadersTestCase.java | 4 +- .../IndexNameExpressionResolverTests.java | 2 +- .../MetadataIndexStateServiceTests.java | 2 +- .../org/opensearch/common/StringsTests.java | 14 - .../common/geo/GeoJsonParserTests.java | 4 +- .../org/opensearch/http/CorsHandlerTests.java | 4 +- .../index/engine/InternalEngineTests.java | 2 +- .../AbstractFieldDataImplTestCase.java | 2 +- .../index/mapper/DynamicMappingTests.java | 1 - .../index/query/InnerHitBuilderTests.java | 6 +- ...ulkByScrollTaskStatusOrExceptionTests.java | 2 +- .../reindex/BulkByScrollTaskStatusTests.java | 2 +- .../RemoteStoreRefreshListenerTests.java | 2 +- .../CustomUnifiedHighlighterTests.java | 2 +- .../opensearch/monitor/jvm/JvmStatsTests.java | 2 +- .../rest/BytesRestResponseTests.java | 4 +- ...stAddVotingConfigExclusionActionTests.java | 2 +- .../opensearch/search/SearchServiceTests.java | 2 +- .../AggregatorFactoriesTests.java | 8 +- .../pipeline/DerivativeAggregatorTests.java | 4 +- .../fetch/subphase/FetchSourcePhaseTests.java | 4 +- .../search/query/QuerySearchResultTests.java | 2 +- .../search/slice/SliceBuilderTests.java | 6 +- .../tasks/CancelTasksResponseTests.java | 2 +- .../tasks/ListTasksResponseTests.java | 2 +- .../RemoteClusterConnectionTests.java | 2 +- .../transport/RemoteClusterServiceTests.java | 2 +- .../SniffConnectionStrategyTests.java | 2 +- .../transport/TransportActionProxyTests.java | 4 +- .../gcs/GoogleCloudStorageHttpHandler.java | 2 +- .../bootstrap/BootstrapForTesting.java | 2 +- .../AbstractThirdPartyRepositoryTestCase.java | 2 +- .../blobstore/BlobStoreTestUtil.java | 2 +- ...chMockAPIBasedRepositoryIntegTestCase.java | 2 +- .../test/AbstractSerializingTestCase.java | 2 +- .../test/AbstractXContentTestCase.java | 2 +- .../opensearch/test/InternalTestCluster.java | 4 +- .../test/OpenSearchIntegTestCase.java | 16 +- .../test/OpenSearchSingleNodeTestCase.java | 4 +- .../junit/listeners/ReproduceInfoPrinter.java | 2 +- .../test/rest/OpenSearchRestTestCase.java | 4 +- .../OpenSearchClientYamlSuiteTestCase.java | 2 +- .../org/opensearch/test/rest/yaml/Stash.java | 2 +- .../test/rest/yaml/section/DoSection.java | 2 +- .../test/rest/yaml/section/SkipSection.java | 6 +- .../AbstractSimpleTransportTestCase.java | 11 +- 465 files changed, 2003 insertions(+), 1743 deletions(-) create mode 100644 libs/core/src/main/java/org/opensearch/BaseExceptionsHelper.java create mode 100644 libs/core/src/main/java/org/opensearch/BaseOpenSearchException.java rename {server => libs/core}/src/main/java/org/opensearch/OpenSearchWrapperException.java (100%) create mode 100644 libs/core/src/main/java/org/opensearch/core/common/Strings.java rename {server/src/main/java/org/opensearch => libs/core/src/main/java/org/opensearch/core}/common/logging/LoggerMessageFormat.java (99%) create mode 100644 libs/core/src/main/java/org/opensearch/core/common/logging/package-info.java create mode 100644 libs/core/src/main/java/org/opensearch/core/common/package-info.java create mode 100644 libs/core/src/test/java/org/opensearch/core/common/StringsTests.java diff --git a/CHANGELOG.md b/CHANGELOG.md index 1f133b71e1fad..eace57377dccd 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -83,6 +83,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - [Refactor] remaining ImmutableOpenMap usage to j.u.Map and remove class ([#7309](https://github.com/opensearch-project/OpenSearch/pull/7309)) - [Refactor] Version and LegacyESVersion from server module to core lib ([#7328](https://github.com/opensearch-project/OpenSearch/pull/7328)) - [Refactor] Stream Reader and Write Generics ([#7465](https://github.com/opensearch-project/OpenSearch/pull/7465)) +- [Refactor] OpenSearchException and ExceptionsHelper foundation to base class ([#7508](https://github.com/opensearch-project/OpenSearch/pull/7508)) - Move ZSTD compression codecs out of the sandbox ([#7908](https://github.com/opensearch-project/OpenSearch/pull/7908)) ### Deprecated diff --git a/client/rest-high-level/src/main/java/org/opensearch/client/ClusterRequestConverters.java b/client/rest-high-level/src/main/java/org/opensearch/client/ClusterRequestConverters.java index 9a386fe732adb..8014eb946b144 100644 --- a/client/rest-high-level/src/main/java/org/opensearch/client/ClusterRequestConverters.java +++ b/client/rest-high-level/src/main/java/org/opensearch/client/ClusterRequestConverters.java @@ -45,7 +45,7 @@ import org.opensearch.client.indices.DeleteComponentTemplateRequest; import org.opensearch.client.indices.GetComponentTemplatesRequest; import org.opensearch.client.indices.PutComponentTemplateRequest; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import java.io.IOException; diff --git a/client/rest-high-level/src/main/java/org/opensearch/client/IndicesRequestConverters.java b/client/rest-high-level/src/main/java/org/opensearch/client/IndicesRequestConverters.java index 2d76ebeebe4ef..57c21284e7fd8 100644 --- a/client/rest-high-level/src/main/java/org/opensearch/client/IndicesRequestConverters.java +++ b/client/rest-high-level/src/main/java/org/opensearch/client/IndicesRequestConverters.java @@ -73,7 +73,7 @@ import org.opensearch.client.indices.SimulateIndexTemplateRequest; import org.opensearch.client.indices.rollover.RolloverRequest; import org.opensearch.cluster.metadata.IndexMetadata; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import java.io.IOException; import java.util.Locale; 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 03b1a2ebbff74..a65dd5ebb1092 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 @@ -72,7 +72,6 @@ import org.opensearch.cluster.health.ClusterHealthStatus; import org.opensearch.common.Nullable; import org.opensearch.common.Priority; -import org.opensearch.common.Strings; import org.opensearch.common.SuppressForbidden; import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.lucene.uid.Versions; @@ -80,6 +79,7 @@ import org.opensearch.common.util.CollectionUtils; import org.opensearch.common.xcontent.XContentHelper; import org.opensearch.common.xcontent.XContentType; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.DeprecationHandler; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.core.xcontent.ToXContent; diff --git a/client/rest-high-level/src/main/java/org/opensearch/client/SnapshotRequestConverters.java b/client/rest-high-level/src/main/java/org/opensearch/client/SnapshotRequestConverters.java index fcc1de58883b9..357bb75d5fadb 100644 --- a/client/rest-high-level/src/main/java/org/opensearch/client/SnapshotRequestConverters.java +++ b/client/rest-high-level/src/main/java/org/opensearch/client/SnapshotRequestConverters.java @@ -47,7 +47,7 @@ import org.opensearch.action.admin.cluster.snapshots.get.GetSnapshotsRequest; import org.opensearch.action.admin.cluster.snapshots.restore.RestoreSnapshotRequest; import org.opensearch.action.admin.cluster.snapshots.status.SnapshotsStatusRequest; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import java.io.IOException; diff --git a/client/rest-high-level/src/main/java/org/opensearch/client/core/CountRequest.java b/client/rest-high-level/src/main/java/org/opensearch/client/core/CountRequest.java index ec81c5f343b92..22be6c2e7ac9d 100644 --- a/client/rest-high-level/src/main/java/org/opensearch/client/core/CountRequest.java +++ b/client/rest-high-level/src/main/java/org/opensearch/client/core/CountRequest.java @@ -36,7 +36,7 @@ import org.opensearch.action.ActionRequestValidationException; import org.opensearch.action.IndicesRequest; import org.opensearch.action.support.IndicesOptions; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ToXContentObject; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.index.query.QueryBuilder; diff --git a/client/rest-high-level/src/main/java/org/opensearch/client/indices/AnalyzeRequest.java b/client/rest-high-level/src/main/java/org/opensearch/client/indices/AnalyzeRequest.java index 87d24c11f95c1..a2b69962c392e 100644 --- a/client/rest-high-level/src/main/java/org/opensearch/client/indices/AnalyzeRequest.java +++ b/client/rest-high-level/src/main/java/org/opensearch/client/indices/AnalyzeRequest.java @@ -33,8 +33,8 @@ package org.opensearch.client.indices; import org.opensearch.client.Validatable; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ToXContentFragment; 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/indices/ComponentTemplatesExistRequest.java b/client/rest-high-level/src/main/java/org/opensearch/client/indices/ComponentTemplatesExistRequest.java index 4a518d851cdd8..98395626ef392 100644 --- a/client/rest-high-level/src/main/java/org/opensearch/client/indices/ComponentTemplatesExistRequest.java +++ b/client/rest-high-level/src/main/java/org/opensearch/client/indices/ComponentTemplatesExistRequest.java @@ -32,7 +32,7 @@ package org.opensearch.client.indices; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; /** * A request to check for the existence of component templates diff --git a/client/rest-high-level/src/main/java/org/opensearch/client/indices/ComposableIndexTemplateExistRequest.java b/client/rest-high-level/src/main/java/org/opensearch/client/indices/ComposableIndexTemplateExistRequest.java index 4efe0e6b33aac..a9297d0dd0559 100644 --- a/client/rest-high-level/src/main/java/org/opensearch/client/indices/ComposableIndexTemplateExistRequest.java +++ b/client/rest-high-level/src/main/java/org/opensearch/client/indices/ComposableIndexTemplateExistRequest.java @@ -32,7 +32,7 @@ package org.opensearch.client.indices; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; /** * A request to check for the existence of index templates diff --git a/client/rest-high-level/src/main/java/org/opensearch/client/indices/DetailAnalyzeResponse.java b/client/rest-high-level/src/main/java/org/opensearch/client/indices/DetailAnalyzeResponse.java index 5cb39d856d42a..c2ac8169b0a4e 100644 --- a/client/rest-high-level/src/main/java/org/opensearch/client/indices/DetailAnalyzeResponse.java +++ b/client/rest-high-level/src/main/java/org/opensearch/client/indices/DetailAnalyzeResponse.java @@ -32,8 +32,8 @@ package org.opensearch.client.indices; +import org.opensearch.core.common.Strings; import org.opensearch.core.ParseField; -import org.opensearch.common.Strings; import org.opensearch.core.xcontent.ConstructingObjectParser; import org.opensearch.core.xcontent.XContentParser; diff --git a/client/rest-high-level/src/main/java/org/opensearch/client/indices/GetFieldMappingsRequest.java b/client/rest-high-level/src/main/java/org/opensearch/client/indices/GetFieldMappingsRequest.java index 19792001170f3..70eae3a360005 100644 --- a/client/rest-high-level/src/main/java/org/opensearch/client/indices/GetFieldMappingsRequest.java +++ b/client/rest-high-level/src/main/java/org/opensearch/client/indices/GetFieldMappingsRequest.java @@ -34,7 +34,7 @@ import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.Validatable; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; /** Request the mappings of specific fields */ public class GetFieldMappingsRequest implements Validatable { diff --git a/client/rest-high-level/src/main/java/org/opensearch/client/indices/GetIndexTemplatesRequest.java b/client/rest-high-level/src/main/java/org/opensearch/client/indices/GetIndexTemplatesRequest.java index c1bf001e525b3..dc1759a7272e8 100644 --- a/client/rest-high-level/src/main/java/org/opensearch/client/indices/GetIndexTemplatesRequest.java +++ b/client/rest-high-level/src/main/java/org/opensearch/client/indices/GetIndexTemplatesRequest.java @@ -35,8 +35,8 @@ import org.opensearch.client.TimedRequest; import org.opensearch.client.Validatable; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import java.util.Arrays; import java.util.List; diff --git a/client/rest-high-level/src/main/java/org/opensearch/client/indices/GetMappingsRequest.java b/client/rest-high-level/src/main/java/org/opensearch/client/indices/GetMappingsRequest.java index e049e70c7587c..2e5b0cb89cedb 100644 --- a/client/rest-high-level/src/main/java/org/opensearch/client/indices/GetMappingsRequest.java +++ b/client/rest-high-level/src/main/java/org/opensearch/client/indices/GetMappingsRequest.java @@ -34,7 +34,7 @@ import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.TimedRequest; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; public class GetMappingsRequest extends TimedRequest { diff --git a/client/rest-high-level/src/main/java/org/opensearch/client/indices/PutComponentTemplateRequest.java b/client/rest-high-level/src/main/java/org/opensearch/client/indices/PutComponentTemplateRequest.java index d2425b5fd9bc9..cfcd594658ea1 100644 --- a/client/rest-high-level/src/main/java/org/opensearch/client/indices/PutComponentTemplateRequest.java +++ b/client/rest-high-level/src/main/java/org/opensearch/client/indices/PutComponentTemplateRequest.java @@ -33,7 +33,7 @@ import org.opensearch.client.TimedRequest; import org.opensearch.cluster.metadata.ComponentTemplate; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; 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/indices/PutComposableIndexTemplateRequest.java b/client/rest-high-level/src/main/java/org/opensearch/client/indices/PutComposableIndexTemplateRequest.java index 4257627b0667d..309519d6f7b76 100644 --- a/client/rest-high-level/src/main/java/org/opensearch/client/indices/PutComposableIndexTemplateRequest.java +++ b/client/rest-high-level/src/main/java/org/opensearch/client/indices/PutComposableIndexTemplateRequest.java @@ -33,7 +33,7 @@ import org.opensearch.client.TimedRequest; import org.opensearch.cluster.metadata.ComposableIndexTemplate; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; 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/indices/SimulateIndexTemplateRequest.java b/client/rest-high-level/src/main/java/org/opensearch/client/indices/SimulateIndexTemplateRequest.java index ab894dfcf7fa1..abc095b3ccac9 100644 --- a/client/rest-high-level/src/main/java/org/opensearch/client/indices/SimulateIndexTemplateRequest.java +++ b/client/rest-high-level/src/main/java/org/opensearch/client/indices/SimulateIndexTemplateRequest.java @@ -34,7 +34,7 @@ import org.opensearch.client.TimedRequest; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; /** * A request to simulate matching a provided index name and an optional new index template against the existing index templates. diff --git a/client/rest-high-level/src/main/resources/forbidden/rest-high-level-signatures.txt b/client/rest-high-level/src/main/resources/forbidden/rest-high-level-signatures.txt index 68dc509e5ff27..227f380324902 100644 --- a/client/rest-high-level/src/main/resources/forbidden/rest-high-level-signatures.txt +++ b/client/rest-high-level/src/main/resources/forbidden/rest-high-level-signatures.txt @@ -23,7 +23,7 @@ org.apache.http.entity.ContentType#create(java.lang.String,org.apache.http.NameV @defaultMessage ES's logging infrastructure uses log4j2 which we don't want to force on high level rest client users org.opensearch.common.logging.DeprecationLogger org.opensearch.common.logging.LogConfigurator -org.opensearch.common.logging.LoggerMessageFormat +org.opensearch.core.common.logging.LoggerMessageFormat org.opensearch.common.logging.Loggers org.opensearch.common.logging.NodeNamePatternConverter org.opensearch.common.logging.PrefixLogger diff --git a/client/rest-high-level/src/test/java/org/opensearch/client/CrudIT.java b/client/rest-high-level/src/test/java/org/opensearch/client/CrudIT.java index 86efeb5d84b10..06a6fb6802e3c 100644 --- a/client/rest-high-level/src/test/java/org/opensearch/client/CrudIT.java +++ b/client/rest-high-level/src/test/java/org/opensearch/client/CrudIT.java @@ -58,11 +58,11 @@ import org.opensearch.client.core.TermVectorsRequest; import org.opensearch.client.core.TermVectorsResponse; import org.opensearch.client.indices.GetIndexRequest; -import org.opensearch.common.Strings; import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.ByteSizeUnit; import org.opensearch.common.unit.ByteSizeValue; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.common.xcontent.XContentFactory; import org.opensearch.common.xcontent.XContentType; diff --git a/client/rest-high-level/src/test/java/org/opensearch/client/IndicesClientIT.java b/client/rest-high-level/src/test/java/org/opensearch/client/IndicesClientIT.java index a03c23329218f..b982a9e74a72e 100644 --- a/client/rest-high-level/src/test/java/org/opensearch/client/IndicesClientIT.java +++ b/client/rest-high-level/src/test/java/org/opensearch/client/IndicesClientIT.java @@ -107,7 +107,6 @@ import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.MappingMetadata; import org.opensearch.cluster.metadata.Template; -import org.opensearch.common.Strings; import org.opensearch.common.ValidationException; import org.opensearch.common.bytes.BytesArray; import org.opensearch.common.compress.CompressedXContent; @@ -116,6 +115,7 @@ import org.opensearch.common.unit.ByteSizeUnit; import org.opensearch.common.unit.ByteSizeValue; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.common.xcontent.XContentFactory; import org.opensearch.common.xcontent.XContentType; 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 9d7cc1cec420b..4cf88903f4380 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 @@ -72,11 +72,11 @@ import org.opensearch.client.indices.ResizeRequest; import org.opensearch.client.indices.rollover.RolloverRequest; import org.opensearch.common.CheckedFunction; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.CollectionUtils; import org.opensearch.common.xcontent.XContentType; +import org.opensearch.core.common.Strings; import org.opensearch.test.OpenSearchTestCase; import org.junit.Assert; import org.opensearch.common.unit.ByteSizeValue; diff --git a/client/rest-high-level/src/test/java/org/opensearch/client/RequestConvertersTests.java b/client/rest-high-level/src/test/java/org/opensearch/client/RequestConvertersTests.java index fa7000a1ce32a..2475a4987b165 100644 --- a/client/rest-high-level/src/test/java/org/opensearch/client/RequestConvertersTests.java +++ b/client/rest-high-level/src/test/java/org/opensearch/client/RequestConvertersTests.java @@ -73,7 +73,6 @@ import org.opensearch.client.core.TermVectorsRequest; import org.opensearch.client.indices.AnalyzeRequest; import org.opensearch.common.CheckedBiConsumer; -import org.opensearch.common.Strings; import org.opensearch.common.bytes.BytesArray; import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.collect.Tuple; @@ -81,12 +80,13 @@ import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.lucene.uid.Versions; import org.opensearch.common.unit.TimeValue; -import org.opensearch.core.xcontent.ToXContent; -import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.common.xcontent.XContentHelper; -import org.opensearch.core.xcontent.XContentParser; import org.opensearch.common.xcontent.XContentType; import org.opensearch.common.xcontent.json.JsonXContent; +import org.opensearch.core.common.Strings; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParser; import org.opensearch.index.VersionType; import org.opensearch.index.mapper.MapperService; import org.opensearch.index.query.MatchAllQueryBuilder; diff --git a/client/rest-high-level/src/test/java/org/opensearch/client/indices/CloseIndexRequestTests.java b/client/rest-high-level/src/test/java/org/opensearch/client/indices/CloseIndexRequestTests.java index 15ae0bbca910a..1b93c478fcb7d 100644 --- a/client/rest-high-level/src/test/java/org/opensearch/client/indices/CloseIndexRequestTests.java +++ b/client/rest-high-level/src/test/java/org/opensearch/client/indices/CloseIndexRequestTests.java @@ -35,8 +35,8 @@ import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.TimedRequest; import org.opensearch.client.ValidationException; -import org.opensearch.common.Strings; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import org.opensearch.test.OpenSearchTestCase; import java.util.Optional; diff --git a/libs/common/src/test/java/org/opensearch/bootstrap/JarHellTests.java b/libs/common/src/test/java/org/opensearch/bootstrap/JarHellTests.java index 57f5f393ce49f..d1851850e78e1 100644 --- a/libs/common/src/test/java/org/opensearch/bootstrap/JarHellTests.java +++ b/libs/common/src/test/java/org/opensearch/bootstrap/JarHellTests.java @@ -32,8 +32,8 @@ package org.opensearch.bootstrap; -import org.opensearch.common.Strings; import org.opensearch.common.io.PathUtils; +import org.opensearch.core.common.Strings; import org.opensearch.test.OpenSearchTestCase; import java.io.IOException; diff --git a/libs/core/src/main/java/org/opensearch/BaseExceptionsHelper.java b/libs/core/src/main/java/org/opensearch/BaseExceptionsHelper.java new file mode 100644 index 0000000000000..2b282d1fc48c3 --- /dev/null +++ b/libs/core/src/main/java/org/opensearch/BaseExceptionsHelper.java @@ -0,0 +1,283 @@ +/* + * 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. + */ + +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ +package org.opensearch; + +import com.fasterxml.jackson.core.JsonParseException; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.core.ParseField; +import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.XContentBuilder; + +import java.io.IOException; +import java.io.PrintWriter; +import java.io.StringWriter; +import java.util.List; +import java.util.Map; + +import static java.util.Collections.emptyMap; + +/** + * Base helper class for OpenSearch Exceptions + * + * @opensearch.internal + */ +public abstract class BaseExceptionsHelper { + /** + * Passed in the {@link ToXContent.Params} of {@link #generateThrowableXContent(XContentBuilder, ToXContent.Params, Throwable)} + * to control if the {@code stack_trace} element should render. Unlike most parameters to {@code toXContent} methods this parameter is + * internal only and not available as a URL parameter. Use the {@code error_trace} parameter instead. + */ + public static final String REST_EXCEPTION_SKIP_STACK_TRACE = "rest.exception.stacktrace.skip"; + public static final boolean REST_EXCEPTION_SKIP_STACK_TRACE_DEFAULT = true; + public static final String STACK_TRACE = "stack_trace"; + protected static final Logger logger = LogManager.getLogger(BaseExceptionsHelper.class); + protected static final String OPENSEARCH_PREFIX_KEY = "opensearch."; + /** + * Passed in the {@link ToXContent.Params} of {@link #generateThrowableXContent(XContentBuilder, ToXContent.Params, Throwable)} + * to control if the {@code caused_by} element should render. Unlike most parameters to {@code toXContent} methods this parameter is + * internal only and not available as a URL parameter. + */ + protected static final String REST_EXCEPTION_SKIP_CAUSE = "rest.exception.cause.skip"; + protected static final String TYPE = "type"; + protected static final String REASON = "reason"; + protected static final String CAUSED_BY = "caused_by"; + protected static final ParseField SUPPRESSED = new ParseField("suppressed"); + protected static final String HEADER = "header"; + private static final boolean REST_EXCEPTION_SKIP_CAUSE_DEFAULT = false; + + public static Throwable unwrapCause(Throwable t) { + int counter = 0; + Throwable result = t; + while (result instanceof OpenSearchWrapperException) { + if (result.getCause() == null) { + return result; + } + if (result.getCause() == result) { + return result; + } + if (counter++ > 10) { + // dear god, if we got more than 10 levels down, WTF? just bail + logger.warn("Exception cause unwrapping ran for 10 levels...", t); + return result; + } + result = result.getCause(); + } + return result; + } + + /** + * @deprecated Don't swallow exceptions, allow them to propagate. + */ + @Deprecated + public static String detailedMessage(Throwable t) { + if (t == null) { + return "Unknown"; + } + if (t.getCause() != null) { + StringBuilder sb = new StringBuilder(); + while (t != null) { + sb.append(t.getClass().getSimpleName()); + if (t.getMessage() != null) { + sb.append("["); + sb.append(t.getMessage()); + sb.append("]"); + } + sb.append("; "); + t = t.getCause(); + if (t != null) { + sb.append("nested: "); + } + } + return sb.toString(); + } else { + return t.getClass().getSimpleName() + "[" + t.getMessage() + "]"; + } + } + + public static String stackTrace(Throwable e) { + StringWriter stackTraceStringWriter = new StringWriter(); + PrintWriter printWriter = new PrintWriter(stackTraceStringWriter); + e.printStackTrace(printWriter); + return stackTraceStringWriter.toString(); + } + + public static String summaryMessage(Throwable t) { + if (t != null) { + if (t instanceof BaseOpenSearchException) { + return t.getClass().getSimpleName() + "[" + t.getMessage() + "]"; + } else if (t instanceof IllegalArgumentException) { + return "Invalid argument"; + } else if (t instanceof JsonParseException) { + return "Failed to parse JSON"; + } else if (t instanceof OpenSearchRejectedExecutionException) { + return "Too many requests"; + } + } + return "Internal failure"; + } + + public static void innerToXContent( + XContentBuilder builder, + ToXContent.Params params, + Throwable throwable, + String type, + String message, + Map> headers, + Map> metadata, + Throwable cause + ) throws IOException { + builder.field(TYPE, type); + builder.field(REASON, message); + + for (Map.Entry> entry : metadata.entrySet()) { + headerToXContent(builder, entry.getKey().substring(OPENSEARCH_PREFIX_KEY.length()), entry.getValue()); + } + + if (throwable instanceof BaseOpenSearchException) { + BaseOpenSearchException exception = (BaseOpenSearchException) throwable; + exception.metadataToXContent(builder, params); + } + + if (params.paramAsBoolean(REST_EXCEPTION_SKIP_CAUSE, REST_EXCEPTION_SKIP_CAUSE_DEFAULT) == false) { + if (cause != null) { + builder.field(CAUSED_BY); + builder.startObject(); + generateThrowableXContent(builder, params, cause); + builder.endObject(); + } + } + + if (headers.isEmpty() == false) { + builder.startObject(HEADER); + for (Map.Entry> entry : headers.entrySet()) { + headerToXContent(builder, entry.getKey(), entry.getValue()); + } + builder.endObject(); + } + + if (params.paramAsBoolean(REST_EXCEPTION_SKIP_STACK_TRACE, REST_EXCEPTION_SKIP_STACK_TRACE_DEFAULT) == false) { + builder.field(STACK_TRACE, stackTrace(throwable)); + } + + Throwable[] allSuppressed = throwable.getSuppressed(); + if (allSuppressed.length > 0) { + builder.startArray(SUPPRESSED.getPreferredName()); + for (Throwable suppressed : allSuppressed) { + builder.startObject(); + generateThrowableXContent(builder, params, suppressed); + builder.endObject(); + } + builder.endArray(); + } + } + + /** + * Static toXContent helper method that renders {@link BaseOpenSearchException} or {@link Throwable} instances + * as XContent, delegating the rendering to {@link BaseOpenSearchException#toXContent(XContentBuilder, ToXContent.Params)} + * or {@link #innerToXContent(XContentBuilder, ToXContent.Params, Throwable, String, String, Map, Map, Throwable)}. + * + * This method is usually used when the {@link Throwable} is rendered as a part of another XContent object, and its result can + * be parsed back using the {@code OpenSearchException.fromXContent(XContentParser)} method. + */ + public static void generateThrowableXContent(XContentBuilder builder, ToXContent.Params params, Throwable t) throws IOException { + t = unwrapCause(t); + + if (t instanceof BaseOpenSearchException) { + ((BaseOpenSearchException) t).toXContent(builder, params); + } else { + innerToXContent(builder, params, t, getExceptionName(t), t.getMessage(), emptyMap(), emptyMap(), t.getCause()); + } + } + + /** + * Returns an underscore case name for the given exception. This method strips {@code OpenSearch} prefixes from exception names. + */ + public static String getExceptionName(Throwable ex) { + String simpleName = ex.getClass().getSimpleName(); + if (simpleName.startsWith("OpenSearch")) { + simpleName = simpleName.substring("OpenSearch".length()); + } + // TODO: do we really need to make the exception name in underscore casing? + return toUnderscoreCase(simpleName); + } + + // lower cases and adds underscores to transitions in a name + private static String toUnderscoreCase(String value) { + StringBuilder sb = new StringBuilder(); + boolean changed = false; + for (int i = 0; i < value.length(); i++) { + char c = value.charAt(i); + if (Character.isUpperCase(c)) { + if (!changed) { + // copy it over here + for (int j = 0; j < i; j++) { + sb.append(value.charAt(j)); + } + changed = true; + if (i == 0) { + sb.append(Character.toLowerCase(c)); + } else { + sb.append('_'); + sb.append(Character.toLowerCase(c)); + } + } else { + sb.append('_'); + sb.append(Character.toLowerCase(c)); + } + } else { + if (changed) { + sb.append(c); + } + } + } + if (!changed) { + return value; + } + return sb.toString(); + } + + protected static void headerToXContent(XContentBuilder builder, String key, List values) throws IOException { + if (values != null && values.isEmpty() == false) { + if (values.size() == 1) { + builder.field(key, values.get(0)); + } else { + builder.startArray(key); + for (String value : values) { + builder.value(value); + } + builder.endArray(); + } + } + } +} diff --git a/libs/core/src/main/java/org/opensearch/BaseOpenSearchException.java b/libs/core/src/main/java/org/opensearch/BaseOpenSearchException.java new file mode 100644 index 0000000000000..5a70782632ae7 --- /dev/null +++ b/libs/core/src/main/java/org/opensearch/BaseOpenSearchException.java @@ -0,0 +1,437 @@ +/* + * 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. + */ + +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +/* + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ +package org.opensearch; + +import org.opensearch.common.Nullable; +import org.opensearch.core.common.Strings; +import org.opensearch.core.common.logging.LoggerMessageFormat; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.ToXContentFragment; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParseException; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static java.util.Collections.singletonMap; + +/** + * A core library base class for all opensearch exceptions. + * + * @opensearch.internal + */ +public abstract class BaseOpenSearchException extends RuntimeException implements ToXContentFragment { + + protected static final String ERROR = "error"; + protected static final String ROOT_CAUSE = "root_cause"; + protected static final String RESOURCE_METADATA_TYPE_KEY = "opensearch.resource.type"; + protected static final String RESOURCE_METADATA_ID_KEY = "opensearch.resource.id"; + protected static final Version UNKNOWN_VERSION_ADDED = Version.fromId(0); + protected static final String INDEX_METADATA_KEY = "opensearch.index"; + protected static final String SHARD_METADATA_KEY = "opensearch.shard"; + protected static final String INDEX_METADATA_KEY_UUID = "opensearch.index_uuid"; + protected final Map> metadata = new HashMap<>(); + protected final Map> headers = new HashMap<>(); + + /** + * Construct a BaseOpenSearchException with the specified cause exception. + */ + public BaseOpenSearchException(Throwable cause) { + super(cause); + } + + /** + * Construct a OpenSearchException with the specified detail message. + * + * The message can be parameterized using {} as placeholders for the given + * arguments + * + * @param msg the detail message + * @param args the arguments for the message + */ + public BaseOpenSearchException(String msg, Object... args) { + super(LoggerMessageFormat.format(msg, args)); + } + + /** + * Construct a OpenSearchException with the specified detail message + * and nested exception. + * + * The message can be parameterized using {} as placeholders for the given + * arguments + * + * @param msg the detail message + * @param cause the nested exception + * @param args the arguments for the message + */ + public BaseOpenSearchException(String msg, Throwable cause, Object... args) { + super(LoggerMessageFormat.format(msg, args), cause); + } + + /** + * Render any exception as a xcontent, encapsulated within a field or object named "error". The level of details that are rendered + * depends on the value of the "detailed" parameter: when it's false only a simple message based on the type and message of the + * exception is rendered. When it's true all detail are provided including guesses root causes, cause and potentially stack + * trace. + * + * This method is usually used when the {@link Exception} is rendered as a full XContent object, and its output can be parsed + * by the {@code #OpenSearchException.failureFromXContent(XContentParser)} method. + */ + public static void generateFailureXContent(XContentBuilder builder, ToXContent.Params params, @Nullable Exception e, boolean detailed) + throws IOException { + // No exception to render as an error + if (e == null) { + builder.field(ERROR, "unknown"); + return; + } + + // Render the exception with a simple message + if (detailed == false) { + Throwable t = e; + for (int counter = 0; counter < 10 && t != null; counter++) { + if (t instanceof BaseOpenSearchException) { + break; + } + t = t.getCause(); + } + builder.field(ERROR, BaseExceptionsHelper.summaryMessage(t != null ? t : e)); + return; + } + + // Render the exception with all details + final BaseOpenSearchException[] rootCauses = BaseOpenSearchException.guessRootCauses(e); + builder.startObject(ERROR); + { + builder.startArray(ROOT_CAUSE); + for (BaseOpenSearchException rootCause : rootCauses) { + builder.startObject(); + rootCause.toXContent( + builder, + new ToXContent.DelegatingMapParams(singletonMap(BaseExceptionsHelper.REST_EXCEPTION_SKIP_CAUSE, "true"), params) + ); + builder.endObject(); + } + builder.endArray(); + } + BaseExceptionsHelper.generateThrowableXContent(builder, params, e); + builder.endObject(); + } + + /** + * Returns the root cause of this exception or multiple if different shards caused different exceptions. + * If the given exception is not an instance of {@link BaseOpenSearchException} an empty array + * is returned. + */ + public static BaseOpenSearchException[] guessRootCauses(Throwable t) { + Throwable ex = BaseExceptionsHelper.unwrapCause(t); + if (ex instanceof BaseOpenSearchException) { + // OpenSearchException knows how to guess its own root cause + return ((BaseOpenSearchException) ex).guessRootCauses(); + } + if (ex instanceof XContentParseException) { + /* + * We'd like to unwrap parsing exceptions to the inner-most + * parsing exception because that is generally the most interesting + * exception to return to the user. If that exception is caused by + * an OpenSearchException we'd like to keep unwrapping because + * OpenSearchException instances tend to contain useful information + * for the user. + */ + Throwable cause = ex.getCause(); + if (cause != null) { + if (cause instanceof XContentParseException || cause instanceof BaseOpenSearchException) { + return BaseOpenSearchException.guessRootCauses(ex.getCause()); + } + } + } + return new BaseOpenSearchException[] { new BaseOpenSearchException(ex.getMessage(), ex) { + @Override + protected String getExceptionName() { + return BaseExceptionsHelper.getExceptionName(getCause()); + } + } }; + } + + static String buildMessage(String type, String reason, String stack) { + StringBuilder message = new StringBuilder("OpenSearch exception ["); + message.append(BaseExceptionsHelper.TYPE).append('=').append(type).append(", "); + message.append(BaseExceptionsHelper.REASON).append('=').append(reason); + if (stack != null) { + message.append(", ").append(BaseExceptionsHelper.STACK_TRACE).append('=').append(stack); + } + message.append(']'); + return message.toString(); + } + + /** + * Adds a new piece of metadata with the given key. + * If the provided key is already present, the corresponding metadata will be replaced + */ + public void addMetadata(String key, String... values) { + addMetadata(key, Arrays.asList(values)); + } + + /** + * Adds a new piece of metadata with the given key. + * If the provided key is already present, the corresponding metadata will be replaced + */ + public void addMetadata(String key, List values) { + // we need to enforce this otherwise bw comp doesn't work properly, as "opensearch." + // was the previous criteria to split headers in two sets + if (key.startsWith(BaseExceptionsHelper.OPENSEARCH_PREFIX_KEY) == false) { + throw new IllegalArgumentException("exception metadata must start with [opensearch.], found [" + key + "] instead"); + } + this.metadata.put(key, values); + } + + /** + * Returns a set of all metadata keys on this exception + */ + public Set getMetadataKeys() { + return metadata.keySet(); + } + + /** + * Returns the list of metadata values for the given key or {@code null} if no metadata for the + * given key exists. + */ + public List getMetadata(String key) { + return metadata.get(key); + } + + protected Map> getMetadata() { + return metadata; + } + + /** + * Adds a new header with the given key. + * This method will replace existing header if a header with the same key already exists + */ + public void addHeader(String key, List value) { + // we need to enforce this otherwise bw comp doesn't work properly, as "opensearch." + // was the previous criteria to split headers in two sets + if (key.startsWith(BaseExceptionsHelper.OPENSEARCH_PREFIX_KEY)) { + throw new IllegalArgumentException("exception headers must not start with [opensearch.], found [" + key + "] instead"); + } + this.headers.put(key, value); + } + + /** + * Adds a new header with the given key. + * This method will replace existing header if a header with the same key already exists + */ + public void addHeader(String key, String... value) { + addHeader(key, Arrays.asList(value)); + } + + /** + * Returns a set of all header keys on this exception + */ + public Set getHeaderKeys() { + return headers.keySet(); + } + + /** + * Returns the list of header values for the given key or {@code null} if no header for the + * given key exists. + */ + public List getHeader(String key) { + return headers.get(key); + } + + protected Map> getHeaders() { + return headers; + } + + /** + * Unwraps the actual cause from the exception for cases when the exception is a + * {@link OpenSearchWrapperException}. + * + * @see BaseExceptionsHelper#unwrapCause(Throwable) + */ + public Throwable unwrapCause() { + return BaseExceptionsHelper.unwrapCause(this); + } + + /** + * Return the detail message, including the message from the nested exception + * if there is one. + */ + public String getDetailedMessage() { + if (getCause() != null) { + StringBuilder sb = new StringBuilder(); + sb.append(toString()).append("; "); + if (getCause() instanceof BaseOpenSearchException) { + sb.append(((BaseOpenSearchException) getCause()).getDetailedMessage()); + } else { + sb.append(getCause()); + } + return sb.toString(); + } else { + return toString(); + } + } + + /** + * Retrieve the innermost cause of this exception, if none, returns the current exception. + */ + public Throwable getRootCause() { + Throwable rootCause = this; + Throwable cause = getCause(); + while (cause != null && cause != rootCause) { + rootCause = cause; + cause = cause.getCause(); + } + return rootCause; + } + + /** + * Renders additional per exception information into the XContent + */ + protected void metadataToXContent(XContentBuilder builder, ToXContent.Params params) throws IOException {} + + public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params params) throws IOException { + Throwable ex = BaseExceptionsHelper.unwrapCause(this); + if (ex != this) { + BaseExceptionsHelper.generateThrowableXContent(builder, params, this); + } else { + BaseExceptionsHelper.innerToXContent(builder, params, this, getExceptionName(), getMessage(), headers, metadata, getCause()); + } + return builder; + } + + protected String getExceptionName() { + return BaseExceptionsHelper.getExceptionName(this); + } + + /** + * Returns the root cause of this exception or multiple if different shards caused different exceptions + */ + public BaseOpenSearchException[] guessRootCauses() { + final Throwable cause = getCause(); + if (cause != null && cause instanceof BaseOpenSearchException) { + return ((BaseOpenSearchException) cause).guessRootCauses(); + } + return new BaseOpenSearchException[] { this }; + } + + public void setResources(String type, String... id) { + assert type != null; + addMetadata(RESOURCE_METADATA_ID_KEY, id); + addMetadata(RESOURCE_METADATA_TYPE_KEY, type); + } + + public List getResourceId() { + return getMetadata(RESOURCE_METADATA_ID_KEY); + } + + public String getResourceType() { + List header = getMetadata(RESOURCE_METADATA_TYPE_KEY); + if (header != null && header.isEmpty() == false) { + assert header.size() == 1; + return header.get(0); + } + return null; + } + + public String getIndexName() { + List index = getMetadata(INDEX_METADATA_KEY); + if (index != null && index.isEmpty() == false) { + return index.get(0); + } + return null; + } + + /** + * Get index uuid as a string + * + * @deprecated remove in favor of Index#toString once Index class is moved to core library + */ + @Deprecated + private String getIndexUUID() { + List index_uuid = getMetadata(INDEX_METADATA_KEY_UUID); + if (index_uuid != null && index_uuid.isEmpty() == false) { + return index_uuid.get(0); + } + return null; + } + + @Override + public String toString() { + StringBuilder builder = new StringBuilder(); + if (metadata.containsKey(INDEX_METADATA_KEY)) { + builder.append(getIndexString()); + if (metadata.containsKey(SHARD_METADATA_KEY)) { + builder.append('[').append(getShardIdString()).append(']'); + } + builder.append(' '); + } + return builder.append(BaseExceptionsHelper.detailedMessage(this).trim()).toString(); + } + + /** + * Get index string + * + * @deprecated remove in favor of Index#toString once Index class is moved to core library + */ + @Deprecated + private String getIndexString() { + String uuid = getIndexUUID(); + if (uuid != null) { + String name = getIndexName(); + if (Strings.UNKNOWN_UUID_VALUE.equals(uuid)) { + return "[" + name + "]"; + } + return "[" + name + "/" + uuid + "]"; + } + return null; + } + + /** + * Get shard id string + * + * @deprecated remove in favor of ShardId#toString once ShardId class is moved to core library + */ + @Deprecated + private String getShardIdString() { + String indexName = getIndexName(); + List shard = getMetadata(SHARD_METADATA_KEY); + if (indexName != null && shard != null && shard.isEmpty() == false) { + return "[" + indexName + "][" + Integer.parseInt(shard.get(0)) + "]"; + } + return null; + } +} diff --git a/server/src/main/java/org/opensearch/OpenSearchWrapperException.java b/libs/core/src/main/java/org/opensearch/OpenSearchWrapperException.java similarity index 100% rename from server/src/main/java/org/opensearch/OpenSearchWrapperException.java rename to libs/core/src/main/java/org/opensearch/OpenSearchWrapperException.java 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 new file mode 100644 index 0000000000000..0cae7da3798e8 --- /dev/null +++ b/libs/core/src/main/java/org/opensearch/core/common/Strings.java @@ -0,0 +1,518 @@ +/* + * 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.core.common; + +import org.opensearch.common.Nullable; + +import java.io.BufferedReader; +import java.io.StringReader; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import java.util.StringTokenizer; +import java.util.TreeSet; +import java.util.function.Supplier; + +/** + * String utility class. + * + * TODO replace Strings in :server + * + * @opensearch.internal + */ +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. + * + * @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); + } + } + } + + /** + * 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, ","); + } + + /** + * 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. + *
+     * StringUtils.hasLength(null) = false
+     * StringUtils.hasLength("") = false
+     * StringUtils.hasLength(" ") = true
+     * StringUtils.hasLength("Hello") = true
+     * 
+ * + * @param str the CharSequence to check (may be null) + * @return true if the CharSequence is not null and has length + * @see #hasText(String) + */ + public static boolean hasLength(CharSequence str) { + return (str != null && str.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. + * + * @param str the String to check (may be null) + * @return true if the String is not null and has length + * @see Strings#hasLength(CharSequence) + */ + public static boolean hasLength(String str) { + return hasLength((CharSequence) str); + } + + /** + * Check that the given CharSequence is either null or of length 0. + * Note: Will return false for a CharSequence that purely consists of whitespace. + *
+     * StringUtils.isEmpty(null) = true
+     * StringUtils.isEmpty("") = true
+     * StringUtils.isEmpty(" ") = false
+     * StringUtils.isEmpty("Hello") = false
+     * 
+ * + * @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) { + return hasLength(str) == false; + } + + /** + * Check whether the given CharSequence has actual text. + * More specifically, returns true if the string not null, + * its length is greater than 0, and it contains at least one non-whitespace character. + *
+     * StringUtils.hasText(null) = false
+     * StringUtils.hasText("") = false
+     * StringUtils.hasText(" ") = false
+     * StringUtils.hasText("12345") = true
+     * StringUtils.hasText(" 12345 ") = true
+     * 
+ * + * @param str the CharSequence to check (may be null) + * @return true if the CharSequence is not null, + * its length is greater than 0, and it does not contain whitespace only + * @see Character#isWhitespace + */ + public static boolean hasText(CharSequence str) { + if (!hasLength(str)) { + return false; + } + int strLen = str.length(); + for (int i = 0; i < strLen; i++) { + if (!Character.isWhitespace(str.charAt(i))) { + return true; + } + } + return false; + } + + /** + * Check whether the given String has actual text. + * More specifically, returns true if the string not null, + * its length is greater than 0, and it contains at least one non-whitespace character. + * + * @param str the String to check (may be null) + * @return true if the String is not null, its length is + * greater than 0, and it does not contain whitespace only + * @see Strings#hasText(CharSequence) + */ + public static boolean hasText(String str) { + return hasText((CharSequence) str); + } + + /** + * Replace all occurrences of a substring within a string with + * another string. + * + * @param inString String to examine + * @param oldPattern String to replace + * @param newPattern String to insert + * @return a String with the replacements + */ + public static String replace(String inString, String oldPattern, String newPattern) { + if (!hasLength(inString) || !hasLength(oldPattern) || newPattern == null) { + return inString; + } + StringBuilder sb = new StringBuilder(); + int pos = 0; // our position in the old string + int index = inString.indexOf(oldPattern); + // the index of an occurrence we've found, or -1 + int patLen = oldPattern.length(); + while (index >= 0) { + sb.append(inString.substring(pos, index)); + sb.append(newPattern); + pos = index + patLen; + index = inString.indexOf(oldPattern, pos); + } + sb.append(inString.substring(pos)); + // remember to append any characters to the right of a match + 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. + * + * @param inString the original String + * @param pattern the pattern to delete all occurrences of + * @return the resulting String + */ + public static String delete(String inString, String pattern) { + return replace(inString, pattern, ""); + } + + /** + * Delete any character in a given String. + * + * @param inString the original String + * @param charsToDelete a set of characters to delete. + * E.g. "az\n" will delete 'a's, 'z's and new lines. + * @return the resulting String + */ + public static String deleteAny(String inString, String charsToDelete) { + if (hasLength(inString) == false || hasLength(charsToDelete) == false) { + return inString; + } + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < inString.length(); i++) { + char c = inString.charAt(i); + if (charsToDelete.indexOf(c) == -1) { + sb.append(c); + } + } + 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'); + } + } + } + + /** + * Copy the given Collection into a String array. + * The Collection must contain String elements only. + * + * @param collection the Collection to copy + * @return the String array (null if the passed-in + * Collection was null) + */ + public static String[] toStringArray(final Collection collection) { + if (collection == null) { + return null; + } + return collection.toArray(new String[collection.size()]); + } + + /** + * 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. + * + * @param s the string to tokenize + * @return the set of tokens + */ + public static Set tokenizeByCommaToSet(final String s) { + if (s == null) return Collections.emptySet(); + return tokenizeToCollection(s, ",", HashSet::new); + } + + /** + * Tokenize the given String into a String array via a StringTokenizer. + * Trims tokens and omits empty tokens. + *

The given delimiters string is supposed to consist of any number of + * delimiter characters. Each of those characters can be used to separate + * tokens. A delimiter is always a single character; for multi-character + * delimiters, consider using delimitedListToStringArray + * + * @param s the String to tokenize + * @param delimiters the delimiter characters, assembled as String + * (each of those characters is individually considered as delimiter). + * @return an array of the tokens + * @see StringTokenizer + * @see String#trim() + * @see Strings#delimitedListToStringArray + */ + public static String[] tokenizeToStringArray(final String s, final String delimiters) { + if (s == null) { + return Strings.EMPTY_ARRAY; + } + return toStringArray(tokenizeToCollection(s, delimiters, ArrayList::new)); + } + + /** + * Tokenizes the specified string to a collection using the specified delimiters as the token delimiters. This method trims whitespace + * from tokens and ignores empty tokens. + * + * @param s the string to tokenize. + * @param delimiters the token delimiters + * @param supplier a collection supplier + * @param the type of the collection + * @return the tokens + * @see StringTokenizer + */ + private static > T tokenizeToCollection( + final String s, + final String delimiters, + final Supplier supplier + ) { + if (s == null) { + return null; + } + final StringTokenizer tokenizer = new StringTokenizer(s, delimiters); + final T tokens = supplier.get(); + while (tokenizer.hasMoreTokens()) { + final String token = tokenizer.nextToken().trim(); + if (token.length() > 0) { + tokens.add(token); + } + } + 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) + * @return an array of the tokens in the list + * @see Strings#tokenizeToStringArray + */ + public static String[] delimitedListToStringArray(String str, String delimiter) { + return delimitedListToStringArray(str, delimiter, null); + } + + /** + * Convert a CSV list into an array of Strings. + * + * @param str the input String + * @return an array of Strings, or the empty array in case of empty input + */ + public static String[] commaDelimitedListToStringArray(String str) { + return delimitedListToStringArray(str, ","); + } + + /** + * Convenience method to convert a CSV string list to a set. + * Note that this will suppress duplicates. + * + * @param str the input String + * @return a Set of String entries in the list + */ + public static Set commaDelimitedListToSet(String str) { + Set set = new TreeSet<>(); + String[] tokens = commaDelimitedListToStringArray(str); + set.addAll(Arrays.asList(tokens)); + 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)}. + * 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(); + } +} diff --git a/server/src/main/java/org/opensearch/common/logging/LoggerMessageFormat.java b/libs/core/src/main/java/org/opensearch/core/common/logging/LoggerMessageFormat.java similarity index 99% rename from server/src/main/java/org/opensearch/common/logging/LoggerMessageFormat.java rename to libs/core/src/main/java/org/opensearch/core/common/logging/LoggerMessageFormat.java index ad9981809ae3a..bca919e12ea7e 100644 --- a/server/src/main/java/org/opensearch/common/logging/LoggerMessageFormat.java +++ b/libs/core/src/main/java/org/opensearch/core/common/logging/LoggerMessageFormat.java @@ -30,7 +30,7 @@ * GitHub history for details. */ -package org.opensearch.common.logging; +package org.opensearch.core.common.logging; import java.util.HashSet; import java.util.Set; diff --git a/libs/core/src/main/java/org/opensearch/core/common/logging/package-info.java b/libs/core/src/main/java/org/opensearch/core/common/logging/package-info.java new file mode 100644 index 0000000000000..b3094b1e2782c --- /dev/null +++ b/libs/core/src/main/java/org/opensearch/core/common/logging/package-info.java @@ -0,0 +1,18 @@ +/* + * 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. + */ + +/* + * 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. + */ + +/** core logging classes */ +package org.opensearch.core.common.logging; diff --git a/libs/core/src/main/java/org/opensearch/core/common/package-info.java b/libs/core/src/main/java/org/opensearch/core/common/package-info.java new file mode 100644 index 0000000000000..b5cf352aa5495 --- /dev/null +++ b/libs/core/src/main/java/org/opensearch/core/common/package-info.java @@ -0,0 +1,10 @@ +/* + * 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. + */ + +/** common core classes that require third party dependencies */ +package org.opensearch.core.common; 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 new file mode 100644 index 0000000000000..532251e02e685 --- /dev/null +++ b/libs/core/src/test/java/org/opensearch/core/common/StringsTests.java @@ -0,0 +1,28 @@ +/* + * 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.core.common; + +import org.opensearch.common.util.set.Sets; +import org.opensearch.test.OpenSearchTestCase; + +/** tests for Strings utility class */ +public class StringsTests extends OpenSearchTestCase { + public void testSplitStringToSet() { + assertEquals(Strings.tokenizeByCommaToSet(null), Sets.newHashSet()); + assertEquals(Strings.tokenizeByCommaToSet(""), Sets.newHashSet()); + assertEquals(Strings.tokenizeByCommaToSet("a,b,c"), Sets.newHashSet("a", "b", "c")); + assertEquals(Strings.tokenizeByCommaToSet("a, b, c"), Sets.newHashSet("a", "b", "c")); + assertEquals(Strings.tokenizeByCommaToSet(" a , b, c "), Sets.newHashSet("a", "b", "c")); + assertEquals(Strings.tokenizeByCommaToSet("aa, bb, cc"), Sets.newHashSet("aa", "bb", "cc")); + assertEquals(Strings.tokenizeByCommaToSet(" a "), Sets.newHashSet("a")); + assertEquals(Strings.tokenizeByCommaToSet(" a "), Sets.newHashSet("a")); + assertEquals(Strings.tokenizeByCommaToSet(" aa "), Sets.newHashSet("aa")); + assertEquals(Strings.tokenizeByCommaToSet(" "), Sets.newHashSet()); + } +} diff --git a/modules/analysis-common/src/main/java/org/opensearch/analysis/common/MultiplexerTokenFilterFactory.java b/modules/analysis-common/src/main/java/org/opensearch/analysis/common/MultiplexerTokenFilterFactory.java index 27203efa6a230..fde47a53733e4 100644 --- a/modules/analysis-common/src/main/java/org/opensearch/analysis/common/MultiplexerTokenFilterFactory.java +++ b/modules/analysis-common/src/main/java/org/opensearch/analysis/common/MultiplexerTokenFilterFactory.java @@ -37,8 +37,8 @@ import org.apache.lucene.analysis.miscellaneous.ConditionalTokenFilter; import org.apache.lucene.analysis.miscellaneous.RemoveDuplicatesTokenFilter; import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; import org.opensearch.env.Environment; import org.opensearch.index.IndexSettings; import org.opensearch.index.analysis.AbstractTokenFilterFactory; diff --git a/modules/analysis-common/src/main/java/org/opensearch/analysis/common/PatternReplaceCharFilterFactory.java b/modules/analysis-common/src/main/java/org/opensearch/analysis/common/PatternReplaceCharFilterFactory.java index 3c6cdbf5c5a69..d6fca8ac67d55 100644 --- a/modules/analysis-common/src/main/java/org/opensearch/analysis/common/PatternReplaceCharFilterFactory.java +++ b/modules/analysis-common/src/main/java/org/opensearch/analysis/common/PatternReplaceCharFilterFactory.java @@ -32,9 +32,9 @@ package org.opensearch.analysis.common; import org.apache.lucene.analysis.pattern.PatternReplaceCharFilter; -import org.opensearch.common.Strings; import org.opensearch.common.regex.Regex; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; import org.opensearch.env.Environment; import org.opensearch.index.IndexSettings; import org.opensearch.index.analysis.AbstractCharFilterFactory; diff --git a/modules/analysis-common/src/main/java/org/opensearch/analysis/common/SnowballTokenFilterFactory.java b/modules/analysis-common/src/main/java/org/opensearch/analysis/common/SnowballTokenFilterFactory.java index 591d1947c1eb8..759febb81728b 100644 --- a/modules/analysis-common/src/main/java/org/opensearch/analysis/common/SnowballTokenFilterFactory.java +++ b/modules/analysis-common/src/main/java/org/opensearch/analysis/common/SnowballTokenFilterFactory.java @@ -33,8 +33,8 @@ import org.apache.lucene.analysis.TokenStream; import org.apache.lucene.analysis.snowball.SnowballFilter; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; import org.opensearch.env.Environment; import org.opensearch.index.IndexSettings; import org.opensearch.index.analysis.AbstractTokenFilterFactory; diff --git a/modules/analysis-common/src/main/java/org/opensearch/analysis/common/StemmerTokenFilterFactory.java b/modules/analysis-common/src/main/java/org/opensearch/analysis/common/StemmerTokenFilterFactory.java index fc045447e159e..9d22f52aa3712 100644 --- a/modules/analysis-common/src/main/java/org/opensearch/analysis/common/StemmerTokenFilterFactory.java +++ b/modules/analysis-common/src/main/java/org/opensearch/analysis/common/StemmerTokenFilterFactory.java @@ -67,8 +67,8 @@ import org.apache.lucene.analysis.ru.RussianLightStemFilter; import org.apache.lucene.analysis.snowball.SnowballFilter; import org.apache.lucene.analysis.sv.SwedishLightStemFilter; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; import org.opensearch.env.Environment; import org.opensearch.index.IndexSettings; import org.opensearch.index.analysis.AbstractTokenFilterFactory; diff --git a/modules/geo/src/test/java/org/opensearch/geo/search/aggregations/bucket/geogrid/GeoTileGridParserTests.java b/modules/geo/src/test/java/org/opensearch/geo/search/aggregations/bucket/geogrid/GeoTileGridParserTests.java index 0bf41f37c45a7..a39f66f571d0a 100644 --- a/modules/geo/src/test/java/org/opensearch/geo/search/aggregations/bucket/geogrid/GeoTileGridParserTests.java +++ b/modules/geo/src/test/java/org/opensearch/geo/search/aggregations/bucket/geogrid/GeoTileGridParserTests.java @@ -31,7 +31,7 @@ package org.opensearch.geo.search.aggregations.bucket.geogrid; -import org.opensearch.ExceptionsHelper; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.core.xcontent.XContentParseException; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.common.xcontent.json.JsonXContent; @@ -77,7 +77,7 @@ public void testParseErrorOnBooleanPrecision() throws Exception { () -> GeoTileGridAggregationBuilder.PARSER.parse(stParser, "geotile_grid") ); assertThat( - ExceptionsHelper.detailedMessage(e), + BaseExceptionsHelper.detailedMessage(e), containsString("[geotile_grid] precision doesn't support values of type: VALUE_BOOLEAN") ); } diff --git a/modules/ingest-user-agent/src/test/java/org/opensearch/ingest/useragent/UserAgentProcessorFactoryTests.java b/modules/ingest-user-agent/src/test/java/org/opensearch/ingest/useragent/UserAgentProcessorFactoryTests.java index 72815a37f46de..bfb0e2c9ab9a4 100644 --- a/modules/ingest-user-agent/src/test/java/org/opensearch/ingest/useragent/UserAgentProcessorFactoryTests.java +++ b/modules/ingest-user-agent/src/test/java/org/opensearch/ingest/useragent/UserAgentProcessorFactoryTests.java @@ -33,7 +33,7 @@ package org.opensearch.ingest.useragent; import org.opensearch.OpenSearchParseException; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.test.OpenSearchTestCase; import org.junit.BeforeClass; 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 e626d996b0cc4..5fd2f9085539e 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 @@ -33,7 +33,7 @@ package org.opensearch.script.mustache; import org.opensearch.LegacyESVersion; -import org.opensearch.OpenSearchException; +import org.opensearch.BaseOpenSearchException; import org.opensearch.action.ActionResponse; import org.opensearch.action.search.MultiSearchResponse; import org.opensearch.common.Nullable; @@ -173,7 +173,7 @@ public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params par for (Item item : items) { if (item.isFailure()) { builder.startObject(); - OpenSearchException.generateFailureXContent(builder, params, item.getFailure(), true); + BaseOpenSearchException.generateFailureXContent(builder, params, item.getFailure(), true); builder.endObject(); } else { item.getResponse().toXContent(builder, params); diff --git a/modules/lang-mustache/src/test/java/org/opensearch/script/mustache/MultiSearchTemplateResponseTests.java b/modules/lang-mustache/src/test/java/org/opensearch/script/mustache/MultiSearchTemplateResponseTests.java index 31bfc1ea70d54..aeb7cdc2c6a28 100644 --- a/modules/lang-mustache/src/test/java/org/opensearch/script/mustache/MultiSearchTemplateResponseTests.java +++ b/modules/lang-mustache/src/test/java/org/opensearch/script/mustache/MultiSearchTemplateResponseTests.java @@ -34,7 +34,7 @@ import org.opensearch.OpenSearchException; import org.opensearch.action.search.SearchResponse; import org.opensearch.action.search.ShardSearchFailure; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.search.internal.InternalSearchResponse; diff --git a/modules/rank-eval/src/internalClusterTest/java/org/opensearch/index/rankeval/RankEvalRequestIT.java b/modules/rank-eval/src/internalClusterTest/java/org/opensearch/index/rankeval/RankEvalRequestIT.java index ea80b59711b8a..cacccafde2d47 100644 --- a/modules/rank-eval/src/internalClusterTest/java/org/opensearch/index/rankeval/RankEvalRequestIT.java +++ b/modules/rank-eval/src/internalClusterTest/java/org/opensearch/index/rankeval/RankEvalRequestIT.java @@ -32,7 +32,7 @@ package org.opensearch.index.rankeval; -import org.opensearch.OpenSearchException; +import org.opensearch.BaseOpenSearchException; import org.opensearch.action.admin.indices.alias.IndicesAliasesRequest.AliasActions; import org.opensearch.action.search.SearchRequest; import org.opensearch.action.support.IndicesOptions; @@ -274,7 +274,7 @@ public void testBadQuery() { RankEvalResponse response = client().execute(RankEvalAction.INSTANCE, builder.request()).actionGet(); assertEquals(1, response.getFailures().size()); - OpenSearchException[] rootCauses = OpenSearchException.guessRootCauses(response.getFailures().get("broken_query")); + BaseOpenSearchException[] rootCauses = BaseOpenSearchException.guessRootCauses(response.getFailures().get("broken_query")); assertEquals("java.lang.NumberFormatException: For input string: \"noStringOnNumericFields\"", rootCauses[0].getCause().toString()); } diff --git a/modules/rank-eval/src/main/java/org/opensearch/index/rankeval/RankEvalRequest.java b/modules/rank-eval/src/main/java/org/opensearch/index/rankeval/RankEvalRequest.java index 66db397865a0b..64ff9148bc65e 100644 --- a/modules/rank-eval/src/main/java/org/opensearch/index/rankeval/RankEvalRequest.java +++ b/modules/rank-eval/src/main/java/org/opensearch/index/rankeval/RankEvalRequest.java @@ -39,9 +39,9 @@ import org.opensearch.action.search.SearchRequest; import org.opensearch.action.search.SearchType; import org.opensearch.action.support.IndicesOptions; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import java.io.IOException; import java.util.Arrays; 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 180069ab91bd6..58a940e1618da 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 @@ -32,6 +32,7 @@ package org.opensearch.index.rankeval; +import org.opensearch.BaseOpenSearchException; import org.opensearch.OpenSearchException; import org.opensearch.action.ActionResponse; import org.opensearch.core.ParseField; @@ -136,7 +137,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.startObject("failures"); for (String key : failures.keySet()) { builder.startObject(key); - OpenSearchException.generateFailureXContent(builder, params, failures.get(key), true); + BaseOpenSearchException.generateFailureXContent(builder, params, failures.get(key), true); builder.endObject(); } builder.endObject(); 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 0955996bed50f..63e05983409e3 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 @@ -32,7 +32,7 @@ package org.opensearch.index.rankeval; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.io.stream.Writeable; @@ -129,7 +129,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws @Override public String toString() { - return Strings.toString(XContentType.JSON, this); + return org.opensearch.common.Strings.toString(XContentType.JSON, this); } @Override diff --git a/modules/rank-eval/src/main/java/org/opensearch/index/rankeval/RestRankEvalAction.java b/modules/rank-eval/src/main/java/org/opensearch/index/rankeval/RestRankEvalAction.java index 670068fc0b250..cb5138117d0e1 100644 --- a/modules/rank-eval/src/main/java/org/opensearch/index/rankeval/RestRankEvalAction.java +++ b/modules/rank-eval/src/main/java/org/opensearch/index/rankeval/RestRankEvalAction.java @@ -35,7 +35,7 @@ import org.opensearch.action.search.SearchType; import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; diff --git a/modules/reindex/src/main/java/org/opensearch/index/reindex/ReindexValidator.java b/modules/reindex/src/main/java/org/opensearch/index/reindex/ReindexValidator.java index 71c3aad8713e1..d065afceab503 100644 --- a/modules/reindex/src/main/java/org/opensearch/index/reindex/ReindexValidator.java +++ b/modules/reindex/src/main/java/org/opensearch/index/reindex/ReindexValidator.java @@ -45,10 +45,10 @@ import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexNameExpressionResolver; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.Strings; import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.regex.Regex; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; import org.opensearch.index.IndexNotFoundException; import org.opensearch.search.builder.SearchSourceBuilder; diff --git a/modules/reindex/src/main/java/org/opensearch/index/reindex/Reindexer.java b/modules/reindex/src/main/java/org/opensearch/index/reindex/Reindexer.java index bb1723818b8f1..082f84f08c9f7 100644 --- a/modules/reindex/src/main/java/org/opensearch/index/reindex/Reindexer.java +++ b/modules/reindex/src/main/java/org/opensearch/index/reindex/Reindexer.java @@ -54,9 +54,9 @@ import org.opensearch.client.RestClient; import org.opensearch.client.RestClientBuilder; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.Strings; import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.lucene.uid.Versions; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.DeprecationHandler; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.core.xcontent.XContentBuilder; diff --git a/modules/reindex/src/main/java/org/opensearch/index/reindex/remote/RemoteScrollableHitSource.java b/modules/reindex/src/main/java/org/opensearch/index/reindex/remote/RemoteScrollableHitSource.java index 66c62aeaf8a60..4500474706e07 100644 --- a/modules/reindex/src/main/java/org/opensearch/index/reindex/remote/RemoteScrollableHitSource.java +++ b/modules/reindex/src/main/java/org/opensearch/index/reindex/remote/RemoteScrollableHitSource.java @@ -49,11 +49,11 @@ import org.opensearch.client.ResponseListener; import org.opensearch.client.RestClient; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.ThreadContext; import org.opensearch.common.xcontent.LoggingDeprecationHandler; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.core.xcontent.XContentParseException; import org.opensearch.core.xcontent.XContentParser; diff --git a/modules/reindex/src/test/java/org/opensearch/index/reindex/AsyncBulkByScrollActionTests.java b/modules/reindex/src/test/java/org/opensearch/index/reindex/AsyncBulkByScrollActionTests.java index 61336156a83d0..e11ad3b5013c1 100644 --- a/modules/reindex/src/test/java/org/opensearch/index/reindex/AsyncBulkByScrollActionTests.java +++ b/modules/reindex/src/test/java/org/opensearch/index/reindex/AsyncBulkByScrollActionTests.java @@ -33,6 +33,7 @@ package org.opensearch.index.reindex; import org.apache.lucene.search.TotalHits; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.OpenSearchException; import org.opensearch.ExceptionsHelper; import org.opensearch.Version; @@ -215,7 +216,7 @@ public void testStartRetriesOnRejectionButFailsOnTooManyRejections() throws Exce assertBusy(() -> assertEquals(testRequest.getMaxRetries() + 1, client.searchAttempts.get())); assertBusy(() -> assertTrue(listener.isDone())); ExecutionException e = expectThrows(ExecutionException.class, () -> listener.get()); - assertThat(ExceptionsHelper.stackTrace(e), containsString(OpenSearchRejectedExecutionException.class.getSimpleName())); + assertThat(BaseExceptionsHelper.stackTrace(e), containsString(OpenSearchRejectedExecutionException.class.getSimpleName())); assertNull("There shouldn't be a search attempt pending that we didn't reject", client.lastSearch.get()); assertEquals(testRequest.getMaxRetries(), testTask.getStatus().getSearchRetries()); } diff --git a/plugins/discovery-azure-classic/src/main/java/org/opensearch/cloud/azure/classic/management/AzureComputeServiceImpl.java b/plugins/discovery-azure-classic/src/main/java/org/opensearch/cloud/azure/classic/management/AzureComputeServiceImpl.java index 6a8b8d83f539b..9dbf08a3e1a01 100644 --- a/plugins/discovery-azure-classic/src/main/java/org/opensearch/cloud/azure/classic/management/AzureComputeServiceImpl.java +++ b/plugins/discovery-azure-classic/src/main/java/org/opensearch/cloud/azure/classic/management/AzureComputeServiceImpl.java @@ -51,10 +51,10 @@ import org.opensearch.OpenSearchException; import org.opensearch.SpecialPermission; import org.opensearch.cloud.azure.classic.AzureServiceRemoteException; -import org.opensearch.common.Strings; import org.opensearch.common.component.AbstractLifecycleComponent; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; public class AzureComputeServiceImpl extends AbstractLifecycleComponent implements AzureComputeService { private static final Logger logger = LogManager.getLogger(AzureComputeServiceImpl.class); diff --git a/plugins/discovery-azure-classic/src/main/java/org/opensearch/discovery/azure/classic/AzureSeedHostsProvider.java b/plugins/discovery-azure-classic/src/main/java/org/opensearch/discovery/azure/classic/AzureSeedHostsProvider.java index 0fe6904e83242..e2bc180876a17 100644 --- a/plugins/discovery-azure-classic/src/main/java/org/opensearch/discovery/azure/classic/AzureSeedHostsProvider.java +++ b/plugins/discovery-azure-classic/src/main/java/org/opensearch/discovery/azure/classic/AzureSeedHostsProvider.java @@ -44,13 +44,13 @@ import org.opensearch.cloud.azure.classic.AzureServiceRemoteException; import org.opensearch.cloud.azure.classic.management.AzureComputeService; import org.opensearch.cloud.azure.classic.management.AzureComputeService.Discovery; -import org.opensearch.common.Strings; import org.opensearch.common.network.InetAddresses; import org.opensearch.common.network.NetworkAddress; import org.opensearch.common.network.NetworkService; import org.opensearch.common.settings.Settings; import org.opensearch.common.transport.TransportAddress; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import org.opensearch.discovery.SeedHostsProvider; import org.opensearch.transport.TransportService; diff --git a/plugins/discovery-ec2/src/internalClusterTest/java/org/opensearch/discovery/ec2/AbstractAwsTestCase.java b/plugins/discovery-ec2/src/internalClusterTest/java/org/opensearch/discovery/ec2/AbstractAwsTestCase.java index 1e7422ea0ac02..9529d1e67ea09 100644 --- a/plugins/discovery-ec2/src/internalClusterTest/java/org/opensearch/discovery/ec2/AbstractAwsTestCase.java +++ b/plugins/discovery-ec2/src/internalClusterTest/java/org/opensearch/discovery/ec2/AbstractAwsTestCase.java @@ -32,10 +32,10 @@ package org.opensearch.discovery.ec2; -import org.opensearch.common.Strings; import org.opensearch.common.io.PathUtils; import org.opensearch.common.settings.Settings; import org.opensearch.common.settings.SettingsException; +import org.opensearch.core.common.Strings; import org.opensearch.env.Environment; import org.opensearch.plugins.Plugin; import org.opensearch.test.OpenSearchIntegTestCase; diff --git a/plugins/discovery-ec2/src/main/java/org/opensearch/discovery/ec2/AwsEc2ServiceImpl.java b/plugins/discovery-ec2/src/main/java/org/opensearch/discovery/ec2/AwsEc2ServiceImpl.java index bdd293c33c524..0a9004dc0ca58 100644 --- a/plugins/discovery-ec2/src/main/java/org/opensearch/discovery/ec2/AwsEc2ServiceImpl.java +++ b/plugins/discovery-ec2/src/main/java/org/opensearch/discovery/ec2/AwsEc2ServiceImpl.java @@ -39,9 +39,9 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.opensearch.OpenSearchException; -import org.opensearch.common.Strings; import org.opensearch.common.util.LazyInitializable; import org.opensearch.common.SuppressForbidden; +import org.opensearch.core.common.Strings; import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider; import software.amazon.awssdk.core.exception.SdkException; diff --git a/plugins/discovery-ec2/src/test/java/org/opensearch/discovery/ec2/Ec2NetworkTests.java b/plugins/discovery-ec2/src/test/java/org/opensearch/discovery/ec2/Ec2NetworkTests.java index 1f1977869b005..c8cb26026228b 100644 --- a/plugins/discovery-ec2/src/test/java/org/opensearch/discovery/ec2/Ec2NetworkTests.java +++ b/plugins/discovery-ec2/src/test/java/org/opensearch/discovery/ec2/Ec2NetworkTests.java @@ -34,10 +34,10 @@ import com.sun.net.httpserver.HttpServer; -import org.opensearch.common.Strings; import org.opensearch.common.SuppressForbidden; import org.opensearch.common.network.NetworkService; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; import org.opensearch.rest.RestStatus; import org.junit.AfterClass; diff --git a/plugins/discovery-gce/src/main/java/org/opensearch/cloud/gce/network/GceNameResolver.java b/plugins/discovery-gce/src/main/java/org/opensearch/cloud/gce/network/GceNameResolver.java index 7482dfac401ef..28e41963de489 100644 --- a/plugins/discovery-gce/src/main/java/org/opensearch/cloud/gce/network/GceNameResolver.java +++ b/plugins/discovery-gce/src/main/java/org/opensearch/cloud/gce/network/GceNameResolver.java @@ -34,8 +34,8 @@ import org.opensearch.cloud.gce.GceMetadataService; import org.opensearch.cloud.gce.util.Access; -import org.opensearch.common.Strings; import org.opensearch.common.network.NetworkService.CustomNameResolver; +import org.opensearch.core.common.Strings; import java.io.IOException; import java.net.InetAddress; diff --git a/plugins/discovery-gce/src/main/java/org/opensearch/discovery/gce/GceSeedHostsProvider.java b/plugins/discovery-gce/src/main/java/org/opensearch/discovery/gce/GceSeedHostsProvider.java index 19247a7bb536d..dfd60f52730a6 100644 --- a/plugins/discovery-gce/src/main/java/org/opensearch/discovery/gce/GceSeedHostsProvider.java +++ b/plugins/discovery-gce/src/main/java/org/opensearch/discovery/gce/GceSeedHostsProvider.java @@ -41,7 +41,6 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.logging.log4j.util.Supplier; import org.opensearch.cloud.gce.GceInstancesService; -import org.opensearch.common.Strings; import org.opensearch.common.network.NetworkAddress; import org.opensearch.common.network.NetworkService; import org.opensearch.common.settings.Setting; @@ -49,6 +48,7 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.transport.TransportAddress; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import org.opensearch.discovery.SeedHostsProvider; import org.opensearch.transport.TransportService; diff --git a/plugins/discovery-gce/src/test/java/org/opensearch/discovery/gce/GceMockUtils.java b/plugins/discovery-gce/src/test/java/org/opensearch/discovery/gce/GceMockUtils.java index 24790aa4a900d..6d6028eca713a 100644 --- a/plugins/discovery-gce/src/test/java/org/opensearch/discovery/gce/GceMockUtils.java +++ b/plugins/discovery-gce/src/test/java/org/opensearch/discovery/gce/GceMockUtils.java @@ -41,7 +41,7 @@ import com.google.api.client.testing.http.MockLowLevelHttpResponse; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.util.FileSystemUtils; import org.opensearch.common.io.Streams; diff --git a/plugins/discovery-gce/src/test/java/org/opensearch/discovery/gce/GceNetworkTests.java b/plugins/discovery-gce/src/test/java/org/opensearch/discovery/gce/GceNetworkTests.java index a1d7613bf2ba4..091a4f720d258 100644 --- a/plugins/discovery-gce/src/test/java/org/opensearch/discovery/gce/GceNetworkTests.java +++ b/plugins/discovery-gce/src/test/java/org/opensearch/discovery/gce/GceNetworkTests.java @@ -33,9 +33,9 @@ package org.opensearch.discovery.gce; import org.opensearch.cloud.gce.network.GceNameResolver; -import org.opensearch.common.Strings; import org.opensearch.common.network.NetworkService; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; import org.opensearch.test.OpenSearchTestCase; import java.io.IOException; diff --git a/plugins/ingest-attachment/src/main/java/org/opensearch/ingest/attachment/AttachmentProcessor.java b/plugins/ingest-attachment/src/main/java/org/opensearch/ingest/attachment/AttachmentProcessor.java index b8e8300755686..3f7b341edc409 100644 --- a/plugins/ingest-attachment/src/main/java/org/opensearch/ingest/attachment/AttachmentProcessor.java +++ b/plugins/ingest-attachment/src/main/java/org/opensearch/ingest/attachment/AttachmentProcessor.java @@ -39,7 +39,7 @@ import org.apache.tika.metadata.TikaCoreProperties; import org.opensearch.OpenSearchParseException; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.ingest.AbstractProcessor; import org.opensearch.ingest.IngestDocument; import org.opensearch.ingest.Processor; diff --git a/plugins/mapper-annotated-text/src/test/java/org/opensearch/search/fetch/subphase/highlight/AnnotatedTextHighlighterTests.java b/plugins/mapper-annotated-text/src/test/java/org/opensearch/search/fetch/subphase/highlight/AnnotatedTextHighlighterTests.java index 17a1abc64e8cd..7e6cf47fa92fd 100644 --- a/plugins/mapper-annotated-text/src/test/java/org/opensearch/search/fetch/subphase/highlight/AnnotatedTextHighlighterTests.java +++ b/plugins/mapper-annotated-text/src/test/java/org/opensearch/search/fetch/subphase/highlight/AnnotatedTextHighlighterTests.java @@ -56,7 +56,7 @@ import org.apache.lucene.search.uhighlight.SplittingBreakIterator; import org.apache.lucene.store.Directory; import org.apache.lucene.tests.index.RandomIndexWriter; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.index.mapper.annotatedtext.AnnotatedTextFieldMapper.AnnotatedHighlighterAnalyzer; import org.opensearch.index.mapper.annotatedtext.AnnotatedTextFieldMapper.AnnotatedText; import org.opensearch.index.mapper.annotatedtext.AnnotatedTextFieldMapper.AnnotationAnalyzerWrapper; diff --git a/plugins/repository-azure/src/internalClusterTest/java/org/opensearch/repositories/azure/AzureStorageCleanupThirdPartyTests.java b/plugins/repository-azure/src/internalClusterTest/java/org/opensearch/repositories/azure/AzureStorageCleanupThirdPartyTests.java index 6d71a65a35a4c..1dc411a5a27fb 100644 --- a/plugins/repository-azure/src/internalClusterTest/java/org/opensearch/repositories/azure/AzureStorageCleanupThirdPartyTests.java +++ b/plugins/repository-azure/src/internalClusterTest/java/org/opensearch/repositories/azure/AzureStorageCleanupThirdPartyTests.java @@ -32,6 +32,7 @@ package org.opensearch.repositories.azure; +import org.opensearch.core.common.Strings; import reactor.core.scheduler.Schedulers; import com.azure.core.util.Context; @@ -43,7 +44,6 @@ import org.opensearch.action.ActionRunnable; import org.opensearch.action.support.PlainActionFuture; import org.opensearch.action.support.master.AcknowledgedResponse; -import org.opensearch.common.Strings; import org.opensearch.common.collect.Tuple; import org.opensearch.common.settings.MockSecureSettings; import org.opensearch.common.settings.SecureSettings; diff --git a/plugins/repository-azure/src/main/java/org/opensearch/repositories/azure/AzureRepository.java b/plugins/repository-azure/src/main/java/org/opensearch/repositories/azure/AzureRepository.java index abcf56a82ef4b..2677604ecb622 100644 --- a/plugins/repository-azure/src/main/java/org/opensearch/repositories/azure/AzureRepository.java +++ b/plugins/repository-azure/src/main/java/org/opensearch/repositories/azure/AzureRepository.java @@ -37,12 +37,12 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.cluster.metadata.RepositoryMetadata; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.Strings; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.blobstore.BlobStore; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Setting.Property; import org.opensearch.common.unit.ByteSizeValue; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.repositories.blobstore.MeteredBlobStoreRepository; diff --git a/plugins/repository-azure/src/main/java/org/opensearch/repositories/azure/AzureStorageSettings.java b/plugins/repository-azure/src/main/java/org/opensearch/repositories/azure/AzureStorageSettings.java index 4a9aa51334d0a..f7c8484a887ae 100644 --- a/plugins/repository-azure/src/main/java/org/opensearch/repositories/azure/AzureStorageSettings.java +++ b/plugins/repository-azure/src/main/java/org/opensearch/repositories/azure/AzureStorageSettings.java @@ -33,7 +33,6 @@ package org.opensearch.repositories.azure; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.collect.MapBuilder; import org.opensearch.common.settings.SecureSetting; import org.opensearch.common.settings.SecureString; @@ -43,6 +42,8 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.settings.SettingsException; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; + import java.net.InetAddress; import java.net.UnknownHostException; import java.util.Collections; diff --git a/plugins/repository-azure/src/main/java/org/opensearch/repositories/azure/ProxySettings.java b/plugins/repository-azure/src/main/java/org/opensearch/repositories/azure/ProxySettings.java index df8c95e69acf2..03097626c9741 100644 --- a/plugins/repository-azure/src/main/java/org/opensearch/repositories/azure/ProxySettings.java +++ b/plugins/repository-azure/src/main/java/org/opensearch/repositories/azure/ProxySettings.java @@ -9,8 +9,8 @@ package org.opensearch.repositories.azure; import com.azure.core.http.ProxyOptions; -import org.opensearch.common.Strings; import org.opensearch.common.settings.SettingsException; +import org.opensearch.core.common.Strings; import java.net.InetAddress; import java.net.InetSocketAddress; diff --git a/plugins/repository-azure/src/test/java/org/opensearch/repositories/azure/AzureBlobContainerRetriesTests.java b/plugins/repository-azure/src/test/java/org/opensearch/repositories/azure/AzureBlobContainerRetriesTests.java index 05c750832d231..63fad5774a3a4 100644 --- a/plugins/repository-azure/src/test/java/org/opensearch/repositories/azure/AzureBlobContainerRetriesTests.java +++ b/plugins/repository-azure/src/test/java/org/opensearch/repositories/azure/AzureBlobContainerRetriesTests.java @@ -38,12 +38,12 @@ import com.sun.net.httpserver.HttpExchange; import com.sun.net.httpserver.HttpServer; import fixture.azure.AzureHttpHandler; +import org.opensearch.core.common.Strings; import reactor.core.scheduler.Schedulers; import org.apache.http.HttpStatus; import org.opensearch.cluster.metadata.RepositoryMetadata; -import org.opensearch.common.Strings; import org.opensearch.common.SuppressForbidden; import org.opensearch.common.blobstore.BlobContainer; import org.opensearch.common.blobstore.BlobPath; diff --git a/plugins/repository-azure/src/test/java/org/opensearch/repositories/azure/AzureStorageServiceTests.java b/plugins/repository-azure/src/test/java/org/opensearch/repositories/azure/AzureStorageServiceTests.java index 7f5ca73a507ad..1ddf8bb1e94e1 100644 --- a/plugins/repository-azure/src/test/java/org/opensearch/repositories/azure/AzureStorageServiceTests.java +++ b/plugins/repository-azure/src/test/java/org/opensearch/repositories/azure/AzureStorageServiceTests.java @@ -32,7 +32,6 @@ package org.opensearch.repositories.azure; -import org.opensearch.common.Strings; import reactor.core.scheduler.Schedulers; import com.azure.core.http.policy.HttpPipelinePolicy; @@ -45,6 +44,7 @@ import org.opensearch.common.settings.SettingsException; import org.opensearch.common.settings.SettingsModule; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import org.opensearch.test.OpenSearchTestCase; import java.io.IOException; diff --git a/plugins/repository-gcs/src/internalClusterTest/java/org/opensearch/repositories/gcs/GoogleCloudStorageThirdPartyTests.java b/plugins/repository-gcs/src/internalClusterTest/java/org/opensearch/repositories/gcs/GoogleCloudStorageThirdPartyTests.java index f1b2f78a37380..1e11b1d111d8f 100644 --- a/plugins/repository-gcs/src/internalClusterTest/java/org/opensearch/repositories/gcs/GoogleCloudStorageThirdPartyTests.java +++ b/plugins/repository-gcs/src/internalClusterTest/java/org/opensearch/repositories/gcs/GoogleCloudStorageThirdPartyTests.java @@ -33,10 +33,10 @@ package org.opensearch.repositories.gcs; import org.opensearch.action.support.master.AcknowledgedResponse; -import org.opensearch.common.Strings; import org.opensearch.common.settings.MockSecureSettings; import org.opensearch.common.settings.SecureSettings; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; import org.opensearch.plugins.Plugin; import org.opensearch.repositories.AbstractThirdPartyRepositoryTestCase; diff --git a/plugins/repository-gcs/src/main/java/org/opensearch/repositories/gcs/GoogleCloudStorageClientSettings.java b/plugins/repository-gcs/src/main/java/org/opensearch/repositories/gcs/GoogleCloudStorageClientSettings.java index e8700570d2801..d31cac55ffd64 100644 --- a/plugins/repository-gcs/src/main/java/org/opensearch/repositories/gcs/GoogleCloudStorageClientSettings.java +++ b/plugins/repository-gcs/src/main/java/org/opensearch/repositories/gcs/GoogleCloudStorageClientSettings.java @@ -34,13 +34,13 @@ import com.google.api.services.storage.StorageScopes; import com.google.auth.oauth2.ServiceAccountCredentials; -import org.opensearch.common.Strings; import org.opensearch.common.settings.SecureSetting; import org.opensearch.common.settings.SecureString; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; import org.opensearch.common.settings.SettingsException; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import java.io.IOException; import java.io.InputStream; diff --git a/plugins/repository-gcs/src/main/java/org/opensearch/repositories/gcs/GoogleCloudStorageRepository.java b/plugins/repository-gcs/src/main/java/org/opensearch/repositories/gcs/GoogleCloudStorageRepository.java index 86f5521d44dbb..a743ac72bdb8b 100644 --- a/plugins/repository-gcs/src/main/java/org/opensearch/repositories/gcs/GoogleCloudStorageRepository.java +++ b/plugins/repository-gcs/src/main/java/org/opensearch/repositories/gcs/GoogleCloudStorageRepository.java @@ -36,11 +36,11 @@ import org.apache.logging.log4j.Logger; import org.opensearch.cluster.metadata.RepositoryMetadata; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.Strings; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.settings.Setting; import org.opensearch.common.unit.ByteSizeUnit; import org.opensearch.common.unit.ByteSizeValue; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.repositories.RepositoryException; diff --git a/plugins/repository-gcs/src/main/java/org/opensearch/repositories/gcs/GoogleCloudStorageService.java b/plugins/repository-gcs/src/main/java/org/opensearch/repositories/gcs/GoogleCloudStorageService.java index f4b501327d52c..7211eac59ad4e 100644 --- a/plugins/repository-gcs/src/main/java/org/opensearch/repositories/gcs/GoogleCloudStorageService.java +++ b/plugins/repository-gcs/src/main/java/org/opensearch/repositories/gcs/GoogleCloudStorageService.java @@ -45,9 +45,9 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; -import org.opensearch.common.Strings; import org.opensearch.common.collect.MapBuilder; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import java.io.IOException; import java.net.Authenticator; diff --git a/plugins/repository-gcs/src/main/java/org/opensearch/repositories/gcs/ProxySettings.java b/plugins/repository-gcs/src/main/java/org/opensearch/repositories/gcs/ProxySettings.java index ddc6446d2c8c5..11f0cbd83e62e 100644 --- a/plugins/repository-gcs/src/main/java/org/opensearch/repositories/gcs/ProxySettings.java +++ b/plugins/repository-gcs/src/main/java/org/opensearch/repositories/gcs/ProxySettings.java @@ -8,7 +8,7 @@ package org.opensearch.repositories.gcs; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import java.net.InetAddress; import java.net.InetSocketAddress; diff --git a/plugins/repository-gcs/src/test/java/org/opensearch/repositories/gcs/GoogleCloudStorageBlobContainerRetriesTests.java b/plugins/repository-gcs/src/test/java/org/opensearch/repositories/gcs/GoogleCloudStorageBlobContainerRetriesTests.java index 616a1ae9feb4f..8cd7b1adc31bb 100644 --- a/plugins/repository-gcs/src/test/java/org/opensearch/repositories/gcs/GoogleCloudStorageBlobContainerRetriesTests.java +++ b/plugins/repository-gcs/src/test/java/org/opensearch/repositories/gcs/GoogleCloudStorageBlobContainerRetriesTests.java @@ -40,7 +40,6 @@ import org.apache.http.HttpStatus; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.SuppressForbidden; import org.opensearch.common.UUIDs; import org.opensearch.common.blobstore.BlobContainer; @@ -57,6 +56,7 @@ import org.opensearch.common.unit.ByteSizeValue; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.CountDown; +import org.opensearch.core.common.Strings; import org.opensearch.repositories.blobstore.AbstractBlobContainerRetriesTestCase; import org.opensearch.repositories.blobstore.OpenSearchMockAPIBasedRepositoryIntegTestCase; import org.opensearch.rest.RestStatus; diff --git a/plugins/repository-hdfs/src/main/java/org/opensearch/repositories/hdfs/HdfsRepository.java b/plugins/repository-hdfs/src/main/java/org/opensearch/repositories/hdfs/HdfsRepository.java index 978b8dec2dc2e..88c58942e9bbf 100644 --- a/plugins/repository-hdfs/src/main/java/org/opensearch/repositories/hdfs/HdfsRepository.java +++ b/plugins/repository-hdfs/src/main/java/org/opensearch/repositories/hdfs/HdfsRepository.java @@ -45,11 +45,11 @@ import org.opensearch.SpecialPermission; import org.opensearch.cluster.metadata.RepositoryMetadata; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.Strings; import org.opensearch.common.SuppressForbidden; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.ByteSizeValue; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.env.Environment; import org.opensearch.indices.recovery.RecoverySettings; diff --git a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/ProxySettings.java b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/ProxySettings.java index ca063b5e73ea0..828974c7b93b9 100644 --- a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/ProxySettings.java +++ b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/ProxySettings.java @@ -8,8 +8,8 @@ package org.opensearch.repositories.s3; -import org.opensearch.common.Strings; import org.opensearch.common.settings.SettingsException; +import org.opensearch.core.common.Strings; import org.opensearch.repositories.s3.utils.Protocol; import software.amazon.awssdk.core.exception.SdkException; diff --git a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3BlobContainer.java b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3BlobContainer.java index 8078fcd9b27d1..005626a4c7e7f 100644 --- a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3BlobContainer.java +++ b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3BlobContainer.java @@ -39,7 +39,6 @@ import org.opensearch.action.ActionListener; import org.opensearch.common.Nullable; import org.opensearch.common.SetOnce; -import org.opensearch.common.Strings; import org.opensearch.common.blobstore.BlobContainer; import org.opensearch.common.blobstore.BlobMetadata; import org.opensearch.common.blobstore.BlobPath; @@ -50,6 +49,7 @@ import org.opensearch.common.collect.Tuple; import org.opensearch.common.unit.ByteSizeUnit; import org.opensearch.common.unit.ByteSizeValue; +import org.opensearch.core.common.Strings; import software.amazon.awssdk.core.exception.SdkException; import software.amazon.awssdk.core.sync.RequestBody; import software.amazon.awssdk.services.s3.model.AbortMultipartUploadRequest; diff --git a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3ClientSettings.java b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3ClientSettings.java index 69d4ca0e12860..9ce72533d50e5 100644 --- a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3ClientSettings.java +++ b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3ClientSettings.java @@ -32,7 +32,6 @@ package org.opensearch.repositories.s3; -import org.opensearch.common.Strings; import org.opensearch.common.SuppressForbidden; import org.opensearch.common.io.PathUtils; import org.opensearch.common.logging.DeprecationLogger; @@ -43,6 +42,7 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.settings.SettingsException; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import org.opensearch.repositories.s3.utils.Protocol; import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; import software.amazon.awssdk.auth.credentials.AwsCredentials; diff --git a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3Repository.java b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3Repository.java index 335a944d465e4..b59233655032c 100644 --- a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3Repository.java +++ b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3Repository.java @@ -43,7 +43,6 @@ import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.metadata.RepositoryMetadata; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.Strings; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.blobstore.BlobStore; import org.opensearch.common.logging.DeprecationLogger; @@ -53,6 +52,7 @@ import org.opensearch.common.unit.ByteSizeUnit; import org.opensearch.common.unit.ByteSizeValue; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.monitor.jvm.JvmInfo; diff --git a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3Service.java b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3Service.java index 749808cb462bc..c13e5b76b9269 100644 --- a/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3Service.java +++ b/plugins/repository-s3/src/main/java/org/opensearch/repositories/s3/S3Service.java @@ -39,10 +39,10 @@ import org.apache.logging.log4j.Logger; import org.opensearch.cluster.metadata.RepositoryMetadata; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.SuppressForbidden; import org.opensearch.common.collect.MapBuilder; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; import org.opensearch.repositories.s3.S3ClientSettings.IrsaCredentials; import org.opensearch.repositories.s3.utils.Protocol; import org.opensearch.repositories.s3.utils.AwsRequestSigner; diff --git a/qa/die-with-dignity/src/javaRestTest/java/org/opensearch/qa/die_with_dignity/DieWithDignityIT.java b/qa/die-with-dignity/src/javaRestTest/java/org/opensearch/qa/die_with_dignity/DieWithDignityIT.java index ec891ef8d44ef..8a2bb3e8be38a 100644 --- a/qa/die-with-dignity/src/javaRestTest/java/org/opensearch/qa/die_with_dignity/DieWithDignityIT.java +++ b/qa/die-with-dignity/src/javaRestTest/java/org/opensearch/qa/die_with_dignity/DieWithDignityIT.java @@ -76,7 +76,7 @@ public void testDieWithDignity() throws Exception { try { while (it.hasNext() && (fatalError == false || fatalErrorInThreadExiting == false)) { final String line = it.next(); - if (line.matches(".*ERROR.*o\\.o\\.ExceptionsHelper.*javaRestTest-0.*fatal error.*")) { + if (line.matches(".*ERROR.*o\\.o\\.(Base)?ExceptionsHelper.*javaRestTest-0.*fatal error.*")) { fatalError = true; } else if (line.matches( ".*ERROR.*o\\.o\\.b\\.OpenSearchUncaughtExceptionHandler.*javaRestTest-0.*" diff --git a/qa/mixed-cluster/src/test/java/org/opensearch/backwards/SearchingIT.java b/qa/mixed-cluster/src/test/java/org/opensearch/backwards/SearchingIT.java index 2cb819a3f6f27..183c7955e10db 100644 --- a/qa/mixed-cluster/src/test/java/org/opensearch/backwards/SearchingIT.java +++ b/qa/mixed-cluster/src/test/java/org/opensearch/backwards/SearchingIT.java @@ -38,7 +38,7 @@ public void testMultiGet() throws Exception { try (RestHighLevelClient client = new RestHighLevelClient(RestClient.builder(nodes.toArray(HttpHost[]::new)))) { MultiGetResponse response = client.mget(multiGetRequest, RequestOptions.DEFAULT); assertEquals(1, response.getResponses().length); - + assertTrue(response.getResponses()[0].isFailed()); assertNotNull(response.getResponses()[0].getFailure()); assertEquals(response.getResponses()[0].getFailure().getId(), "id1"); diff --git a/server/src/internalClusterTest/java/org/opensearch/action/RejectionActionIT.java b/server/src/internalClusterTest/java/org/opensearch/action/RejectionActionIT.java index bda24b48b7f10..f2f55b238ab7c 100644 --- a/server/src/internalClusterTest/java/org/opensearch/action/RejectionActionIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/action/RejectionActionIT.java @@ -32,7 +32,7 @@ package org.opensearch.action; -import org.opensearch.ExceptionsHelper; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.action.search.SearchPhaseExecutionException; import org.opensearch.action.search.SearchResponse; import org.opensearch.action.search.SearchType; @@ -105,7 +105,7 @@ public void onFailure(Exception e) { } } else { Exception t = (Exception) response; - Throwable unwrap = ExceptionsHelper.unwrapCause(t); + Throwable unwrap = BaseExceptionsHelper.unwrapCause(t); if (unwrap instanceof SearchPhaseExecutionException) { SearchPhaseExecutionException e = (SearchPhaseExecutionException) unwrap; for (ShardSearchFailure failure : e.shardFailures()) { diff --git a/server/src/internalClusterTest/java/org/opensearch/action/admin/ReloadSecureSettingsIT.java b/server/src/internalClusterTest/java/org/opensearch/action/admin/ReloadSecureSettingsIT.java index 7e6dad47121a9..abc567c4ccf88 100644 --- a/server/src/internalClusterTest/java/org/opensearch/action/admin/ReloadSecureSettingsIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/action/admin/ReloadSecureSettingsIT.java @@ -35,11 +35,11 @@ import org.opensearch.OpenSearchException; import org.opensearch.action.ActionListener; import org.opensearch.action.admin.cluster.node.reload.NodesReloadSecureSettingsResponse; -import org.opensearch.common.Strings; import org.opensearch.common.settings.KeyStoreWrapper; import org.opensearch.common.settings.SecureSettings; import org.opensearch.common.settings.SecureString; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; import org.opensearch.env.Environment; import org.opensearch.plugins.Plugin; import org.opensearch.plugins.PluginsService; diff --git a/server/src/internalClusterTest/java/org/opensearch/action/admin/cluster/node/tasks/AbstractTasksIT.java b/server/src/internalClusterTest/java/org/opensearch/action/admin/cluster/node/tasks/AbstractTasksIT.java index fcfe9cb0aab00..84e7d82d25ab2 100644 --- a/server/src/internalClusterTest/java/org/opensearch/action/admin/cluster/node/tasks/AbstractTasksIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/action/admin/cluster/node/tasks/AbstractTasksIT.java @@ -14,10 +14,10 @@ import org.opensearch.action.support.WriteRequest; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.Strings; import org.opensearch.common.collect.Tuple; import org.opensearch.common.settings.Settings; import org.opensearch.common.xcontent.XContentType; +import org.opensearch.core.common.Strings; import org.opensearch.plugins.Plugin; import org.opensearch.tasks.TaskId; import org.opensearch.tasks.TaskInfo; diff --git a/server/src/internalClusterTest/java/org/opensearch/action/search/TransportSearchIT.java b/server/src/internalClusterTest/java/org/opensearch/action/search/TransportSearchIT.java index 321f66627ccf0..895d7ebea88b6 100644 --- a/server/src/internalClusterTest/java/org/opensearch/action/search/TransportSearchIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/action/search/TransportSearchIT.java @@ -36,7 +36,7 @@ import org.apache.lucene.search.CollectionTerminatedException; import org.apache.lucene.search.ScoreMode; -import org.opensearch.ExceptionsHelper; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.action.ActionListener; import org.opensearch.action.admin.cluster.node.stats.NodeStats; import org.opensearch.action.admin.cluster.node.stats.NodesStatsRequest; @@ -47,13 +47,13 @@ import org.opensearch.action.support.WriteRequest; import org.opensearch.client.Client; import org.opensearch.cluster.metadata.IndexMetadata; -import org.opensearch.common.Strings; import org.opensearch.common.breaker.CircuitBreaker; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.AtomicArray; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ObjectParser; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.index.IndexSettings; @@ -415,7 +415,7 @@ public void onFailure(Exception e) { SearchPhaseExecutionException.class, () -> client.prepareSearch("test").addAggregation(new TestAggregationBuilder("test")).get() ); - assertThat(ExceptionsHelper.unwrapCause(exc).getCause().getMessage(), containsString("")); + assertThat(BaseExceptionsHelper.unwrapCause(exc).getCause().getMessage(), containsString("")); }); final AtomicArray exceptions = new AtomicArray<>(10); @@ -443,7 +443,7 @@ public void onFailure(Exception exc) { latch.await(); assertThat(exceptions.asList().size(), equalTo(10)); for (Exception exc : exceptions.asList()) { - assertThat(ExceptionsHelper.unwrapCause(exc).getCause().getMessage(), containsString("")); + assertThat(BaseExceptionsHelper.unwrapCause(exc).getCause().getMessage(), containsString("")); } assertBusy(() -> assertThat(requestBreakerUsed(), equalTo(0L))); } finally { @@ -482,7 +482,7 @@ public void onFailure(Exception exc) { latch.await(); assertThat(exceptions.asList().size(), equalTo(10)); for (Exception exc : exceptions.asList()) { - assertThat(ExceptionsHelper.unwrapCause(exc).getCause().getMessage(), containsString("boom")); + assertThat(BaseExceptionsHelper.unwrapCause(exc).getCause().getMessage(), containsString("boom")); } assertBusy(() -> assertThat(requestBreakerUsed(), equalTo(0L))); } diff --git a/server/src/internalClusterTest/java/org/opensearch/action/termvectors/GetTermVectorsIT.java b/server/src/internalClusterTest/java/org/opensearch/action/termvectors/GetTermVectorsIT.java index cb9a81027d492..e84f40b3813eb 100644 --- a/server/src/internalClusterTest/java/org/opensearch/action/termvectors/GetTermVectorsIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/action/termvectors/GetTermVectorsIT.java @@ -43,9 +43,9 @@ import org.opensearch.action.admin.cluster.shards.ClusterSearchShardsResponse; import org.opensearch.action.admin.indices.alias.Alias; import org.opensearch.action.index.IndexRequestBuilder; -import org.opensearch.common.Strings; import org.opensearch.common.lucene.uid.Versions; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.index.engine.VersionConflictEngineException; diff --git a/server/src/internalClusterTest/java/org/opensearch/blocks/SimpleBlocksIT.java b/server/src/internalClusterTest/java/org/opensearch/blocks/SimpleBlocksIT.java index 8ede3e25b2e1a..872da7bb12b8e 100644 --- a/server/src/internalClusterTest/java/org/opensearch/blocks/SimpleBlocksIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/blocks/SimpleBlocksIT.java @@ -32,7 +32,7 @@ package org.opensearch.blocks; -import org.opensearch.ExceptionsHelper; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.action.ActionRequestValidationException; import org.opensearch.action.admin.indices.create.CreateIndexResponse; import org.opensearch.action.admin.indices.exists.indices.IndicesExistsResponse; @@ -428,7 +428,7 @@ public void testAddBlockWhileIndexingDocuments() throws Exception { try { try (BackgroundIndexer indexer = new BackgroundIndexer(indexName, "_doc", client(), 1000)) { indexer.setFailureAssertion(t -> { - Throwable cause = ExceptionsHelper.unwrapCause(t); + Throwable cause = BaseExceptionsHelper.unwrapCause(t); assertThat(cause, instanceOf(ClusterBlockException.class)); ClusterBlockException e = (ClusterBlockException) cause; assertThat(e.blocks(), hasSize(1)); @@ -474,7 +474,7 @@ public void testAddBlockWhileDeletingIndices() throws Exception { final APIBlock block = randomAddableBlock(); Consumer exceptionConsumer = t -> { - Throwable cause = ExceptionsHelper.unwrapCause(t); + Throwable cause = BaseExceptionsHelper.unwrapCause(t); if (cause instanceof ClusterBlockException) { ClusterBlockException e = (ClusterBlockException) cause; assertThat(e.blocks(), hasSize(1)); diff --git a/server/src/internalClusterTest/java/org/opensearch/cluster/ClusterInfoServiceIT.java b/server/src/internalClusterTest/java/org/opensearch/cluster/ClusterInfoServiceIT.java index 7ec5daf2b908b..17e8526acfd74 100644 --- a/server/src/internalClusterTest/java/org/opensearch/cluster/ClusterInfoServiceIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/cluster/ClusterInfoServiceIT.java @@ -45,9 +45,9 @@ import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.allocation.decider.EnableAllocationDecider; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import org.opensearch.index.IndexService; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.store.Store; diff --git a/server/src/internalClusterTest/java/org/opensearch/cluster/SimpleClusterStateIT.java b/server/src/internalClusterTest/java/org/opensearch/cluster/SimpleClusterStateIT.java index c1cfbbbf1fda4..d2c4859e35b1d 100644 --- a/server/src/internalClusterTest/java/org/opensearch/cluster/SimpleClusterStateIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/cluster/SimpleClusterStateIT.java @@ -45,13 +45,13 @@ import org.opensearch.cluster.routing.RoutingTable; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.Priority; -import org.opensearch.common.Strings; import org.opensearch.common.UUIDs; import org.opensearch.common.io.stream.NamedWriteableRegistry; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.ByteSizeValue; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.common.xcontent.XContentFactory; diff --git a/server/src/internalClusterTest/java/org/opensearch/cluster/allocation/FilteringAllocationIT.java b/server/src/internalClusterTest/java/org/opensearch/cluster/allocation/FilteringAllocationIT.java index 398adbd0d1ca5..ff95cca5ffde9 100644 --- a/server/src/internalClusterTest/java/org/opensearch/cluster/allocation/FilteringAllocationIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/cluster/allocation/FilteringAllocationIT.java @@ -42,9 +42,9 @@ import org.opensearch.cluster.routing.ShardRoutingState; import org.opensearch.cluster.routing.allocation.decider.FilterAllocationDecider; import org.opensearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; import org.opensearch.index.query.QueryBuilders; import org.opensearch.test.OpenSearchIntegTestCase; import org.opensearch.test.OpenSearchIntegTestCase.ClusterScope; diff --git a/server/src/internalClusterTest/java/org/opensearch/discovery/StableClusterManagerDisruptionIT.java b/server/src/internalClusterTest/java/org/opensearch/discovery/StableClusterManagerDisruptionIT.java index ffbaa35f61b72..c12718704e194 100644 --- a/server/src/internalClusterTest/java/org/opensearch/discovery/StableClusterManagerDisruptionIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/discovery/StableClusterManagerDisruptionIT.java @@ -42,10 +42,10 @@ import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.Priority; -import org.opensearch.common.Strings; import org.opensearch.common.collect.Tuple; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.set.Sets; +import org.opensearch.core.common.Strings; import org.opensearch.plugins.Plugin; import org.opensearch.test.OpenSearchIntegTestCase; import org.opensearch.test.disruption.LongGCDisruption; diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/IndicesLifecycleListenerIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/IndicesLifecycleListenerIT.java index 66226fd81da78..610cb7020054e 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/IndicesLifecycleListenerIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/IndicesLifecycleListenerIT.java @@ -44,8 +44,8 @@ import org.opensearch.cluster.routing.allocation.decider.EnableAllocationDecider; import org.opensearch.common.CheckedRunnable; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; import org.opensearch.index.Index; import org.opensearch.index.shard.IndexEventListener; import org.opensearch.index.shard.IndexShard; diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/IndicesOptionsIntegrationIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/IndicesOptionsIntegrationIT.java index 1f3d865811939..4d0399080b814 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/IndicesOptionsIntegrationIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/IndicesOptionsIntegrationIT.java @@ -52,10 +52,10 @@ import org.opensearch.action.search.SearchResponse; import org.opensearch.action.support.IndicesOptions; import org.opensearch.action.support.master.AcknowledgedResponse; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Setting.Property; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; import org.opensearch.index.IndexNotFoundException; import org.opensearch.plugins.Plugin; import org.opensearch.test.OpenSearchIntegTestCase; diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/memory/breaker/CircuitBreakerServiceIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/memory/breaker/CircuitBreakerServiceIT.java index 2dc6b2085b866..6ef3848cec9bb 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/memory/breaker/CircuitBreakerServiceIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/memory/breaker/CircuitBreakerServiceIT.java @@ -32,7 +32,7 @@ package org.opensearch.indices.memory.breaker; -import org.opensearch.ExceptionsHelper; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.action.admin.cluster.node.stats.NodeStats; import org.opensearch.action.admin.cluster.node.stats.NodesStatsResponse; import org.opensearch.action.bulk.BulkItemResponse; @@ -421,7 +421,7 @@ public void testLimitsRequestSize() { } else { // each item must have failed with CircuitBreakingException for (BulkItemResponse bulkItemResponse : response) { - Throwable cause = ExceptionsHelper.unwrapCause(bulkItemResponse.getFailure().getCause()); + Throwable cause = BaseExceptionsHelper.unwrapCause(bulkItemResponse.getFailure().getCause()); assertThat(cause, instanceOf(CircuitBreakingException.class)); assertEquals(((CircuitBreakingException) cause).getByteLimit(), inFlightRequestsLimit.getBytes()); } diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/state/CloseIndexIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/state/CloseIndexIT.java index 28bd5a6ae252d..995f11bcd79c6 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/state/CloseIndexIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/state/CloseIndexIT.java @@ -32,7 +32,7 @@ package org.opensearch.indices.state; -import org.opensearch.ExceptionsHelper; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.action.ActionRequestValidationException; import org.opensearch.action.admin.indices.close.CloseIndexRequestBuilder; import org.opensearch.action.admin.indices.close.CloseIndexResponse; @@ -647,7 +647,7 @@ static void assertIndexIsOpened(final String... indices) { } static void assertException(final Throwable throwable, final String indexName) { - final Throwable t = ExceptionsHelper.unwrapCause(throwable); + final Throwable t = BaseExceptionsHelper.unwrapCause(throwable); if (t instanceof ClusterBlockException) { ClusterBlockException clusterBlockException = (ClusterBlockException) t; assertThat(clusterBlockException.blocks(), hasSize(1)); diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/state/ReopenWhileClosingIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/state/ReopenWhileClosingIT.java index 028baa0176367..3bd349fbdd575 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/state/ReopenWhileClosingIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/state/ReopenWhileClosingIT.java @@ -39,10 +39,10 @@ import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.common.Glob; -import org.opensearch.common.Strings; import org.opensearch.common.lease.Releasable; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.concurrent.RunOnce; +import org.opensearch.core.common.Strings; import org.opensearch.plugins.Plugin; import org.opensearch.test.OpenSearchIntegTestCase; import org.opensearch.test.transport.MockTransportService; diff --git a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/AggregationsIntegrationIT.java b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/AggregationsIntegrationIT.java index b73b7722f9728..60a7da4f4912a 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/AggregationsIntegrationIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/AggregationsIntegrationIT.java @@ -32,7 +32,7 @@ package org.opensearch.search.aggregations; -import org.opensearch.OpenSearchException; +import org.opensearch.BaseOpenSearchException; import org.opensearch.action.index.IndexRequestBuilder; import org.opensearch.action.search.SearchPhaseExecutionException; import org.opensearch.action.search.SearchResponse; @@ -137,7 +137,7 @@ private void runLargeStringAggregationTest(AggregationBuilder aggregation) { exceptionThrown = true; Throwable nestedException = ex.getCause(); assertNotNull(nestedException); - assertTrue(nestedException instanceof OpenSearchException); + assertTrue(nestedException instanceof BaseOpenSearchException); assertNotNull(nestedException.getCause()); assertTrue(nestedException.getCause() instanceof IllegalArgumentException); String actualExceptionMessage = nestedException.getCause().getMessage(); 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..6f97b8ef40ffe 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 @@ -31,7 +31,7 @@ package org.opensearch.search.aggregations.bucket; -import org.opensearch.OpenSearchException; +import org.opensearch.BaseOpenSearchException; import org.opensearch.action.index.IndexRequestBuilder; import org.opensearch.action.search.SearchPhaseExecutionException; import org.opensearch.action.search.SearchResponse; @@ -650,9 +650,9 @@ public void testSingleValuedFieldOrderedByIllegalAgg() throws Exception { .get(); fail("Expected an exception"); } catch (SearchPhaseExecutionException e) { - OpenSearchException[] rootCauses = e.guessRootCauses(); + BaseOpenSearchException[] rootCauses = e.guessRootCauses(); if (rootCauses.length == 1) { - OpenSearchException rootCause = rootCauses[0]; + BaseOpenSearchException rootCause = rootCauses[0]; if (rootCause instanceof AggregationExecutionException) { AggregationExecutionException aggException = (AggregationExecutionException) rootCause; assertThat(aggException.getMessage(), Matchers.startsWith("Invalid aggregation order path")); diff --git a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/HistogramIT.java b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/HistogramIT.java index dae788abe0d10..e56c2b3c713bd 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/HistogramIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/HistogramIT.java @@ -32,7 +32,7 @@ package org.opensearch.search.aggregations.bucket; import com.carrotsearch.hppc.LongHashSet; -import org.opensearch.OpenSearchException; +import org.opensearch.BaseOpenSearchException; import org.opensearch.action.index.IndexRequestBuilder; import org.opensearch.action.search.SearchPhaseExecutionException; import org.opensearch.action.search.SearchResponse; @@ -689,9 +689,9 @@ public void testSingleValuedFieldOrderedByIllegalAgg() throws Exception { .get(); fail("Expected an exception"); } catch (SearchPhaseExecutionException e) { - OpenSearchException[] rootCauses = e.guessRootCauses(); + BaseOpenSearchException[] rootCauses = e.guessRootCauses(); if (rootCauses.length == 1) { - OpenSearchException rootCause = rootCauses[0]; + BaseOpenSearchException rootCause = rootCauses[0]; if (rootCause instanceof AggregationExecutionException) { AggregationExecutionException aggException = (AggregationExecutionException) rootCause; assertThat(aggException.getMessage(), Matchers.startsWith("Invalid aggregation order path")); diff --git a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/terms/StringTermsIT.java b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/terms/StringTermsIT.java index fa8e823545b36..96b256dc81ce1 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/terms/StringTermsIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/bucket/terms/StringTermsIT.java @@ -31,6 +31,7 @@ package org.opensearch.search.aggregations.bucket.terms; +import org.opensearch.BaseOpenSearchException; import org.opensearch.OpenSearchException; import org.opensearch.action.search.SearchPhaseExecutionException; import org.opensearch.action.search.SearchResponse; @@ -420,9 +421,9 @@ public void testSingleValuedFieldOrderedByIllegalAgg() throws Exception { .get(); fail("Expected an exception"); } catch (SearchPhaseExecutionException e) { - OpenSearchException[] rootCauses = e.guessRootCauses(); + BaseOpenSearchException[] rootCauses = e.guessRootCauses(); if (rootCauses.length == 1) { - OpenSearchException rootCause = rootCauses[0]; + BaseOpenSearchException rootCause = rootCauses[0]; if (rootCause instanceof AggregationExecutionException) { AggregationExecutionException aggException = (AggregationExecutionException) rootCause; assertThat(aggException.getMessage(), startsWith("Invalid aggregation order path")); diff --git a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/pipeline/DerivativeIT.java b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/pipeline/DerivativeIT.java index 406c57d044259..eb6153493323c 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/pipeline/DerivativeIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/pipeline/DerivativeIT.java @@ -32,7 +32,7 @@ package org.opensearch.search.aggregations.pipeline; -import org.opensearch.ExceptionsHelper; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.action.index.IndexRequestBuilder; import org.opensearch.action.search.SearchPhaseExecutionException; import org.opensearch.action.search.SearchResponse; @@ -639,7 +639,7 @@ public void testSingleValueAggDerivative_invalidPath() throws Exception { .get(); fail("Expected an Exception but didn't get one"); } catch (Exception e) { - Throwable cause = ExceptionsHelper.unwrapCause(e); + Throwable cause = BaseExceptionsHelper.unwrapCause(e); if (cause == null) { throw e; } else if (cause instanceof SearchPhaseExecutionException) { diff --git a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/pipeline/ExtendedStatsBucketIT.java b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/pipeline/ExtendedStatsBucketIT.java index 85fe794b05fc6..fc4a2908ae563 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/pipeline/ExtendedStatsBucketIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/pipeline/ExtendedStatsBucketIT.java @@ -32,7 +32,7 @@ package org.opensearch.search.aggregations.pipeline; -import org.opensearch.ExceptionsHelper; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.action.index.IndexRequestBuilder; import org.opensearch.action.search.SearchPhaseExecutionException; import org.opensearch.action.search.SearchResponse; @@ -489,7 +489,7 @@ public void testBadSigmaAsSubAgg() throws Exception { ) .get() ); - Throwable cause = ExceptionsHelper.unwrapCause(ex); + Throwable cause = BaseExceptionsHelper.unwrapCause(ex); if (cause == null) { throw ex; } else if (cause instanceof SearchPhaseExecutionException) { diff --git a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/pipeline/PercentilesBucketIT.java b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/pipeline/PercentilesBucketIT.java index 1da079781dc63..4d7fa05ba5043 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/aggregations/pipeline/PercentilesBucketIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/aggregations/pipeline/PercentilesBucketIT.java @@ -32,7 +32,7 @@ package org.opensearch.search.aggregations.pipeline; -import org.opensearch.ExceptionsHelper; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.action.index.IndexRequestBuilder; import org.opensearch.action.search.SearchPhaseExecutionException; import org.opensearch.action.search.SearchResponse; @@ -439,7 +439,7 @@ public void testBadPercents() throws Exception { fail("Illegal percent's were provided but no exception was thrown."); } catch (Exception e) { - Throwable cause = ExceptionsHelper.unwrapCause(e); + Throwable cause = BaseExceptionsHelper.unwrapCause(e); if (cause == null) { throw e; } else if (cause instanceof SearchPhaseExecutionException) { @@ -474,7 +474,7 @@ public void testBadPercents_asSubAgg() throws Exception { fail("Illegal percent's were provided but no exception was thrown."); } catch (Exception e) { - Throwable cause = ExceptionsHelper.unwrapCause(e); + Throwable cause = BaseExceptionsHelper.unwrapCause(e); if (cause == null) { throw e; } else if (cause instanceof SearchPhaseExecutionException) { diff --git a/server/src/internalClusterTest/java/org/opensearch/search/preference/SearchPreferenceIT.java b/server/src/internalClusterTest/java/org/opensearch/search/preference/SearchPreferenceIT.java index 1b68320bcbdf5..e519a74ad8eaf 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/preference/SearchPreferenceIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/preference/SearchPreferenceIT.java @@ -41,9 +41,9 @@ import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.routing.OperationRouting; import org.opensearch.cluster.routing.allocation.decider.EnableAllocationDecider; -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.node.Node; import org.opensearch.rest.RestStatus; import org.opensearch.test.OpenSearchIntegTestCase; diff --git a/server/src/internalClusterTest/java/org/opensearch/snapshots/ConcurrentSnapshotsIT.java b/server/src/internalClusterTest/java/org/opensearch/snapshots/ConcurrentSnapshotsIT.java index 433c168425371..7175d20bc240d 100644 --- a/server/src/internalClusterTest/java/org/opensearch/snapshots/ConcurrentSnapshotsIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/snapshots/ConcurrentSnapshotsIT.java @@ -45,11 +45,11 @@ import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.SnapshotDeletionsInProgress; import org.opensearch.cluster.SnapshotsInProgress; -import org.opensearch.common.Strings; import org.opensearch.common.UUIDs; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.UncategorizedExecutionException; +import org.opensearch.core.common.Strings; import org.opensearch.discovery.AbstractDisruptionTestCase; import org.opensearch.plugins.Plugin; import org.opensearch.repositories.RepositoryData; diff --git a/server/src/internalClusterTest/java/org/opensearch/snapshots/DedicatedClusterSnapshotRestoreIT.java b/server/src/internalClusterTest/java/org/opensearch/snapshots/DedicatedClusterSnapshotRestoreIT.java index 87b487dce3db9..a42b281b88bc8 100644 --- a/server/src/internalClusterTest/java/org/opensearch/snapshots/DedicatedClusterSnapshotRestoreIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/snapshots/DedicatedClusterSnapshotRestoreIT.java @@ -59,9 +59,9 @@ import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.CheckedFunction; +import org.opensearch.core.common.Strings; import org.opensearch.core.ParseField; import org.opensearch.common.Priority; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.NamedWriteableRegistry; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.Writeable; diff --git a/server/src/internalClusterTest/java/org/opensearch/snapshots/SharedClusterSnapshotRestoreIT.java b/server/src/internalClusterTest/java/org/opensearch/snapshots/SharedClusterSnapshotRestoreIT.java index 5c91c56a0c09d..27dd72ae3d022 100644 --- a/server/src/internalClusterTest/java/org/opensearch/snapshots/SharedClusterSnapshotRestoreIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/snapshots/SharedClusterSnapshotRestoreIT.java @@ -34,8 +34,8 @@ import org.apache.lucene.util.BytesRef; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.OpenSearchException; -import org.opensearch.ExceptionsHelper; import org.opensearch.Version; import org.opensearch.action.ActionFuture; import org.opensearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse; @@ -1986,7 +1986,7 @@ public void testSnapshotSucceedsAfterSnapshotFailure() throws Exception { } } catch (SnapshotException | RepositoryException ex) { // sometimes, the snapshot will fail with a top level I/O exception - assertThat(ExceptionsHelper.stackTrace(ex), containsString("Random IOException")); + assertThat(BaseExceptionsHelper.stackTrace(ex), containsString("Random IOException")); } logger.info("--> snapshot with no I/O failures"); diff --git a/server/src/internalClusterTest/java/org/opensearch/snapshots/SnapshotStatusApisIT.java b/server/src/internalClusterTest/java/org/opensearch/snapshots/SnapshotStatusApisIT.java index ebde9f2ab4d1d..72c64b56c19c1 100644 --- a/server/src/internalClusterTest/java/org/opensearch/snapshots/SnapshotStatusApisIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/snapshots/SnapshotStatusApisIT.java @@ -45,11 +45,11 @@ import org.opensearch.action.admin.cluster.snapshots.status.SnapshotsStatusResponse; import org.opensearch.client.Client; import org.opensearch.cluster.SnapshotsInProgress; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.ByteSizeUnit; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.io.IOUtils; +import org.opensearch.core.common.Strings; import org.opensearch.repositories.blobstore.BlobStoreRepository; import org.opensearch.threadpool.ThreadPool; diff --git a/server/src/main/java/org/opensearch/ExceptionsHelper.java b/server/src/main/java/org/opensearch/ExceptionsHelper.java index fff6a7e7684de..3d82070d351dc 100644 --- a/server/src/main/java/org/opensearch/ExceptionsHelper.java +++ b/server/src/main/java/org/opensearch/ExceptionsHelper.java @@ -33,8 +33,6 @@ package org.opensearch; import com.fasterxml.jackson.core.JsonParseException; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.IndexFormatTooNewException; import org.apache.lucene.index.IndexFormatTooOldException; @@ -47,8 +45,6 @@ import org.opensearch.rest.RestStatus; import java.io.IOException; -import java.io.PrintWriter; -import java.io.StringWriter; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -68,9 +64,7 @@ * * @opensearch.internal */ -public final class ExceptionsHelper { - - private static final Logger logger = LogManager.getLogger(ExceptionsHelper.class); +public final class ExceptionsHelper extends BaseExceptionsHelper { public static RuntimeException convertToRuntime(Exception e) { if (e instanceof RuntimeException) { @@ -101,77 +95,6 @@ public static RestStatus status(Throwable t) { return RestStatus.INTERNAL_SERVER_ERROR; } - public static String summaryMessage(Throwable t) { - if (t != null) { - if (t instanceof OpenSearchException) { - return t.getClass().getSimpleName() + "[" + t.getMessage() + "]"; - } else if (t instanceof IllegalArgumentException) { - return "Invalid argument"; - } else if (t instanceof JsonParseException) { - return "Failed to parse JSON"; - } else if (t instanceof OpenSearchRejectedExecutionException) { - return "Too many requests"; - } - } - return "Internal failure"; - } - - public static Throwable unwrapCause(Throwable t) { - int counter = 0; - Throwable result = t; - while (result instanceof OpenSearchWrapperException) { - if (result.getCause() == null) { - return result; - } - if (result.getCause() == result) { - return result; - } - if (counter++ > 10) { - // dear god, if we got more than 10 levels down, WTF? just bail - logger.warn("Exception cause unwrapping ran for 10 levels...", t); - return result; - } - result = result.getCause(); - } - return result; - } - - /** - * @deprecated Don't swallow exceptions, allow them to propagate. - */ - @Deprecated - public static String detailedMessage(Throwable t) { - if (t == null) { - return "Unknown"; - } - if (t.getCause() != null) { - StringBuilder sb = new StringBuilder(); - while (t != null) { - sb.append(t.getClass().getSimpleName()); - if (t.getMessage() != null) { - sb.append("["); - sb.append(t.getMessage()); - sb.append("]"); - } - sb.append("; "); - t = t.getCause(); - if (t != null) { - sb.append("nested: "); - } - } - return sb.toString(); - } else { - return t.getClass().getSimpleName() + "[" + t.getMessage() + "]"; - } - } - - public static String stackTrace(Throwable e) { - StringWriter stackTraceStringWriter = new StringWriter(); - PrintWriter printWriter = new PrintWriter(stackTraceStringWriter); - e.printStackTrace(printWriter); - return stackTraceStringWriter.toString(); - } - public static String formatStackTrace(final StackTraceElement[] stackTrace) { return Arrays.stream(stackTrace).skip(1).map(e -> "\tat " + e).collect(Collectors.joining("\n")); } diff --git a/server/src/main/java/org/opensearch/OpenSearchException.java b/server/src/main/java/org/opensearch/OpenSearchException.java index 54f6ffa1cc8fa..6c11ed0784ed9 100644 --- a/server/src/main/java/org/opensearch/OpenSearchException.java +++ b/server/src/main/java/org/opensearch/OpenSearchException.java @@ -39,16 +39,11 @@ import org.opensearch.cluster.routing.UnsupportedWeightedRoutingStateException; import org.opensearch.cluster.service.ClusterManagerThrottlingException; import org.opensearch.common.CheckedFunction; -import org.opensearch.common.Nullable; -import org.opensearch.core.ParseField; import org.opensearch.common.collect.Tuple; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.io.stream.Writeable; -import org.opensearch.common.logging.LoggerMessageFormat; -import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.XContentBuilder; -import org.opensearch.core.xcontent.XContentParseException; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.index.Index; import org.opensearch.index.shard.ShardId; @@ -70,8 +65,6 @@ import java.util.regex.Pattern; import java.util.stream.Collectors; -import static java.util.Collections.emptyMap; -import static java.util.Collections.singletonMap; import static java.util.Collections.unmodifiableMap; import static org.opensearch.Version.V_2_1_0; import static org.opensearch.Version.V_2_3_0; @@ -88,53 +81,19 @@ * * @opensearch.internal */ -public class OpenSearchException extends RuntimeException implements ToXContentFragment, Writeable { - - private static final Version UNKNOWN_VERSION_ADDED = Version.fromId(0); +public class OpenSearchException extends BaseOpenSearchException implements Writeable { /** * Setting a higher base exception id to avoid conflicts. */ private static final int CUSTOM_ELASTICSEARCH_EXCEPTIONS_BASE_ID = 10000; - /** - * Passed in the {@link Params} of {@link #generateThrowableXContent(XContentBuilder, Params, Throwable)} - * to control if the {@code caused_by} element should render. Unlike most parameters to {@code toXContent} methods this parameter is - * internal only and not available as a URL parameter. - */ - private static final String REST_EXCEPTION_SKIP_CAUSE = "rest.exception.cause.skip"; - /** - * Passed in the {@link Params} of {@link #generateThrowableXContent(XContentBuilder, Params, Throwable)} - * to control if the {@code stack_trace} element should render. Unlike most parameters to {@code toXContent} methods this parameter is - * internal only and not available as a URL parameter. Use the {@code error_trace} parameter instead. - */ - public static final String REST_EXCEPTION_SKIP_STACK_TRACE = "rest.exception.stacktrace.skip"; - public static final boolean REST_EXCEPTION_SKIP_STACK_TRACE_DEFAULT = true; - private static final boolean REST_EXCEPTION_SKIP_CAUSE_DEFAULT = false; - private static final String INDEX_METADATA_KEY = "opensearch.index"; - private static final String INDEX_METADATA_KEY_UUID = "opensearch.index_uuid"; - private static final String SHARD_METADATA_KEY = "opensearch.shard"; - private static final String RESOURCE_METADATA_TYPE_KEY = "opensearch.resource.type"; - private static final String RESOURCE_METADATA_ID_KEY = "opensearch.resource.id"; - - private static final String TYPE = "type"; - private static final String REASON = "reason"; - private static final String CAUSED_BY = "caused_by"; - private static final ParseField SUPPRESSED = new ParseField("suppressed"); - public static final String STACK_TRACE = "stack_trace"; - private static final String HEADER = "header"; - private static final String ERROR = "error"; - private static final String ROOT_CAUSE = "root_cause"; - private static final Map> ID_TO_SUPPLIER; private static final Map, OpenSearchExceptionHandle> CLASS_TO_OPENSEARCH_EXCEPTION_HANDLE; private static final Pattern OS_METADATA = Pattern.compile("^opensearch\\."); private static final Pattern ES_METADATA = Pattern.compile("^es\\."); - private final Map> metadata = new HashMap<>(); - private final Map> headers = new HashMap<>(); - /** * Construct a OpenSearchException with the specified cause exception. */ @@ -152,7 +111,7 @@ public OpenSearchException(Throwable cause) { * @param args the arguments for the message */ public OpenSearchException(String msg, Object... args) { - super(LoggerMessageFormat.format(msg, args)); + super(msg, args); } /** @@ -167,7 +126,7 @@ public OpenSearchException(String msg, Object... args) { * @param args the arguments for the message */ public OpenSearchException(String msg, Throwable cause, Object... args) { - super(LoggerMessageFormat.format(msg, args), cause); + super(msg, cause, args); } public OpenSearchException(StreamInput in) throws IOException { @@ -186,86 +145,6 @@ private static void replaceAndWrite(StreamOutput out, String str) throws IOExcep out.writeString(out.getVersion().onOrBefore(LegacyESVersion.V_7_10_2) ? OS_METADATA.matcher(str).replaceFirst("es.") : str); } - /** - * Adds a new piece of metadata with the given key. - * If the provided key is already present, the corresponding metadata will be replaced - */ - public void addMetadata(String key, String... values) { - addMetadata(key, Arrays.asList(values)); - } - - /** - * Adds a new piece of metadata with the given key. - * If the provided key is already present, the corresponding metadata will be replaced - */ - public void addMetadata(String key, List values) { - // we need to enforce this otherwise bw comp doesn't work properly, as "opensearch." - // was the previous criteria to split headers in two sets - if (key.startsWith("opensearch.") == false) { - throw new IllegalArgumentException("exception metadata must start with [opensearch.], found [" + key + "] instead"); - } - this.metadata.put(key, values); - } - - /** - * Returns a set of all metadata keys on this exception - */ - public Set getMetadataKeys() { - return metadata.keySet(); - } - - /** - * Returns the list of metadata values for the given key or {@code null} if no metadata for the - * given key exists. - */ - public List getMetadata(String key) { - return metadata.get(key); - } - - protected Map> getMetadata() { - return metadata; - } - - /** - * Adds a new header with the given key. - * This method will replace existing header if a header with the same key already exists - */ - public void addHeader(String key, List value) { - // we need to enforce this otherwise bw comp doesn't work properly, as "opensearch." - // was the previous criteria to split headers in two sets - if (key.startsWith("opensearch.")) { - throw new IllegalArgumentException("exception headers must not start with [opensearch.], found [" + key + "] instead"); - } - this.headers.put(key, value); - } - - /** - * Adds a new header with the given key. - * This method will replace existing header if a header with the same key already exists - */ - public void addHeader(String key, String... value) { - addHeader(key, Arrays.asList(value)); - } - - /** - * Returns a set of all header keys on this exception - */ - public Set getHeaderKeys() { - return headers.keySet(); - } - - /** - * Returns the list of header values for the given key or {@code null} if no header for the - * given key exists. - */ - public List getHeader(String key) { - return headers.get(key); - } - - protected Map> getHeaders() { - return headers; - } - /** * Returns the rest status code associated with this exception. */ @@ -278,48 +157,6 @@ public RestStatus status() { } } - /** - * Unwraps the actual cause from the exception for cases when the exception is a - * {@link OpenSearchWrapperException}. - * - * @see ExceptionsHelper#unwrapCause(Throwable) - */ - public Throwable unwrapCause() { - return ExceptionsHelper.unwrapCause(this); - } - - /** - * Return the detail message, including the message from the nested exception - * if there is one. - */ - public String getDetailedMessage() { - if (getCause() != null) { - StringBuilder sb = new StringBuilder(); - sb.append(toString()).append("; "); - if (getCause() instanceof OpenSearchException) { - sb.append(((OpenSearchException) getCause()).getDetailedMessage()); - } else { - sb.append(getCause()); - } - return sb.toString(); - } else { - return super.toString(); - } - } - - /** - * Retrieve the innermost cause of this exception, if none, returns the current exception. - */ - public Throwable getRootCause() { - Throwable rootCause = this; - Throwable cause = getCause(); - while (cause != null && cause != rootCause) { - rootCause = cause; - cause = cause.getCause(); - } - return rootCause; - } - @Override public void writeTo(StreamOutput out) throws IOException { out.writeOptionalString(this.getMessage()); @@ -363,91 +200,6 @@ public static int getId(Class exception) { return CLASS_TO_OPENSEARCH_EXCEPTION_HANDLE.get(exception).id; } - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - Throwable ex = ExceptionsHelper.unwrapCause(this); - if (ex != this) { - generateThrowableXContent(builder, params, this); - } else { - innerToXContent(builder, params, this, getExceptionName(), getMessage(), headers, metadata, getCause()); - } - return builder; - } - - protected static void innerToXContent( - XContentBuilder builder, - Params params, - Throwable throwable, - String type, - String message, - Map> headers, - Map> metadata, - Throwable cause - ) throws IOException { - builder.field(TYPE, type); - builder.field(REASON, message); - - for (Map.Entry> entry : metadata.entrySet()) { - headerToXContent(builder, entry.getKey().substring("opensearch.".length()), entry.getValue()); - } - - if (throwable instanceof OpenSearchException) { - OpenSearchException exception = (OpenSearchException) throwable; - exception.metadataToXContent(builder, params); - } - - if (params.paramAsBoolean(REST_EXCEPTION_SKIP_CAUSE, REST_EXCEPTION_SKIP_CAUSE_DEFAULT) == false) { - if (cause != null) { - builder.field(CAUSED_BY); - builder.startObject(); - generateThrowableXContent(builder, params, cause); - builder.endObject(); - } - } - - if (headers.isEmpty() == false) { - builder.startObject(HEADER); - for (Map.Entry> entry : headers.entrySet()) { - headerToXContent(builder, entry.getKey(), entry.getValue()); - } - builder.endObject(); - } - - if (params.paramAsBoolean(REST_EXCEPTION_SKIP_STACK_TRACE, REST_EXCEPTION_SKIP_STACK_TRACE_DEFAULT) == false) { - builder.field(STACK_TRACE, ExceptionsHelper.stackTrace(throwable)); - } - - Throwable[] allSuppressed = throwable.getSuppressed(); - if (allSuppressed.length > 0) { - builder.startArray(SUPPRESSED.getPreferredName()); - for (Throwable suppressed : allSuppressed) { - builder.startObject(); - generateThrowableXContent(builder, params, suppressed); - builder.endObject(); - } - builder.endArray(); - } - } - - private static void headerToXContent(XContentBuilder builder, String key, List values) throws IOException { - if (values != null && values.isEmpty() == false) { - if (values.size() == 1) { - builder.field(key, values.get(0)); - } else { - builder.startArray(key); - for (String value : values) { - builder.value(value); - } - builder.endArray(); - } - } - } - - /** - * Renders additional per exception information into the XContent - */ - protected void metadataToXContent(XContentBuilder builder, Params params) throws IOException {} - /** * Generate a {@link OpenSearchException} from a {@link XContentParser}. This does not * return the original exception type (ie NodeClosedException for example) but just wraps @@ -477,19 +229,19 @@ public static OpenSearchException innerFromXContent(XContentParser parser, boole token = parser.nextToken(); if (token.isValue()) { - if (TYPE.equals(currentFieldName)) { + if (BaseExceptionsHelper.TYPE.equals(currentFieldName)) { type = parser.text(); - } else if (REASON.equals(currentFieldName)) { + } else if (BaseExceptionsHelper.REASON.equals(currentFieldName)) { reason = parser.text(); - } else if (STACK_TRACE.equals(currentFieldName)) { + } else if (BaseExceptionsHelper.STACK_TRACE.equals(currentFieldName)) { stack = parser.text(); } else if (token == XContentParser.Token.VALUE_STRING) { metadata.put(currentFieldName, Collections.singletonList(parser.text())); } } else if (token == XContentParser.Token.START_OBJECT) { - if (CAUSED_BY.equals(currentFieldName)) { + if (BaseExceptionsHelper.CAUSED_BY.equals(currentFieldName)) { cause = fromXContent(parser); - } else if (HEADER.equals(currentFieldName)) { + } else if (BaseExceptionsHelper.HEADER.equals(currentFieldName)) { while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { if (token == XContentParser.Token.FIELD_NAME) { currentFieldName = parser.currentName(); @@ -522,7 +274,7 @@ public static OpenSearchException innerFromXContent(XContentParser parser, boole while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { rootCauses.add(fromXContent(parser)); } - } else if (SUPPRESSED.match(currentFieldName, parser.getDeprecationHandler())) { + } else if (BaseExceptionsHelper.SUPPRESSED.match(currentFieldName, parser.getDeprecationHandler())) { while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { suppressed.add(fromXContent(parser)); } @@ -555,7 +307,7 @@ public static OpenSearchException innerFromXContent(XContentParser parser, boole // by addMetadata. The prefix will get stripped out when printing metadata out so it will be effectively invisible. // TODO move subclasses that print out simple metadata to using addMetadata directly and support also numbers and booleans. // TODO rename metadataToXContent and have only SearchPhaseExecutionException use it, which prints out complex objects - e.addMetadata("opensearch." + entry.getKey(), entry.getValue()); + e.addMetadata(BaseExceptionsHelper.OPENSEARCH_PREFIX_KEY + entry.getKey(), entry.getValue()); } for (Map.Entry> header : headers.entrySet()) { e.addHeader(header.getKey(), header.getValue()); @@ -572,70 +324,6 @@ public static OpenSearchException innerFromXContent(XContentParser parser, boole return e; } - /** - * Static toXContent helper method that renders {@link OpenSearchException} or {@link Throwable} instances - * as XContent, delegating the rendering to {@link #toXContent(XContentBuilder, Params)} - * or {@link #innerToXContent(XContentBuilder, Params, Throwable, String, String, Map, Map, Throwable)}. - * - * This method is usually used when the {@link Throwable} is rendered as a part of another XContent object, and its result can - * be parsed back using the {@link #fromXContent(XContentParser)} method. - */ - public static void generateThrowableXContent(XContentBuilder builder, Params params, Throwable t) throws IOException { - t = ExceptionsHelper.unwrapCause(t); - - if (t instanceof OpenSearchException) { - ((OpenSearchException) t).toXContent(builder, params); - } else { - innerToXContent(builder, params, t, getExceptionName(t), t.getMessage(), emptyMap(), emptyMap(), t.getCause()); - } - } - - /** - * Render any exception as a xcontent, encapsulated within a field or object named "error". The level of details that are rendered - * depends on the value of the "detailed" parameter: when it's false only a simple message based on the type and message of the - * exception is rendered. When it's true all detail are provided including guesses root causes, cause and potentially stack - * trace. - * - * This method is usually used when the {@link Exception} is rendered as a full XContent object, and its output can be parsed - * by the {@link #failureFromXContent(XContentParser)} method. - */ - public static void generateFailureXContent(XContentBuilder builder, Params params, @Nullable Exception e, boolean detailed) - throws IOException { - // No exception to render as an error - if (e == null) { - builder.field(ERROR, "unknown"); - return; - } - - // Render the exception with a simple message - if (detailed == false) { - Throwable t = e; - for (int counter = 0; counter < 10 && t != null; counter++) { - if (t instanceof OpenSearchException) { - break; - } - t = t.getCause(); - } - builder.field(ERROR, ExceptionsHelper.summaryMessage(t != null ? t : e)); - return; - } - - // Render the exception with all details - final OpenSearchException[] rootCauses = OpenSearchException.guessRootCauses(e); - builder.startObject(ERROR); - { - builder.startArray(ROOT_CAUSE); - for (OpenSearchException rootCause : rootCauses) { - builder.startObject(); - rootCause.toXContent(builder, new DelegatingMapParams(singletonMap(REST_EXCEPTION_SKIP_CAUSE, "true"), params)); - builder.endObject(); - } - builder.endArray(); - } - generateThrowableXContent(builder, params, e); - builder.endObject(); - } - /** * Parses the output of {@link #generateFailureXContent(XContentBuilder, Params, Exception, boolean)} */ @@ -655,79 +343,6 @@ public static OpenSearchException failureFromXContent(XContentParser parser) thr return innerFromXContent(parser, true); } - /** - * Returns the root cause of this exception or multiple if different shards caused different exceptions - */ - public OpenSearchException[] guessRootCauses() { - final Throwable cause = getCause(); - if (cause != null && cause instanceof OpenSearchException) { - return ((OpenSearchException) cause).guessRootCauses(); - } - return new OpenSearchException[] { this }; - } - - /** - * Returns the root cause of this exception or multiple if different shards caused different exceptions. - * If the given exception is not an instance of {@link OpenSearchException} an empty array - * is returned. - */ - public static OpenSearchException[] guessRootCauses(Throwable t) { - Throwable ex = ExceptionsHelper.unwrapCause(t); - if (ex instanceof OpenSearchException) { - // OpenSearchException knows how to guess its own root cause - return ((OpenSearchException) ex).guessRootCauses(); - } - if (ex instanceof XContentParseException) { - /* - * We'd like to unwrap parsing exceptions to the inner-most - * parsing exception because that is generally the most interesting - * exception to return to the user. If that exception is caused by - * an OpenSearchException we'd like to keep unwrapping because - * OpenSearchException instances tend to contain useful information - * for the user. - */ - Throwable cause = ex.getCause(); - if (cause != null) { - if (cause instanceof XContentParseException || cause instanceof OpenSearchException) { - return guessRootCauses(ex.getCause()); - } - } - } - return new OpenSearchException[] { new OpenSearchException(ex.getMessage(), ex) { - @Override - protected String getExceptionName() { - return getExceptionName(getCause()); - } - } }; - } - - protected String getExceptionName() { - return getExceptionName(this); - } - - /** - * Returns an underscore case name for the given exception. This method strips {@code OpenSearch} prefixes from exception names. - */ - public static String getExceptionName(Throwable ex) { - String simpleName = ex.getClass().getSimpleName(); - if (simpleName.startsWith("OpenSearch")) { - simpleName = simpleName.substring("OpenSearch".length()); - } - // TODO: do we really need to make the exception name in underscore casing? - return toUnderscoreCase(simpleName); - } - - static String buildMessage(String type, String reason, String stack) { - StringBuilder message = new StringBuilder("OpenSearch exception ["); - message.append(TYPE).append('=').append(type).append(", "); - message.append(REASON).append('=').append(reason); - if (stack != null) { - message.append(", ").append(STACK_TRACE).append('=').append(stack); - } - message.append(']'); - return message.toString(); - } - @Override public String toString() { StringBuilder builder = new StringBuilder(); @@ -738,7 +353,7 @@ public String toString() { } builder.append(' '); } - return builder.append(ExceptionsHelper.detailedMessage(this).trim()).toString(); + return builder.append(BaseExceptionsHelper.detailedMessage(this).trim()).toString(); } /** @@ -1757,58 +1372,4 @@ public void setShard(ShardId shardId) { } } - public void setResources(String type, String... id) { - assert type != null; - addMetadata(RESOURCE_METADATA_ID_KEY, id); - addMetadata(RESOURCE_METADATA_TYPE_KEY, type); - } - - public List getResourceId() { - return getMetadata(RESOURCE_METADATA_ID_KEY); - } - - public String getResourceType() { - List header = getMetadata(RESOURCE_METADATA_TYPE_KEY); - if (header != null && header.isEmpty() == false) { - assert header.size() == 1; - return header.get(0); - } - return null; - } - - // lower cases and adds underscores to transitions in a name - private static String toUnderscoreCase(String value) { - StringBuilder sb = new StringBuilder(); - boolean changed = false; - for (int i = 0; i < value.length(); i++) { - char c = value.charAt(i); - if (Character.isUpperCase(c)) { - if (!changed) { - // copy it over here - for (int j = 0; j < i; j++) { - sb.append(value.charAt(j)); - } - changed = true; - if (i == 0) { - sb.append(Character.toLowerCase(c)); - } else { - sb.append('_'); - sb.append(Character.toLowerCase(c)); - } - } else { - sb.append('_'); - sb.append(Character.toLowerCase(c)); - } - } else { - if (changed) { - sb.append(c); - } - } - } - if (!changed) { - return value; - } - return sb.toString(); - } - } diff --git a/server/src/main/java/org/opensearch/action/TaskOperationFailure.java b/server/src/main/java/org/opensearch/action/TaskOperationFailure.java index f938abba9f9b2..e1cf77fb3d530 100644 --- a/server/src/main/java/org/opensearch/action/TaskOperationFailure.java +++ b/server/src/main/java/org/opensearch/action/TaskOperationFailure.java @@ -32,6 +32,7 @@ package org.opensearch.action; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.OpenSearchException; import org.opensearch.ExceptionsHelper; import org.opensearch.core.ParseField; @@ -144,7 +145,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (reason != null) { builder.field(REASON); builder.startObject(); - OpenSearchException.generateThrowableXContent(builder, params, reason); + BaseExceptionsHelper.generateThrowableXContent(builder, params, reason); builder.endObject(); } return builder; diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/configuration/AddVotingConfigExclusionsRequest.java b/server/src/main/java/org/opensearch/action/admin/cluster/configuration/AddVotingConfigExclusionsRequest.java index a2a77a1316898..7b85bf4add71c 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/configuration/AddVotingConfigExclusionsRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/configuration/AddVotingConfigExclusionsRequest.java @@ -38,11 +38,11 @@ import org.opensearch.cluster.coordination.CoordinationMetadata.VotingConfigExclusion; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodes; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import java.io.IOException; import java.util.Arrays; diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/decommission/awareness/get/GetDecommissionStateRequest.java b/server/src/main/java/org/opensearch/action/admin/cluster/decommission/awareness/get/GetDecommissionStateRequest.java index 1f301aa2b5273..773c12c525817 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/decommission/awareness/get/GetDecommissionStateRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/decommission/awareness/get/GetDecommissionStateRequest.java @@ -10,9 +10,9 @@ import org.opensearch.action.ActionRequestValidationException; import org.opensearch.action.support.clustermanager.ClusterManagerNodeReadRequest; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/decommission/awareness/put/DecommissionRequest.java b/server/src/main/java/org/opensearch/action/admin/cluster/decommission/awareness/put/DecommissionRequest.java index 44f047137eece..df364b05dd494 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/decommission/awareness/put/DecommissionRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/decommission/awareness/put/DecommissionRequest.java @@ -11,10 +11,10 @@ import org.opensearch.action.ActionRequestValidationException; import org.opensearch.action.support.clustermanager.ClusterManagerNodeRequest; import org.opensearch.cluster.decommission.DecommissionAttribute; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import java.io.IOException; 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 d475826b059d8..606b74b9b85df 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 @@ -57,11 +57,11 @@ import org.opensearch.cluster.routing.WeightedRoutingUtils; import org.opensearch.cluster.routing.allocation.AllocationService; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.Strings; import org.opensearch.common.inject.Inject; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.Strings; import org.opensearch.discovery.ClusterManagerNotDiscoveredException; import org.opensearch.discovery.Discovery; import org.opensearch.index.IndexNotFoundException; 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 f6d9a5c7b6453..874f6c9f13ac7 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 @@ -32,7 +32,7 @@ package org.opensearch.action.admin.cluster.node.reload; -import org.opensearch.OpenSearchException; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.action.FailedNodeException; import org.opensearch.action.support.nodes.BaseNodeResponse; import org.opensearch.action.support.nodes.BaseNodesResponse; @@ -83,7 +83,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws final Exception e = node.reloadException(); if (e != null) { builder.startObject("reload_exception"); - OpenSearchException.generateThrowableXContent(builder, params, e); + BaseExceptionsHelper.generateThrowableXContent(builder, params, e); builder.endObject(); } builder.endObject(); 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 d84943899b3c5..b6c739a75a357 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 @@ -11,10 +11,10 @@ import org.opensearch.action.ActionRequestValidationException; import org.opensearch.action.support.IndicesOptions; import org.opensearch.action.support.clustermanager.ClusterManagerNodeRequest; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.xcontent.XContentType; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ToXContentObject; import org.opensearch.core.xcontent.XContentBuilder; @@ -179,6 +179,6 @@ public int hashCode() { @Override public String toString() { - return Strings.toString(XContentType.JSON, this); + return org.opensearch.common.Strings.toString(XContentType.JSON, this); } } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/repositories/get/GetRepositoriesRequest.java b/server/src/main/java/org/opensearch/action/admin/cluster/repositories/get/GetRepositoriesRequest.java index d042e60638c47..ca70dcce9d0c0 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/repositories/get/GetRepositoriesRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/repositories/get/GetRepositoriesRequest.java @@ -34,9 +34,9 @@ import org.opensearch.action.ActionRequestValidationException; import org.opensearch.action.support.clustermanager.ClusterManagerNodeReadRequest; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/reroute/TransportClusterRerouteAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/reroute/TransportClusterRerouteAction.java index 218dfcbc816d9..94c4269e77109 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/reroute/TransportClusterRerouteAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/reroute/TransportClusterRerouteAction.java @@ -58,10 +58,10 @@ import org.opensearch.cluster.service.ClusterManagerTaskThrottler; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.Priority; -import org.opensearch.common.Strings; import org.opensearch.common.collect.ImmutableOpenIntMap; import org.opensearch.common.inject.Inject; import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.core.common.Strings; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportService; diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/shards/ClusterSearchShardsRequest.java b/server/src/main/java/org/opensearch/action/admin/cluster/shards/ClusterSearchShardsRequest.java index fb3da57db7291..fd758fe8f4d91 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/shards/ClusterSearchShardsRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/shards/ClusterSearchShardsRequest.java @@ -37,9 +37,9 @@ import org.opensearch.action.support.IndicesOptions; import org.opensearch.action.support.clustermanager.ClusterManagerNodeReadRequest; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import java.io.IOException; import java.util.Objects; diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/clone/CloneSnapshotRequestBuilder.java b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/clone/CloneSnapshotRequestBuilder.java index a9472937040e9..6de90bb4e7a06 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/clone/CloneSnapshotRequestBuilder.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/clone/CloneSnapshotRequestBuilder.java @@ -37,7 +37,7 @@ import org.opensearch.action.support.master.AcknowledgedResponse; import org.opensearch.action.support.clustermanager.ClusterManagerNodeOperationRequestBuilder; import org.opensearch.client.OpenSearchClient; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; /** * Transport request builder for cloning a snapshot 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 28192c2069181..4ee0584f53a60 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 @@ -56,11 +56,11 @@ import java.util.Objects; import static org.opensearch.action.ValidateActions.addValidationError; -import static org.opensearch.common.Strings.EMPTY_ARRAY; import static org.opensearch.common.settings.Settings.Builder.EMPTY_SETTINGS; import static org.opensearch.common.settings.Settings.readSettingsFromStream; import static org.opensearch.common.settings.Settings.writeSettingsToStream; import static org.opensearch.common.xcontent.support.XContentMapValues.nodeBooleanValue; +import static org.opensearch.core.common.Strings.EMPTY_ARRAY; import static org.opensearch.snapshots.SnapshotInfo.METADATA_FIELD_INTRODUCED; /** @@ -451,7 +451,7 @@ public CreateSnapshotRequest source(Map source) { String name = entry.getKey(); if (name.equals("indices")) { if (entry.getValue() instanceof String) { - indices(Strings.splitStringByCommaToArray((String) entry.getValue())); + indices(org.opensearch.core.common.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/GetSnapshotsRequest.java b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/get/GetSnapshotsRequest.java index bb50cbf4316a9..5aa695f9bc1dd 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/get/GetSnapshotsRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/get/GetSnapshotsRequest.java @@ -34,9 +34,9 @@ import org.opensearch.action.ActionRequestValidationException; import org.opensearch.action.support.clustermanager.ClusterManagerNodeRequest; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import java.io.IOException; 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 3445c62366cf9..08ca5818c2c98 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 @@ -38,11 +38,11 @@ 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.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ToXContentObject; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.common.xcontent.XContentType; @@ -737,6 +737,6 @@ public int hashCode() { @Override public String toString() { - return Strings.toString(XContentType.JSON, this); + return org.opensearch.common.Strings.toString(XContentType.JSON, this); } } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/SnapshotsStatusRequest.java b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/SnapshotsStatusRequest.java index 7afa7a25c7c0e..5573f36e12a54 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/SnapshotsStatusRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/SnapshotsStatusRequest.java @@ -34,9 +34,9 @@ import org.opensearch.action.ActionRequestValidationException; import org.opensearch.action.support.clustermanager.ClusterManagerNodeRequest; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import java.io.IOException; 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 0400c3afbaf99..fb1c97fc884a0 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 @@ -46,11 +46,11 @@ import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.IndexNameExpressionResolver; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.Strings; import org.opensearch.common.inject.Inject; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.util.CollectionUtils; import org.opensearch.common.util.set.Sets; +import org.opensearch.core.common.Strings; import org.opensearch.index.shard.ShardId; import org.opensearch.index.snapshots.IndexShardSnapshotStatus; import org.opensearch.repositories.IndexId; diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/state/ClusterStateRequest.java b/server/src/main/java/org/opensearch/action/admin/cluster/state/ClusterStateRequest.java index 1c6afaf08a89c..5454cfb72e9c1 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/state/ClusterStateRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/state/ClusterStateRequest.java @@ -36,10 +36,10 @@ import org.opensearch.action.IndicesRequest; import org.opensearch.action.support.IndicesOptions; import org.opensearch.action.support.clustermanager.ClusterManagerNodeReadRequest; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/stats/ClusterStatsNodes.java b/server/src/main/java/org/opensearch/action/admin/cluster/stats/ClusterStatsNodes.java index bbeda453578aa..25a915833c7e2 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/stats/ClusterStatsNodes.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/stats/ClusterStatsNodes.java @@ -40,13 +40,13 @@ import org.opensearch.action.admin.cluster.node.stats.NodeStats; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodeRole; -import org.opensearch.common.Strings; import org.opensearch.common.metrics.OperationStats; import org.opensearch.common.network.NetworkModule; import org.opensearch.common.settings.Settings; import org.opensearch.common.transport.TransportAddress; import org.opensearch.common.unit.ByteSizeValue; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.discovery.DiscoveryModule; diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/storedscripts/PutStoredScriptRequest.java b/server/src/main/java/org/opensearch/action/admin/cluster/storedscripts/PutStoredScriptRequest.java index 466941038bd33..583f208287e68 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/storedscripts/PutStoredScriptRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/storedscripts/PutStoredScriptRequest.java @@ -37,6 +37,7 @@ import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.common.xcontent.XContentHelper; @@ -152,7 +153,7 @@ public void writeTo(StreamOutput out) throws IOException { @Override public String toString() { - String source = "_na_"; + String source = Strings.UNKNOWN_UUID_VALUE; try { source = XContentHelper.convertToJson(content, false, xContentType); 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 b7ba7085cb19d..665e5e4688a63 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 @@ -41,20 +41,20 @@ import org.opensearch.cluster.metadata.AliasAction; import org.opensearch.core.ParseField; import org.opensearch.common.ParsingException; -import org.opensearch.common.Strings; import org.opensearch.common.bytes.BytesArray; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.io.stream.Writeable; +import org.opensearch.common.xcontent.XContentFactory; +import org.opensearch.common.xcontent.XContentType; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ConstructingObjectParser; import org.opensearch.core.xcontent.ObjectParser; import org.opensearch.core.xcontent.ObjectParser.ValueType; import org.opensearch.core.xcontent.ToXContent; 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.common.xcontent.XContentType; import org.opensearch.index.query.QueryBuilder; import java.io.IOException; @@ -451,7 +451,7 @@ public AliasActions filter(Map filter) { try { XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON); builder.map(filter); - this.filter = Strings.toString(builder); + this.filter = org.opensearch.common.Strings.toString(builder); return this; } catch (IOException e) { throw new OpenSearchGenerationException("Failed to generate [" + filter + "]", e); @@ -467,7 +467,7 @@ public AliasActions filter(QueryBuilder filter) { XContentBuilder builder = XContentFactory.jsonBuilder(); filter.toXContent(builder, ToXContent.EMPTY_PARAMS); builder.close(); - this.filter = Strings.toString(builder); + this.filter = org.opensearch.common.Strings.toString(builder); 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/alias/get/GetAliasesRequest.java b/server/src/main/java/org/opensearch/action/admin/indices/alias/get/GetAliasesRequest.java index 81dd9a6045cf5..c7a7e16ef112c 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/alias/get/GetAliasesRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/alias/get/GetAliasesRequest.java @@ -35,9 +35,9 @@ import org.opensearch.action.AliasesRequest; import org.opensearch.action.support.IndicesOptions; import org.opensearch.action.support.clustermanager.ClusterManagerNodeReadRequest; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/cache/clear/ClearIndicesCacheRequest.java b/server/src/main/java/org/opensearch/action/admin/indices/cache/clear/ClearIndicesCacheRequest.java index 1be3bbe0a791c..56f1e59749f66 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/cache/clear/ClearIndicesCacheRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/cache/clear/ClearIndicesCacheRequest.java @@ -34,9 +34,9 @@ import org.opensearch.Version; import org.opensearch.action.support.broadcast.BroadcastRequest; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; 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 09d1780595165..4cafd076d5d48 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 @@ -32,7 +32,7 @@ package org.opensearch.action.admin.indices.close; import org.opensearch.LegacyESVersion; -import org.opensearch.OpenSearchException; +import org.opensearch.BaseOpenSearchException; import org.opensearch.action.support.DefaultShardOperationFailedException; import org.opensearch.action.support.master.ShardsAcknowledgedResponse; import org.opensearch.common.Nullable; @@ -182,7 +182,7 @@ public XContentBuilder toXContent(final XContentBuilder builder, final Params pa builder.field("closed", false); if (exception != null) { builder.startObject("exception"); - OpenSearchException.generateFailureXContent(builder, params, exception, true); + BaseOpenSearchException.generateFailureXContent(builder, params, exception, true); builder.endObject(); } else { builder.startObject("failedShards"); diff --git a/server/src/main/java/org/opensearch/action/admin/indices/dangling/find/FindDanglingIndexRequest.java b/server/src/main/java/org/opensearch/action/admin/indices/dangling/find/FindDanglingIndexRequest.java index 5c1e3c0a23eae..f9c6d8d2f1d50 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/dangling/find/FindDanglingIndexRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/dangling/find/FindDanglingIndexRequest.java @@ -33,9 +33,9 @@ package org.opensearch.action.admin.indices.dangling.find; import org.opensearch.action.support.nodes.BaseNodesRequest; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/dangling/list/ListDanglingIndicesRequest.java b/server/src/main/java/org/opensearch/action/admin/indices/dangling/list/ListDanglingIndicesRequest.java index ba5fd8fc139ec..a361d321bfd46 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/dangling/list/ListDanglingIndicesRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/dangling/list/ListDanglingIndicesRequest.java @@ -33,9 +33,9 @@ package org.opensearch.action.admin.indices.dangling.list; import org.opensearch.action.support.nodes.BaseNodesRequest; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/datastream/CreateDataStreamAction.java b/server/src/main/java/org/opensearch/action/admin/indices/datastream/CreateDataStreamAction.java index ddc93dd1fcf6c..9f1069e60c625 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/datastream/CreateDataStreamAction.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/datastream/CreateDataStreamAction.java @@ -48,10 +48,10 @@ import org.opensearch.cluster.metadata.MetadataCreateDataStreamService; import org.opensearch.cluster.metadata.MetadataCreateDataStreamService.CreateDataStreamClusterStateUpdateRequest; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.Strings; import org.opensearch.common.inject.Inject; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportService; 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 9260904025df2..ca032e66a34f7 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 @@ -54,13 +54,13 @@ import org.opensearch.cluster.service.ClusterManagerTaskThrottler; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.Priority; -import org.opensearch.common.Strings; import org.opensearch.common.inject.Inject; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.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.Strings; import org.opensearch.index.Index; import org.opensearch.snapshots.SnapshotInProgressException; import org.opensearch.snapshots.SnapshotsService; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/exists/indices/IndicesExistsRequest.java b/server/src/main/java/org/opensearch/action/admin/indices/exists/indices/IndicesExistsRequest.java index 0b83478933cd4..294f395437931 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/exists/indices/IndicesExistsRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/exists/indices/IndicesExistsRequest.java @@ -36,9 +36,9 @@ import org.opensearch.action.IndicesRequest; import org.opensearch.action.support.IndicesOptions; import org.opensearch.action.support.clustermanager.ClusterManagerNodeReadRequest; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/mapping/get/GetFieldMappingsIndexRequest.java b/server/src/main/java/org/opensearch/action/admin/indices/mapping/get/GetFieldMappingsIndexRequest.java index 176eca5e35cec..1ce2c2473c455 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/mapping/get/GetFieldMappingsIndexRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/mapping/get/GetFieldMappingsIndexRequest.java @@ -37,9 +37,9 @@ import org.opensearch.action.OriginalIndices; import org.opensearch.action.support.IndicesOptions; import org.opensearch.action.support.single.shard.SingleShardRequest; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/mapping/get/GetFieldMappingsRequest.java b/server/src/main/java/org/opensearch/action/admin/indices/mapping/get/GetFieldMappingsRequest.java index db00f3d5244ac..55a36a95dfd2d 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/mapping/get/GetFieldMappingsRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/mapping/get/GetFieldMappingsRequest.java @@ -37,9 +37,9 @@ import org.opensearch.action.ActionRequestValidationException; import org.opensearch.action.IndicesRequest; import org.opensearch.action.support.IndicesOptions; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import java.io.IOException; import java.util.Arrays; 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 623ef24e4e87f..32e91d8a5581e 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 @@ -41,7 +41,6 @@ import org.opensearch.action.support.IndicesOptions; import org.opensearch.action.support.master.AcknowledgedRequest; import org.opensearch.action.support.master.AcknowledgedResponse; -import org.opensearch.common.Strings; import org.opensearch.common.bytes.BytesArray; import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.io.stream.StreamInput; @@ -50,6 +49,7 @@ import org.opensearch.common.xcontent.XContentFactory; import org.opensearch.common.xcontent.XContentHelper; import org.opensearch.common.xcontent.XContentType; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.MediaType; import org.opensearch.core.xcontent.ToXContentObject; import org.opensearch.core.xcontent.XContentBuilder; @@ -256,7 +256,7 @@ public static XContentBuilder simpleMapping(String... source) { builder.startObject(fieldName); String[] s1 = Strings.splitStringByCommaToArray(source[i]); for (String s : s1) { - String[] s2 = Strings.split(s, "="); + String[] s2 = org.opensearch.common.Strings.split(s, "="); if (s2.length != 2) { throw new IllegalArgumentException("malformed " + s); } @@ -276,7 +276,7 @@ public static XContentBuilder simpleMapping(String... source) { builder.startObject(fieldName); String[] s1 = Strings.splitStringByCommaToArray(source[i]); for (String s : s1) { - String[] s2 = Strings.split(s, "="); + String[] s2 = org.opensearch.common.Strings.split(s, "="); if (s2.length != 2) { throw new IllegalArgumentException("malformed " + s); } 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 6d89170fefd7c..6d31dfb98b7c8 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 @@ -31,7 +31,7 @@ package org.opensearch.action.admin.indices.readonly; -import org.opensearch.OpenSearchException; +import org.opensearch.BaseOpenSearchException; import org.opensearch.action.support.DefaultShardOperationFailedException; import org.opensearch.action.support.master.ShardsAcknowledgedResponse; import org.opensearch.common.Nullable; @@ -172,7 +172,7 @@ public XContentBuilder toXContent(final XContentBuilder builder, final Params pa if (hasFailures()) { if (exception != null) { builder.startObject("exception"); - OpenSearchException.generateFailureXContent(builder, params, exception, true); + BaseOpenSearchException.generateFailureXContent(builder, params, exception, true); builder.endObject(); } else { builder.startArray("failed_shards"); diff --git a/server/src/main/java/org/opensearch/action/admin/indices/recovery/RecoveryRequest.java b/server/src/main/java/org/opensearch/action/admin/indices/recovery/RecoveryRequest.java index 8b1fe1e88da22..d89c86a8f8e79 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/recovery/RecoveryRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/recovery/RecoveryRequest.java @@ -34,9 +34,9 @@ import org.opensearch.action.support.IndicesOptions; import org.opensearch.action.support.broadcast.BroadcastRequest; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/replication/SegmentReplicationStatsRequest.java b/server/src/main/java/org/opensearch/action/admin/indices/replication/SegmentReplicationStatsRequest.java index 1142bf697db27..6df990aecd3e3 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/replication/SegmentReplicationStatsRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/replication/SegmentReplicationStatsRequest.java @@ -10,9 +10,9 @@ import org.opensearch.action.support.IndicesOptions; import org.opensearch.action.support.broadcast.BroadcastRequest; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/resolve/ResolveIndexAction.java b/server/src/main/java/org/opensearch/action/admin/indices/resolve/ResolveIndexAction.java index e2de74f34222b..413c97d641de1 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/resolve/ResolveIndexAction.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/resolve/ResolveIndexAction.java @@ -51,13 +51,13 @@ import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.Nullable; -import org.opensearch.core.ParseField; -import org.opensearch.common.Strings; import org.opensearch.common.inject.Inject; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.io.stream.Writeable; import org.opensearch.common.util.concurrent.CountDown; +import org.opensearch.core.common.Strings; +import org.opensearch.core.ParseField; import org.opensearch.core.xcontent.ToXContentObject; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.tasks.Task; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/rollover/MetadataRolloverService.java b/server/src/main/java/org/opensearch/action/admin/indices/rollover/MetadataRolloverService.java index 266da30155583..c6193c076ee50 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/rollover/MetadataRolloverService.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/rollover/MetadataRolloverService.java @@ -48,9 +48,9 @@ import org.opensearch.cluster.metadata.MetadataIndexAliasesService; import org.opensearch.cluster.metadata.MetadataIndexTemplateService; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.inject.Inject; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; import org.opensearch.threadpool.ThreadPool; import java.util.Arrays; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/segments/IndicesSegmentsRequest.java b/server/src/main/java/org/opensearch/action/admin/indices/segments/IndicesSegmentsRequest.java index dd8c02a97c0f7..948b95b5a77be 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/segments/IndicesSegmentsRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/segments/IndicesSegmentsRequest.java @@ -33,9 +33,9 @@ package org.opensearch.action.admin.indices.segments; import org.opensearch.action.support.broadcast.BroadcastRequest; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/segments/PitSegmentsRequest.java b/server/src/main/java/org/opensearch/action/admin/indices/segments/PitSegmentsRequest.java index 94815e6001adb..cee1b223c5611 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/segments/PitSegmentsRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/segments/PitSegmentsRequest.java @@ -10,9 +10,9 @@ import org.opensearch.action.ActionRequestValidationException; import org.opensearch.action.support.broadcast.BroadcastRequest; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentParser; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/segments/TransportPitSegmentsAction.java b/server/src/main/java/org/opensearch/action/admin/indices/segments/TransportPitSegmentsAction.java index ca379234dcee7..3886dd94ce543 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/segments/TransportPitSegmentsAction.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/segments/TransportPitSegmentsAction.java @@ -27,11 +27,11 @@ import org.opensearch.cluster.routing.ShardsIterator; import org.opensearch.cluster.routing.UnassignedInfo; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.Strings; import org.opensearch.common.inject.Inject; import org.opensearch.common.io.stream.NamedWriteableRegistry; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.index.shard.ShardId; import org.opensearch.indices.IndicesService; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/settings/get/GetSettingsRequest.java b/server/src/main/java/org/opensearch/action/admin/indices/settings/get/GetSettingsRequest.java index b917e6734e932..37587bd5e4c25 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/settings/get/GetSettingsRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/settings/get/GetSettingsRequest.java @@ -37,9 +37,9 @@ import org.opensearch.action.ValidateActions; import org.opensearch.action.support.IndicesOptions; import org.opensearch.action.support.clustermanager.ClusterManagerNodeReadRequest; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import java.io.IOException; import java.util.Arrays; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/shards/IndicesShardStoresRequest.java b/server/src/main/java/org/opensearch/action/admin/indices/shards/IndicesShardStoresRequest.java index e73addbc56ce0..a910e849fe87a 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/shards/IndicesShardStoresRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/shards/IndicesShardStoresRequest.java @@ -36,9 +36,9 @@ import org.opensearch.action.support.IndicesOptions; import org.opensearch.action.support.clustermanager.ClusterManagerNodeReadRequest; import org.opensearch.cluster.health.ClusterHealthStatus; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import java.io.IOException; import java.util.EnumSet; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/shards/IndicesShardStoresResponse.java b/server/src/main/java/org/opensearch/action/admin/indices/shards/IndicesShardStoresResponse.java index 8da11b34c20de..a03c6e9045a6b 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/shards/IndicesShardStoresResponse.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/shards/IndicesShardStoresResponse.java @@ -33,8 +33,8 @@ package org.opensearch.action.admin.indices.shards; import com.carrotsearch.hppc.cursors.IntObjectCursor; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.LegacyESVersion; -import org.opensearch.OpenSearchException; import org.opensearch.action.ActionResponse; import org.opensearch.action.support.DefaultShardOperationFailedException; import org.opensearch.cluster.node.DiscoveryNode; @@ -203,7 +203,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(Fields.ALLOCATED, allocationStatus.value()); if (storeException != null) { builder.startObject(Fields.STORE_EXCEPTION); - OpenSearchException.generateThrowableXContent(builder, params, storeException); + BaseExceptionsHelper.generateThrowableXContent(builder, params, storeException); builder.endObject(); } return builder; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/stats/CommonStatsFlags.java b/server/src/main/java/org/opensearch/action/admin/indices/stats/CommonStatsFlags.java index 9a24d8a42dc9d..a359edee7c9cc 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/stats/CommonStatsFlags.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/stats/CommonStatsFlags.java @@ -34,10 +34,10 @@ import org.opensearch.LegacyESVersion; import org.opensearch.Version; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.io.stream.Writeable; +import org.opensearch.core.common.Strings; import java.io.IOException; import java.util.Collections; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/template/get/GetIndexTemplatesRequest.java b/server/src/main/java/org/opensearch/action/admin/indices/template/get/GetIndexTemplatesRequest.java index c0a990c0d8bbd..e97c97d78b6e6 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/template/get/GetIndexTemplatesRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/template/get/GetIndexTemplatesRequest.java @@ -33,9 +33,9 @@ import org.opensearch.action.ActionRequestValidationException; import org.opensearch.action.support.clustermanager.ClusterManagerNodeReadRequest; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/template/post/SimulateIndexTemplateRequest.java b/server/src/main/java/org/opensearch/action/admin/indices/template/post/SimulateIndexTemplateRequest.java index d2ebccc058f78..2dceb0bb866cd 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/template/post/SimulateIndexTemplateRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/template/post/SimulateIndexTemplateRequest.java @@ -36,9 +36,9 @@ import org.opensearch.action.admin.indices.template.put.PutComposableIndexTemplateAction; import org.opensearch.action.support.clustermanager.ClusterManagerNodeReadRequest; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import java.io.IOException; import java.util.Objects; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/template/put/PutComponentTemplateAction.java b/server/src/main/java/org/opensearch/action/admin/indices/template/put/PutComponentTemplateAction.java index f94e79b685bbc..594bde675239a 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/template/put/PutComponentTemplateAction.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/template/put/PutComponentTemplateAction.java @@ -38,9 +38,9 @@ import org.opensearch.action.support.clustermanager.ClusterManagerNodeRequest; import org.opensearch.cluster.metadata.ComponentTemplate; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/template/put/PutComposableIndexTemplateAction.java b/server/src/main/java/org/opensearch/action/admin/indices/template/put/PutComposableIndexTemplateAction.java index 7b05aaa0da711..bd2ca61cc94c7 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/template/put/PutComposableIndexTemplateAction.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/template/put/PutComposableIndexTemplateAction.java @@ -41,10 +41,10 @@ import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.ComposableIndexTemplate; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.regex.Regex; +import org.opensearch.core.common.Strings; import java.io.IOException; import java.util.Objects; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/upgrade/get/UpgradeStatusRequest.java b/server/src/main/java/org/opensearch/action/admin/indices/upgrade/get/UpgradeStatusRequest.java index 48a19ccdf0b94..f3f0bc70ce201 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/upgrade/get/UpgradeStatusRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/upgrade/get/UpgradeStatusRequest.java @@ -33,8 +33,8 @@ package org.opensearch.action.admin.indices.upgrade.get; import org.opensearch.action.support.broadcast.BroadcastRequest; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.core.common.Strings; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/validate/query/ValidateQueryRequest.java b/server/src/main/java/org/opensearch/action/admin/indices/validate/query/ValidateQueryRequest.java index ce2507605772e..91485d84eceb3 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/validate/query/ValidateQueryRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/validate/query/ValidateQueryRequest.java @@ -37,11 +37,11 @@ import org.opensearch.action.ValidateActions; import org.opensearch.action.support.IndicesOptions; import org.opensearch.action.support.broadcast.BroadcastRequest; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.core.xcontent.ToXContentObject; import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.common.Strings; import org.opensearch.index.query.MatchAllQueryBuilder; import org.opensearch.index.query.QueryBuilder; 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 e9534bf224d77..1ffc6e867df45 100644 --- a/server/src/main/java/org/opensearch/action/bulk/BulkItemResponse.java +++ b/server/src/main/java/org/opensearch/action/bulk/BulkItemResponse.java @@ -32,6 +32,7 @@ package org.opensearch.action.bulk; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.ExceptionsHelper; import org.opensearch.LegacyESVersion; import org.opensearch.OpenSearchException; @@ -95,7 +96,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(_ID, failure.getId()); builder.field(STATUS, failure.getStatus().getStatus()); builder.startObject(ERROR); - OpenSearchException.generateThrowableXContent(builder, params, failure.getCause()); + BaseExceptionsHelper.generateThrowableXContent(builder, params, failure.getCause()); builder.endObject(); } builder.endObject(); @@ -363,7 +364,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(ID_FIELD, id); } builder.startObject(CAUSE_FIELD); - OpenSearchException.generateThrowableXContent(builder, params, cause); + BaseExceptionsHelper.generateThrowableXContent(builder, params, cause); builder.endObject(); builder.field(STATUS_FIELD, status.getStatus()); return builder; diff --git a/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java b/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java index 3af4227bf46ca..2c42b0b8a74b0 100644 --- a/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java +++ b/server/src/main/java/org/opensearch/action/bulk/BulkRequest.java @@ -45,13 +45,13 @@ import org.opensearch.action.support.replication.ReplicationRequest; import org.opensearch.action.update.UpdateRequest; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.bytes.BytesArray; import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.xcontent.XContentType; +import org.opensearch.core.common.Strings; import org.opensearch.search.fetch.subphase.FetchSourceContext; import java.io.IOException; @@ -463,7 +463,7 @@ private void applyGlobalMandatoryParameters(DocWriteRequest request) { } private static String valueOrDefault(String value, String globalDefault) { - if (Strings.isNullOrEmpty(value) && !Strings.isNullOrEmpty(globalDefault)) { + if (Strings.isNullOrEmpty(value) && Strings.isNullOrEmpty(globalDefault) == false) { return globalDefault; } return value; diff --git a/server/src/main/java/org/opensearch/action/bulk/TransportBulkAction.java b/server/src/main/java/org/opensearch/action/bulk/TransportBulkAction.java index 19f610908677f..23050c870697f 100644 --- a/server/src/main/java/org/opensearch/action/bulk/TransportBulkAction.java +++ b/server/src/main/java/org/opensearch/action/bulk/TransportBulkAction.java @@ -35,10 +35,10 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.lucene.util.SparseFixedBitSet; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.LegacyESVersion; import org.opensearch.core.Assertions; import org.opensearch.OpenSearchParseException; -import org.opensearch.ExceptionsHelper; import org.opensearch.ResourceAlreadyExistsException; import org.opensearch.Version; import org.opensearch.action.ActionListener; @@ -326,7 +326,7 @@ protected void doRun() { @Override public void onFailure(Exception e) { - if (!(ExceptionsHelper.unwrapCause(e) instanceof ResourceAlreadyExistsException)) { + if (!(BaseExceptionsHelper.unwrapCause(e) instanceof ResourceAlreadyExistsException)) { // fail all requests involving this index, if create didn't work for (int i = 0; i < bulkRequest.requests.size(); i++) { DocWriteRequest request = bulkRequest.requests.get(i); diff --git a/server/src/main/java/org/opensearch/action/bulk/TransportShardBulkAction.java b/server/src/main/java/org/opensearch/action/bulk/TransportShardBulkAction.java index 8ca90ab7ca577..9e6b1bd59b386 100644 --- a/server/src/main/java/org/opensearch/action/bulk/TransportShardBulkAction.java +++ b/server/src/main/java/org/opensearch/action/bulk/TransportShardBulkAction.java @@ -36,7 +36,7 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.logging.log4j.util.MessageSupplier; -import org.opensearch.ExceptionsHelper; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.action.ActionListener; import org.opensearch.action.ActionListenerResponseHandler; import org.opensearch.action.ActionRunnable; @@ -719,7 +719,7 @@ && isConflictException(executionResult.getFailure().getCause()) } private static boolean isConflictException(final Exception e) { - return ExceptionsHelper.unwrapCause(e) instanceof VersionConflictEngineException; + return BaseExceptionsHelper.unwrapCause(e) instanceof VersionConflictEngineException; } /** diff --git a/server/src/main/java/org/opensearch/action/delete/DeleteRequest.java b/server/src/main/java/org/opensearch/action/delete/DeleteRequest.java index ce723df0c383a..30d5845531f26 100644 --- a/server/src/main/java/org/opensearch/action/delete/DeleteRequest.java +++ b/server/src/main/java/org/opensearch/action/delete/DeleteRequest.java @@ -40,10 +40,10 @@ import org.opensearch.action.DocWriteRequest; import org.opensearch.action.support.replication.ReplicatedWriteRequest; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.lucene.uid.Versions; +import org.opensearch.core.common.Strings; import org.opensearch.index.VersionType; import org.opensearch.index.mapper.MapperService; import org.opensearch.index.shard.ShardId; diff --git a/server/src/main/java/org/opensearch/action/explain/ExplainRequest.java b/server/src/main/java/org/opensearch/action/explain/ExplainRequest.java index 58297dffc6762..8e20a921c7b3f 100644 --- a/server/src/main/java/org/opensearch/action/explain/ExplainRequest.java +++ b/server/src/main/java/org/opensearch/action/explain/ExplainRequest.java @@ -37,9 +37,9 @@ import org.opensearch.action.ValidateActions; import org.opensearch.action.support.single.shard.SingleShardRequest; import org.opensearch.core.ParseField; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ToXContentObject; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.index.mapper.MapperService; diff --git a/server/src/main/java/org/opensearch/action/explain/ExplainRequestBuilder.java b/server/src/main/java/org/opensearch/action/explain/ExplainRequestBuilder.java index a582b29346665..3031cb6067469 100644 --- a/server/src/main/java/org/opensearch/action/explain/ExplainRequestBuilder.java +++ b/server/src/main/java/org/opensearch/action/explain/ExplainRequestBuilder.java @@ -35,7 +35,7 @@ import org.opensearch.action.support.single.shard.SingleShardOperationRequestBuilder; import org.opensearch.client.OpenSearchClient; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.index.query.QueryBuilder; import org.opensearch.search.fetch.subphase.FetchSourceContext; diff --git a/server/src/main/java/org/opensearch/action/fieldcaps/FieldCapabilitiesRequest.java b/server/src/main/java/org/opensearch/action/fieldcaps/FieldCapabilitiesRequest.java index cbee31272d5cc..02d911a13c5ea 100644 --- a/server/src/main/java/org/opensearch/action/fieldcaps/FieldCapabilitiesRequest.java +++ b/server/src/main/java/org/opensearch/action/fieldcaps/FieldCapabilitiesRequest.java @@ -38,9 +38,9 @@ import org.opensearch.action.IndicesRequest; import org.opensearch.action.ValidateActions; import org.opensearch.action.support.IndicesOptions; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ToXContentObject; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.index.query.QueryBuilder; diff --git a/server/src/main/java/org/opensearch/action/fieldcaps/TransportFieldCapabilitiesAction.java b/server/src/main/java/org/opensearch/action/fieldcaps/TransportFieldCapabilitiesAction.java index 2d669a5f1aad9..84f13b7d22c92 100644 --- a/server/src/main/java/org/opensearch/action/fieldcaps/TransportFieldCapabilitiesAction.java +++ b/server/src/main/java/org/opensearch/action/fieldcaps/TransportFieldCapabilitiesAction.java @@ -40,9 +40,9 @@ import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexNameExpressionResolver; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.Strings; import org.opensearch.common.inject.Inject; import org.opensearch.common.util.concurrent.CountDown; +import org.opensearch.core.common.Strings; import org.opensearch.tasks.Task; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.RemoteClusterAware; diff --git a/server/src/main/java/org/opensearch/action/fieldcaps/TransportFieldCapabilitiesIndexAction.java b/server/src/main/java/org/opensearch/action/fieldcaps/TransportFieldCapabilitiesIndexAction.java index 4fffce147c6dc..5544edf8a1771 100644 --- a/server/src/main/java/org/opensearch/action/fieldcaps/TransportFieldCapabilitiesIndexAction.java +++ b/server/src/main/java/org/opensearch/action/fieldcaps/TransportFieldCapabilitiesIndexAction.java @@ -56,7 +56,7 @@ import org.opensearch.common.Nullable; import org.opensearch.common.inject.Inject; import org.opensearch.common.io.stream.StreamInput; -import org.opensearch.common.logging.LoggerMessageFormat; +import org.opensearch.core.common.logging.LoggerMessageFormat; import org.opensearch.index.mapper.MappedFieldType; import org.opensearch.index.mapper.MapperService; import org.opensearch.index.mapper.ObjectMapper; diff --git a/server/src/main/java/org/opensearch/action/get/GetRequest.java b/server/src/main/java/org/opensearch/action/get/GetRequest.java index d0347a16fa73e..f2b4c714ae43e 100644 --- a/server/src/main/java/org/opensearch/action/get/GetRequest.java +++ b/server/src/main/java/org/opensearch/action/get/GetRequest.java @@ -38,10 +38,10 @@ import org.opensearch.action.RealtimeRequest; import org.opensearch.action.ValidateActions; import org.opensearch.action.support.single.shard.SingleShardRequest; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.lucene.uid.Versions; +import org.opensearch.core.common.Strings; import org.opensearch.index.VersionType; import org.opensearch.index.mapper.MapperService; import org.opensearch.search.fetch.subphase.FetchSourceContext; diff --git a/server/src/main/java/org/opensearch/action/get/GetRequestBuilder.java b/server/src/main/java/org/opensearch/action/get/GetRequestBuilder.java index c1004a990747d..6237cf73f0ca8 100644 --- a/server/src/main/java/org/opensearch/action/get/GetRequestBuilder.java +++ b/server/src/main/java/org/opensearch/action/get/GetRequestBuilder.java @@ -35,7 +35,7 @@ import org.opensearch.action.support.single.shard.SingleShardOperationRequestBuilder; import org.opensearch.client.OpenSearchClient; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.index.VersionType; import org.opensearch.search.fetch.subphase.FetchSourceContext; diff --git a/server/src/main/java/org/opensearch/action/get/MultiGetResponse.java b/server/src/main/java/org/opensearch/action/get/MultiGetResponse.java index 88f78d2ca6815..3e2aa1316a7cc 100644 --- a/server/src/main/java/org/opensearch/action/get/MultiGetResponse.java +++ b/server/src/main/java/org/opensearch/action/get/MultiGetResponse.java @@ -32,6 +32,7 @@ package org.opensearch.action.get; +import org.opensearch.BaseOpenSearchException; import org.opensearch.OpenSearchException; import org.opensearch.Version; import org.opensearch.action.ActionResponse; @@ -130,7 +131,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.startObject(); builder.field(INDEX.getPreferredName(), index); builder.field(ID.getPreferredName(), id); - OpenSearchException.generateFailureXContent(builder, params, exception, true); + BaseOpenSearchException.generateFailureXContent(builder, params, exception, true); builder.endObject(); return builder; } diff --git a/server/src/main/java/org/opensearch/action/index/IndexRequest.java b/server/src/main/java/org/opensearch/action/index/IndexRequest.java index 669ae348c31c6..e9a504611782e 100644 --- a/server/src/main/java/org/opensearch/action/index/IndexRequest.java +++ b/server/src/main/java/org/opensearch/action/index/IndexRequest.java @@ -57,6 +57,7 @@ import org.opensearch.common.xcontent.XContentFactory; import org.opensearch.common.xcontent.XContentHelper; import org.opensearch.common.xcontent.XContentType; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.MediaType; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.index.VersionType; @@ -695,7 +696,7 @@ private void writeBody(StreamOutput out) throws IOException { @Override public String toString() { - String sSource = "_na_"; + String sSource = Strings.UNKNOWN_UUID_VALUE; try { if (source.length() > MAX_SOURCE_LENGTH_IN_TOSTRING) { sSource = "n/a, actual length: [" diff --git a/server/src/main/java/org/opensearch/action/ingest/GetPipelineRequest.java b/server/src/main/java/org/opensearch/action/ingest/GetPipelineRequest.java index 1c7b7efe61455..f6bbf44a09782 100644 --- a/server/src/main/java/org/opensearch/action/ingest/GetPipelineRequest.java +++ b/server/src/main/java/org/opensearch/action/ingest/GetPipelineRequest.java @@ -34,9 +34,9 @@ import org.opensearch.action.ActionRequestValidationException; import org.opensearch.action.support.clustermanager.ClusterManagerNodeReadRequest; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/action/ingest/SimulateDocumentBaseResult.java b/server/src/main/java/org/opensearch/action/ingest/SimulateDocumentBaseResult.java index 463f843f2547a..e15021d8db168 100644 --- a/server/src/main/java/org/opensearch/action/ingest/SimulateDocumentBaseResult.java +++ b/server/src/main/java/org/opensearch/action/ingest/SimulateDocumentBaseResult.java @@ -31,6 +31,7 @@ package org.opensearch.action.ingest; +import org.opensearch.BaseOpenSearchException; import org.opensearch.LegacyESVersion; import org.opensearch.OpenSearchException; import org.opensearch.core.ParseField; @@ -146,7 +147,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (failure == null) { ingestDocument.toXContent(builder, params); } else { - OpenSearchException.generateFailureXContent(builder, params, failure, true); + BaseOpenSearchException.generateFailureXContent(builder, params, failure, true); } builder.endObject(); return builder; diff --git a/server/src/main/java/org/opensearch/action/ingest/SimulateProcessorResult.java b/server/src/main/java/org/opensearch/action/ingest/SimulateProcessorResult.java index a44da4eeae4d5..3f0774cb13210 100644 --- a/server/src/main/java/org/opensearch/action/ingest/SimulateProcessorResult.java +++ b/server/src/main/java/org/opensearch/action/ingest/SimulateProcessorResult.java @@ -31,6 +31,7 @@ package org.opensearch.action.ingest; +import org.opensearch.BaseOpenSearchException; import org.opensearch.LegacyESVersion; import org.opensearch.OpenSearchException; import org.opensearch.core.ParseField; @@ -283,10 +284,10 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (failure != null && ingestDocument != null) { builder.startObject(IGNORED_ERROR_FIELD); - OpenSearchException.generateFailureXContent(builder, params, failure, true); + BaseOpenSearchException.generateFailureXContent(builder, params, failure, true); builder.endObject(); } else if (failure != null) { - OpenSearchException.generateFailureXContent(builder, params, failure, true); + BaseOpenSearchException.generateFailureXContent(builder, params, failure, true); } if (ingestDocument != null) { diff --git a/server/src/main/java/org/opensearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/opensearch/action/search/AbstractSearchAsyncAction.java index 9a94737c84385..43cf4c4e8dc79 100644 --- a/server/src/main/java/org/opensearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/opensearch/action/search/AbstractSearchAsyncAction.java @@ -34,8 +34,8 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; +import org.opensearch.BaseOpenSearchException; import org.opensearch.ExceptionsHelper; -import org.opensearch.OpenSearchException; import org.opensearch.Version; import org.opensearch.action.ActionListener; import org.opensearch.action.NoShardAvailableActionException; @@ -367,7 +367,7 @@ public final void executeNextPhase(SearchPhase currentPhase, SearchPhase nextPha final ShardOperationFailedException[] shardSearchFailures = ExceptionsHelper.groupBy(buildShardFailures()); Throwable cause = shardSearchFailures.length == 0 ? null - : OpenSearchException.guessRootCauses(shardSearchFailures[0].getCause())[0]; + : BaseOpenSearchException.guessRootCauses(shardSearchFailures[0].getCause())[0]; logger.debug(() -> new ParameterizedMessage("All shards failed for phase: [{}]", getName()), cause); onPhaseFailure(currentPhase, "all shards failed", cause); } else { @@ -381,7 +381,7 @@ public final void executeNextPhase(SearchPhase currentPhase, SearchPhase nextPha if (logger.isDebugEnabled()) { int numShardFailures = shardSearchFailures.length; shardSearchFailures = ExceptionsHelper.groupBy(shardSearchFailures); - Throwable cause = OpenSearchException.guessRootCauses(shardSearchFailures[0].getCause())[0]; + Throwable cause = BaseOpenSearchException.guessRootCauses(shardSearchFailures[0].getCause())[0]; logger.debug( () -> new ParameterizedMessage("{} shards failed for phase: [{}]", numShardFailures, getName()), cause diff --git a/server/src/main/java/org/opensearch/action/search/ClearScrollController.java b/server/src/main/java/org/opensearch/action/search/ClearScrollController.java index 4bb3cbb88ba01..eb0fa49a94050 100644 --- a/server/src/main/java/org/opensearch/action/search/ClearScrollController.java +++ b/server/src/main/java/org/opensearch/action/search/ClearScrollController.java @@ -38,8 +38,8 @@ import org.opensearch.action.support.GroupedActionListener; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodes; -import org.opensearch.common.Strings; import org.opensearch.common.util.concurrent.CountDown; +import org.opensearch.core.common.Strings; import org.opensearch.transport.Transport; import org.opensearch.transport.TransportResponse; diff --git a/server/src/main/java/org/opensearch/action/search/CreatePitController.java b/server/src/main/java/org/opensearch/action/search/CreatePitController.java index 745139fd1f1e8..2130c1f91b59d 100644 --- a/server/src/main/java/org/opensearch/action/search/CreatePitController.java +++ b/server/src/main/java/org/opensearch/action/search/CreatePitController.java @@ -18,11 +18,11 @@ import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.Strings; import org.opensearch.common.inject.Inject; import org.opensearch.common.io.stream.NamedWriteableRegistry; import org.opensearch.common.settings.Setting; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import org.opensearch.index.shard.ShardId; import org.opensearch.search.SearchPhaseResult; import org.opensearch.search.SearchShardTarget; diff --git a/server/src/main/java/org/opensearch/action/search/CreatePitRequest.java b/server/src/main/java/org/opensearch/action/search/CreatePitRequest.java index f2c7fc374e181..a679a66eb4acc 100644 --- a/server/src/main/java/org/opensearch/action/search/CreatePitRequest.java +++ b/server/src/main/java/org/opensearch/action/search/CreatePitRequest.java @@ -13,10 +13,10 @@ import org.opensearch.action.IndicesRequest; import org.opensearch.action.support.IndicesOptions; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.tasks.Task; diff --git a/server/src/main/java/org/opensearch/action/search/GetSearchPipelineRequest.java b/server/src/main/java/org/opensearch/action/search/GetSearchPipelineRequest.java index 4a1d3eaba87a0..15fefb1671707 100644 --- a/server/src/main/java/org/opensearch/action/search/GetSearchPipelineRequest.java +++ b/server/src/main/java/org/opensearch/action/search/GetSearchPipelineRequest.java @@ -10,9 +10,9 @@ import org.opensearch.action.ActionRequestValidationException; import org.opensearch.action.support.clustermanager.ClusterManagerNodeReadRequest; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import java.io.IOException; import java.util.Objects; 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 f66fe1ba55882..c372e82c094cf 100644 --- a/server/src/main/java/org/opensearch/action/search/MultiSearchResponse.java +++ b/server/src/main/java/org/opensearch/action/search/MultiSearchResponse.java @@ -32,6 +32,7 @@ package org.opensearch.action.search; +import org.opensearch.BaseOpenSearchException; import org.opensearch.LegacyESVersion; import org.opensearch.OpenSearchException; import org.opensearch.ExceptionsHelper; @@ -198,7 +199,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws for (Item item : items) { builder.startObject(); if (item.isFailure()) { - OpenSearchException.generateFailureXContent(builder, params, item.getFailure(), true); + BaseOpenSearchException.generateFailureXContent(builder, params, item.getFailure(), true); builder.field(Fields.STATUS, ExceptionsHelper.status(item.getFailure()).getStatus()); } else { item.getResponse().innerToXContent(builder, params); diff --git a/server/src/main/java/org/opensearch/action/search/PitService.java b/server/src/main/java/org/opensearch/action/search/PitService.java index 995d5fcd09d2a..f35c367dda129 100644 --- a/server/src/main/java/org/opensearch/action/search/PitService.java +++ b/server/src/main/java/org/opensearch/action/search/PitService.java @@ -17,9 +17,9 @@ import org.opensearch.client.node.NodeClient; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.Strings; import org.opensearch.common.inject.Inject; import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.core.common.Strings; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.Transport; import org.opensearch.transport.TransportException; diff --git a/server/src/main/java/org/opensearch/action/search/SearchContextId.java b/server/src/main/java/org/opensearch/action/search/SearchContextId.java index d54f3039395fe..68c5f3dc49596 100644 --- a/server/src/main/java/org/opensearch/action/search/SearchContextId.java +++ b/server/src/main/java/org/opensearch/action/search/SearchContextId.java @@ -33,7 +33,6 @@ package org.opensearch.action.search; import org.opensearch.Version; -import org.opensearch.common.Strings; import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.io.stream.ByteBufferStreamInput; import org.opensearch.common.io.stream.BytesStreamOutput; @@ -41,6 +40,7 @@ import org.opensearch.common.io.stream.NamedWriteableRegistry; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import org.opensearch.index.shard.ShardId; import org.opensearch.search.SearchPhaseResult; import org.opensearch.search.SearchShardTarget; 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 bd3a0eb93b375..891d9a1e2e924 100644 --- a/server/src/main/java/org/opensearch/action/search/SearchPhaseExecutionException.java +++ b/server/src/main/java/org/opensearch/action/search/SearchPhaseExecutionException.java @@ -32,6 +32,8 @@ package org.opensearch.action.search; +import org.opensearch.BaseExceptionsHelper; +import org.opensearch.BaseOpenSearchException; import org.opensearch.OpenSearchException; import org.opensearch.ExceptionsHelper; import org.opensearch.action.ShardOperationFailedException; @@ -121,7 +123,7 @@ public Throwable getCause() { Throwable cause = super.getCause(); if (cause == null) { // fall back to guessed root cause - for (OpenSearchException rootCause : guessRootCauses()) { + for (BaseOpenSearchException rootCause : guessRootCauses()) { return rootCause; } } @@ -159,27 +161,36 @@ protected void metadataToXContent(XContentBuilder builder, Params params) throws @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - Throwable ex = ExceptionsHelper.unwrapCause(this); + Throwable ex = BaseExceptionsHelper.unwrapCause(this); if (ex != this) { - generateThrowableXContent(builder, params, this); + BaseExceptionsHelper.generateThrowableXContent(builder, params, this); } else { // We don't have a cause when all shards failed, but we do have shards failures so we can "guess" a cause // (see {@link #getCause()}). Here, we use super.getCause() because we don't want the guessed exception to // be rendered twice (one in the "cause" field, one in "failed_shards") - innerToXContent(builder, params, this, getExceptionName(), getMessage(), getHeaders(), getMetadata(), super.getCause()); + BaseExceptionsHelper.innerToXContent( + builder, + params, + this, + getExceptionName(), + getMessage(), + getHeaders(), + getMetadata(), + super.getCause() + ); } return builder; } @Override - public OpenSearchException[] guessRootCauses() { + public BaseOpenSearchException[] guessRootCauses() { ShardOperationFailedException[] failures = ExceptionsHelper.groupBy(shardFailures); - List rootCauses = new ArrayList<>(failures.length); + List rootCauses = new ArrayList<>(failures.length); for (ShardOperationFailedException failure : failures) { - OpenSearchException[] guessRootCauses = OpenSearchException.guessRootCauses(failure.getCause()); + BaseOpenSearchException[] guessRootCauses = BaseOpenSearchException.guessRootCauses(failure.getCause()); rootCauses.addAll(Arrays.asList(guessRootCauses)); } - return rootCauses.toArray(new OpenSearchException[0]); + return rootCauses.toArray(new BaseOpenSearchException[0]); } @Override diff --git a/server/src/main/java/org/opensearch/action/search/SearchRequest.java b/server/src/main/java/org/opensearch/action/search/SearchRequest.java index dac0d2e8927cc..24009282567ec 100644 --- a/server/src/main/java/org/opensearch/action/search/SearchRequest.java +++ b/server/src/main/java/org/opensearch/action/search/SearchRequest.java @@ -39,10 +39,10 @@ import org.opensearch.action.IndicesRequest; import org.opensearch.action.support.IndicesOptions; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.search.Scroll; import org.opensearch.search.builder.PointInTimeBuilder; diff --git a/server/src/main/java/org/opensearch/action/search/ShardSearchFailure.java b/server/src/main/java/org/opensearch/action/search/ShardSearchFailure.java index 5edfd92422cbf..c820b7ada427a 100644 --- a/server/src/main/java/org/opensearch/action/search/ShardSearchFailure.java +++ b/server/src/main/java/org/opensearch/action/search/ShardSearchFailure.java @@ -32,6 +32,7 @@ package org.opensearch.action.search; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.ExceptionsHelper; import org.opensearch.OpenSearchException; import org.opensearch.action.OriginalIndices; @@ -85,17 +86,20 @@ public ShardSearchFailure(Exception e) { } public ShardSearchFailure(Exception e, @Nullable SearchShardTarget shardTarget) { + this(e, BaseExceptionsHelper.unwrapCause(e), shardTarget); + } + + private ShardSearchFailure(final Exception e, final Throwable unwrappedCause, @Nullable SearchShardTarget shardTarget) { super( shardTarget == null ? null : shardTarget.getFullyQualifiedIndexName(), shardTarget == null ? -1 : shardTarget.getShardId().getId(), - ExceptionsHelper.detailedMessage(e), - ExceptionsHelper.status(ExceptionsHelper.unwrapCause(e)), - ExceptionsHelper.unwrapCause(e) + BaseExceptionsHelper.detailedMessage(e), + ExceptionsHelper.status(unwrappedCause), + unwrappedCause ); - final Throwable actual = ExceptionsHelper.unwrapCause(e); - if (actual instanceof SearchException) { - this.shardTarget = ((SearchException) actual).shard(); + if (unwrappedCause instanceof SearchException) { + this.shardTarget = ((SearchException) unwrappedCause).shard(); } else if (shardTarget != null) { this.shardTarget = shardTarget; } @@ -116,7 +120,7 @@ public String toString() { + "], reason [" + reason + "], cause [" - + (cause == null ? "_na" : ExceptionsHelper.stackTrace(cause)) + + (cause == null ? "_na" : BaseExceptionsHelper.stackTrace(cause)) + "]"; } @@ -144,7 +148,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws } builder.field(REASON_FIELD); builder.startObject(); - OpenSearchException.generateThrowableXContent(builder, params, cause); + BaseExceptionsHelper.generateThrowableXContent(builder, params, cause); builder.endObject(); } builder.endObject(); diff --git a/server/src/main/java/org/opensearch/action/search/TransportSearchAction.java b/server/src/main/java/org/opensearch/action/search/TransportSearchAction.java index b650df730cb24..69f529fe1d00c 100644 --- a/server/src/main/java/org/opensearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/opensearch/action/search/TransportSearchAction.java @@ -57,7 +57,6 @@ import org.opensearch.cluster.routing.ShardIterator; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.breaker.CircuitBreaker; import org.opensearch.common.inject.Inject; import org.opensearch.common.io.stream.NamedWriteableRegistry; @@ -67,6 +66,7 @@ import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.AtomicArray; import org.opensearch.common.util.concurrent.CountDown; +import org.opensearch.core.common.Strings; import org.opensearch.index.Index; import org.opensearch.index.query.Rewriteable; import org.opensearch.index.shard.ShardId; diff --git a/server/src/main/java/org/opensearch/action/support/AutoCreateIndex.java b/server/src/main/java/org/opensearch/action/support/AutoCreateIndex.java index f0c8a9a7a11ba..9e8cbd7bf40c2 100644 --- a/server/src/main/java/org/opensearch/action/support/AutoCreateIndex.java +++ b/server/src/main/java/org/opensearch/action/support/AutoCreateIndex.java @@ -35,13 +35,13 @@ import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexNameExpressionResolver; import org.opensearch.common.Booleans; -import org.opensearch.common.Strings; import org.opensearch.common.collect.Tuple; import org.opensearch.common.regex.Regex; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Setting.Property; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; import org.opensearch.index.IndexNotFoundException; import org.opensearch.index.mapper.MapperService; import org.opensearch.indices.SystemIndices; diff --git a/server/src/main/java/org/opensearch/action/support/DefaultShardOperationFailedException.java b/server/src/main/java/org/opensearch/action/support/DefaultShardOperationFailedException.java index 687aebf29b183..35334cbbad801 100644 --- a/server/src/main/java/org/opensearch/action/support/DefaultShardOperationFailedException.java +++ b/server/src/main/java/org/opensearch/action/support/DefaultShardOperationFailedException.java @@ -32,6 +32,7 @@ package org.opensearch.action.support; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.OpenSearchException; import org.opensearch.ExceptionsHelper; import org.opensearch.action.ShardOperationFailedException; @@ -135,7 +136,7 @@ protected XContentBuilder innerToXContent(XContentBuilder builder, Params params builder.field("status", status.name()); if (reason != null) { builder.startObject("reason"); - OpenSearchException.generateThrowableXContent(builder, params, cause); + BaseExceptionsHelper.generateThrowableXContent(builder, params, cause); builder.endObject(); } return builder; diff --git a/server/src/main/java/org/opensearch/action/support/TransportActions.java b/server/src/main/java/org/opensearch/action/support/TransportActions.java index 03e7509b3b8e3..62059ca3f0754 100644 --- a/server/src/main/java/org/opensearch/action/support/TransportActions.java +++ b/server/src/main/java/org/opensearch/action/support/TransportActions.java @@ -33,7 +33,7 @@ package org.opensearch.action.support; import org.apache.lucene.store.AlreadyClosedException; -import org.opensearch.ExceptionsHelper; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.action.NoShardAvailableActionException; import org.opensearch.action.UnavailableShardsException; import org.opensearch.index.IndexNotFoundException; @@ -48,7 +48,7 @@ public class TransportActions { public static boolean isShardNotAvailableException(final Throwable e) { - final Throwable actual = ExceptionsHelper.unwrapCause(e); + final Throwable actual = BaseExceptionsHelper.unwrapCause(e); return (actual instanceof ShardNotFoundException || actual instanceof IndexNotFoundException || actual instanceof IllegalIndexShardStateException diff --git a/server/src/main/java/org/opensearch/action/support/clustermanager/info/ClusterInfoRequest.java b/server/src/main/java/org/opensearch/action/support/clustermanager/info/ClusterInfoRequest.java index d0b9d291c16c7..0059f5c836202 100644 --- a/server/src/main/java/org/opensearch/action/support/clustermanager/info/ClusterInfoRequest.java +++ b/server/src/main/java/org/opensearch/action/support/clustermanager/info/ClusterInfoRequest.java @@ -36,9 +36,9 @@ import org.opensearch.action.IndicesRequest; import org.opensearch.action.support.IndicesOptions; import org.opensearch.action.support.clustermanager.ClusterManagerNodeReadRequest; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/action/support/replication/ReplicationOperation.java b/server/src/main/java/org/opensearch/action/support/replication/ReplicationOperation.java index 423fbd12a6447..3cec823422eb2 100644 --- a/server/src/main/java/org/opensearch/action/support/replication/ReplicationOperation.java +++ b/server/src/main/java/org/opensearch/action/support/replication/ReplicationOperation.java @@ -34,6 +34,7 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.store.AlreadyClosedException; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.core.Assertions; import org.opensearch.ExceptionsHelper; import org.opensearch.OpenSearchException; @@ -333,7 +334,7 @@ public void onFinished() { @Override public boolean shouldRetry(Exception e) { - final Throwable cause = ExceptionsHelper.unwrapCause(e); + final Throwable cause = BaseExceptionsHelper.unwrapCause(e); return cause instanceof CircuitBreakingException || cause instanceof OpenSearchRejectedExecutionException || cause instanceof ConnectTransportException; @@ -358,7 +359,7 @@ private void updateCheckPoints(ShardRouting shard, LongSupplier localCheckpointS } private void onNoLongerPrimary(Exception failure) { - final Throwable cause = ExceptionsHelper.unwrapCause(failure); + final Throwable cause = BaseExceptionsHelper.unwrapCause(failure); final boolean nodeIsClosing = cause instanceof NodeClosedException; final String message; if (nodeIsClosing) { diff --git a/server/src/main/java/org/opensearch/action/support/replication/ReplicationResponse.java b/server/src/main/java/org/opensearch/action/support/replication/ReplicationResponse.java index 1e0adfe326918..063e657b25f21 100644 --- a/server/src/main/java/org/opensearch/action/support/replication/ReplicationResponse.java +++ b/server/src/main/java/org/opensearch/action/support/replication/ReplicationResponse.java @@ -32,7 +32,7 @@ package org.opensearch.action.support.replication; -import org.opensearch.ExceptionsHelper; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.OpenSearchException; import org.opensearch.action.ActionResponse; import org.opensearch.action.ShardOperationFailedException; @@ -261,7 +261,7 @@ public Failure(StreamInput in) throws IOException { } public Failure(ShardId shardId, @Nullable String nodeId, Exception cause, RestStatus status, boolean primary) { - super(shardId.getIndexName(), shardId.getId(), ExceptionsHelper.detailedMessage(cause), status, cause); + super(shardId.getIndexName(), shardId.getId(), BaseExceptionsHelper.detailedMessage(cause), status, cause); this.shardId = shardId; this.nodeId = nodeId; this.primary = primary; @@ -304,7 +304,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(_NODE, nodeId); builder.field(REASON); builder.startObject(); - OpenSearchException.generateThrowableXContent(builder, params, cause); + BaseExceptionsHelper.generateThrowableXContent(builder, params, cause); builder.endObject(); builder.field(STATUS, status); builder.field(PRIMARY, primary); diff --git a/server/src/main/java/org/opensearch/action/support/single/shard/TransportSingleShardAction.java b/server/src/main/java/org/opensearch/action/support/single/shard/TransportSingleShardAction.java index 927d3946a3643..39398392c0ee8 100644 --- a/server/src/main/java/org/opensearch/action/support/single/shard/TransportSingleShardAction.java +++ b/server/src/main/java/org/opensearch/action/support/single/shard/TransportSingleShardAction.java @@ -54,7 +54,7 @@ import org.opensearch.common.Nullable; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.Writeable; -import org.opensearch.common.logging.LoggerMessageFormat; +import org.opensearch.core.common.logging.LoggerMessageFormat; import org.opensearch.index.shard.ShardId; import org.opensearch.tasks.Task; import org.opensearch.threadpool.ThreadPool; 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 34a8412ea4129..9172bb0aca098 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 @@ -34,12 +34,12 @@ import org.opensearch.action.ActionRequest; import org.opensearch.action.ActionRequestValidationException; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.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.Strings; import org.opensearch.tasks.Task; import org.opensearch.tasks.TaskId; diff --git a/server/src/main/java/org/opensearch/action/termvectors/MultiTermVectorsResponse.java b/server/src/main/java/org/opensearch/action/termvectors/MultiTermVectorsResponse.java index ff84ceb22e043..f1e05592731a5 100644 --- a/server/src/main/java/org/opensearch/action/termvectors/MultiTermVectorsResponse.java +++ b/server/src/main/java/org/opensearch/action/termvectors/MultiTermVectorsResponse.java @@ -32,7 +32,7 @@ package org.opensearch.action.termvectors; -import org.opensearch.OpenSearchException; +import org.opensearch.BaseOpenSearchException; import org.opensearch.Version; import org.opensearch.action.ActionResponse; import org.opensearch.common.io.stream.StreamInput; @@ -141,7 +141,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws Failure failure = response.getFailure(); builder.field(Fields._INDEX, failure.getIndex()); builder.field(Fields._ID, failure.getId()); - OpenSearchException.generateFailureXContent(builder, params, failure.getCause(), true); + BaseOpenSearchException.generateFailureXContent(builder, params, failure.getCause(), true); builder.endObject(); } else { TermVectorsResponse getResponse = response.getResponse(); diff --git a/server/src/main/java/org/opensearch/action/update/UpdateRequest.java b/server/src/main/java/org/opensearch/action/update/UpdateRequest.java index 4e3b9cd224526..13471059e4bdf 100644 --- a/server/src/main/java/org/opensearch/action/update/UpdateRequest.java +++ b/server/src/main/java/org/opensearch/action/update/UpdateRequest.java @@ -43,12 +43,12 @@ import org.opensearch.action.support.replication.ReplicationRequest; import org.opensearch.action.support.single.instance.InstanceShardOperationRequest; import org.opensearch.common.Nullable; -import org.opensearch.core.ParseField; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.lucene.uid.Versions; import org.opensearch.common.xcontent.LoggingDeprecationHandler; +import org.opensearch.core.ParseField; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.core.xcontent.ObjectParser; import org.opensearch.core.xcontent.ToXContentObject; diff --git a/server/src/main/java/org/opensearch/cluster/ClusterState.java b/server/src/main/java/org/opensearch/cluster/ClusterState.java index fda9b9a0ea588..e04ce938e7fdc 100644 --- a/server/src/main/java/org/opensearch/cluster/ClusterState.java +++ b/server/src/main/java/org/opensearch/cluster/ClusterState.java @@ -48,7 +48,6 @@ import org.opensearch.cluster.routing.RoutingTable; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.Strings; import org.opensearch.common.UUIDs; import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.io.stream.BytesStreamOutput; @@ -58,6 +57,7 @@ import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.io.stream.VersionedNamedWriteable; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.discovery.Discovery; @@ -602,7 +602,7 @@ public static class Builder { private final ClusterName clusterName; private long version = 0; - private String uuid = UNKNOWN_UUID; + private String uuid = Strings.UNKNOWN_UUID_VALUE; private Metadata metadata = Metadata.EMPTY_METADATA; private RoutingTable routingTable = RoutingTable.EMPTY_ROUTING_TABLE; private DiscoveryNodes nodes = DiscoveryNodes.EMPTY_NODES; diff --git a/server/src/main/java/org/opensearch/cluster/action/shard/ShardStateAction.java b/server/src/main/java/org/opensearch/cluster/action/shard/ShardStateAction.java index 002c5fd3b89db..853690fa1e242 100644 --- a/server/src/main/java/org/opensearch/cluster/action/shard/ShardStateAction.java +++ b/server/src/main/java/org/opensearch/cluster/action/shard/ShardStateAction.java @@ -35,6 +35,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.OpenSearchException; import org.opensearch.ExceptionsHelper; import org.opensearch.action.ActionListener; @@ -629,7 +630,7 @@ public String toString() { components.add("primary term [" + primaryTerm + "]"); components.add("message [" + message + "]"); if (failure != null) { - components.add("failure [" + ExceptionsHelper.detailedMessage(failure) + "]"); + components.add("failure [" + BaseExceptionsHelper.detailedMessage(failure) + "]"); } components.add("markAsStale [" + markAsStale + "]"); return String.join(", ", components); diff --git a/server/src/main/java/org/opensearch/cluster/decommission/DecommissionHelper.java b/server/src/main/java/org/opensearch/cluster/decommission/DecommissionHelper.java index 2feee082ed8c8..6c76fd923aad0 100644 --- a/server/src/main/java/org/opensearch/cluster/decommission/DecommissionHelper.java +++ b/server/src/main/java/org/opensearch/cluster/decommission/DecommissionHelper.java @@ -13,8 +13,8 @@ import org.opensearch.cluster.coordination.CoordinationMetadata.VotingConfigExclusion; import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.node.DiscoveryNode; -import org.opensearch.common.Strings; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import java.util.HashSet; import java.util.Iterator; diff --git a/server/src/main/java/org/opensearch/cluster/metadata/AliasAction.java b/server/src/main/java/org/opensearch/cluster/metadata/AliasAction.java index 46702a0d78caf..47c6cad04343e 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/AliasAction.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/AliasAction.java @@ -34,7 +34,7 @@ import org.opensearch.action.admin.indices.alias.IndicesAliasesRequest; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; /** * Individual operation to perform on the cluster state as part of an {@link IndicesAliasesRequest}. 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 56f5e44dc83a0..612057b0de9b5 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/AliasMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/AliasMetadata.java @@ -37,7 +37,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.bytes.BytesReference; import org.opensearch.common.compress.CompressedXContent; import org.opensearch.common.io.stream.StreamInput; @@ -46,6 +45,7 @@ import org.opensearch.common.xcontent.XContentFactory; import org.opensearch.common.xcontent.XContentHelper; import org.opensearch.common.xcontent.XContentType; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.XContentBuilder; @@ -268,7 +268,7 @@ public static Diff readDiffFrom(StreamInput in) throws IOExceptio @Override public String toString() { - return Strings.toString(XContentType.JSON, this, true, true); + return org.opensearch.common.Strings.toString(XContentType.JSON, this, true, true); } @Override @@ -312,7 +312,7 @@ public Builder filter(CompressedXContent filter) { } public Builder filter(String filter) { - if (!Strings.hasLength(filter)) { + if (Strings.hasLength(filter) == false) { this.filter = null; return this; } diff --git a/server/src/main/java/org/opensearch/cluster/metadata/AliasValidator.java b/server/src/main/java/org/opensearch/cluster/metadata/AliasValidator.java index 7e69eaf91c662..948f756331d46 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/AliasValidator.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/AliasValidator.java @@ -34,9 +34,9 @@ import org.opensearch.action.admin.indices.alias.Alias; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.xcontent.LoggingDeprecationHandler; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.common.xcontent.XContentFactory; import org.opensearch.common.xcontent.XContentHelper; @@ -101,7 +101,7 @@ public void validateAliasStandalone(Alias alias) { public void validateAlias(String alias, String index, @Nullable String indexRouting, Function indexLookup) { validateAliasStandalone(alias, indexRouting); - if (!Strings.hasText(index)) { + if (Strings.hasText(index) == false) { throw new IllegalArgumentException("index name is required"); } diff --git a/server/src/main/java/org/opensearch/cluster/metadata/IndexAbstraction.java b/server/src/main/java/org/opensearch/cluster/metadata/IndexAbstraction.java index 11ead3e29c346..0c316373e484f 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/IndexAbstraction.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/IndexAbstraction.java @@ -33,8 +33,8 @@ import org.opensearch.common.Nullable; import org.opensearch.common.SetOnce; -import org.opensearch.common.Strings; import org.opensearch.common.collect.Tuple; +import org.opensearch.core.common.Strings; import java.util.ArrayList; import java.util.Collections; diff --git a/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java b/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java index 45228b6644c7c..2d7d19387b489 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java @@ -56,6 +56,7 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.xcontent.XContentFactory; import org.opensearch.common.xcontent.XContentHelper; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.XContentBuilder; @@ -559,7 +560,7 @@ public static APIBlock readFrom(StreamInput input) throws IOException { Function.identity(), Property.IndexScope ); - public static final String INDEX_UUID_NA_VALUE = "_na_"; + public static final String INDEX_UUID_NA_VALUE = Strings.UNKNOWN_UUID_VALUE; public static final String INDEX_ROUTING_REQUIRE_GROUP_PREFIX = "index.routing.allocation.require"; public static final String INDEX_ROUTING_INCLUDE_GROUP_PREFIX = "index.routing.allocation.include"; 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 4e5a973b41614..ce4f2f171e740 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/IndexNameExpressionResolver.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/IndexNameExpressionResolver.java @@ -40,7 +40,6 @@ import org.opensearch.cluster.ClusterState; import org.opensearch.common.Booleans; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.collect.Tuple; import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.regex.Regex; @@ -50,6 +49,7 @@ import org.opensearch.common.util.CollectionUtils; import org.opensearch.common.util.concurrent.ThreadContext; import org.opensearch.common.util.set.Sets; +import org.opensearch.core.common.Strings; import org.opensearch.index.Index; import org.opensearch.index.IndexNotFoundException; import org.opensearch.indices.IndexClosedException; diff --git a/server/src/main/java/org/opensearch/cluster/metadata/Metadata.java b/server/src/main/java/org/opensearch/cluster/metadata/Metadata.java index 105ae3690d8b6..fe33b91be8805 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/Metadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/Metadata.java @@ -49,7 +49,6 @@ import org.opensearch.cluster.coordination.CoordinationMetadata; import org.opensearch.cluster.decommission.DecommissionAttributeMetadata; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.UUIDs; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; @@ -57,11 +56,12 @@ import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Setting.Property; import org.opensearch.common.settings.Settings; +import org.opensearch.common.xcontent.XContentHelper; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.NamedObjectNotFoundException; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.XContentBuilder; -import org.opensearch.common.xcontent.XContentHelper; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.gateway.MetadataStateFormat; import org.opensearch.index.Index; @@ -105,7 +105,7 @@ public class Metadata implements Iterable, Diffable, To private static final Logger logger = LogManager.getLogger(Metadata.class); public static final String ALL = "_all"; - public static final String UNKNOWN_CLUSTER_UUID = "_na_"; + public static final String UNKNOWN_CLUSTER_UUID = Strings.UNKNOWN_UUID_VALUE; public static final Pattern NUMBER_PATTERN = Pattern.compile("[0-9]+$"); /** 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 99df67699e4d3..ae09bfe363d99 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java @@ -65,7 +65,6 @@ import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.Nullable; import org.opensearch.common.Priority; -import org.opensearch.common.Strings; import org.opensearch.common.UUIDs; import org.opensearch.common.ValidationException; import org.opensearch.common.compress.CompressedXContent; @@ -75,8 +74,9 @@ import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.FeatureFlags; -import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.common.xcontent.XContentHelper; +import org.opensearch.core.common.Strings; +import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.env.Environment; import org.opensearch.index.Index; import org.opensearch.index.IndexModule; @@ -268,8 +268,11 @@ 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 (!Strings.validFileName(index)) { - throw exceptionCtor.apply(index, "must not contain the following characters " + Strings.INVALID_FILENAME_CHARS); + 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 (index.isEmpty()) { throw exceptionCtor.apply(index, "must not be empty"); @@ -1364,7 +1367,7 @@ private static List validatePrivateSettingsNotExplicitlySet(Settings set private static List validateIndexCustomPath(Settings settings, @Nullable Path sharedDataPath) { String customPath = IndexMetadata.INDEX_DATA_PATH_SETTING.get(settings); List validationErrors = new ArrayList<>(); - if (!Strings.isEmpty(customPath)) { + if (Strings.isEmpty(customPath) == false) { if (sharedDataPath == null) { validationErrors.add("path.shared_data must be set in order to use custom data paths"); } else { diff --git a/server/src/main/java/org/opensearch/cluster/metadata/MetadataIndexAliasesService.java b/server/src/main/java/org/opensearch/cluster/metadata/MetadataIndexAliasesService.java index 939d6c7b9af3b..83e2511574867 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataIndexAliasesService.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataIndexAliasesService.java @@ -43,8 +43,8 @@ import org.opensearch.cluster.service.ClusterManagerTaskThrottler; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.Priority; -import org.opensearch.common.Strings; import org.opensearch.common.inject.Inject; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.index.Index; import org.opensearch.index.IndexNotFoundException; diff --git a/server/src/main/java/org/opensearch/cluster/metadata/MetadataIndexStateService.java b/server/src/main/java/org/opensearch/cluster/metadata/MetadataIndexStateService.java index 4f1000e3407fd..840e4e77de73e 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataIndexStateService.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataIndexStateService.java @@ -70,7 +70,6 @@ import org.opensearch.cluster.routing.allocation.AllocationService; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.Priority; -import org.opensearch.common.Strings; import org.opensearch.common.UUIDs; import org.opensearch.common.collect.ImmutableOpenIntMap; import org.opensearch.common.collect.Tuple; @@ -81,6 +80,7 @@ import org.opensearch.common.util.concurrent.AtomicArray; import org.opensearch.common.util.concurrent.ConcurrentCollections; import org.opensearch.common.util.concurrent.CountDown; +import org.opensearch.core.common.Strings; import org.opensearch.index.Index; import org.opensearch.index.IndexNotFoundException; import org.opensearch.index.shard.ShardId; 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 c0f2bbfa40439..76860e568f1a1 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataIndexTemplateService.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataIndexTemplateService.java @@ -48,7 +48,6 @@ import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.Nullable; import org.opensearch.common.Priority; -import org.opensearch.common.Strings; import org.opensearch.common.UUIDs; import org.opensearch.common.ValidationException; import org.opensearch.common.bytes.BytesReference; @@ -60,10 +59,11 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.set.Sets; -import org.opensearch.core.xcontent.NamedXContentRegistry; -import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.common.xcontent.XContentFactory; import org.opensearch.common.xcontent.XContentType; +import org.opensearch.core.common.Strings; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.index.Index; import org.opensearch.index.IndexService; import org.opensearch.index.mapper.MapperParsingException; @@ -302,7 +302,7 @@ ClusterState addComponentTemplate( if (stringMappings != null) { Map parsedMappings = MapperService.parseMapping(xContentRegistry, stringMappings); if (parsedMappings.size() > 0) { - stringMappings = Strings.toString( + stringMappings = org.opensearch.common.Strings.toString( XContentFactory.jsonBuilder().startObject().field(MapperService.SINGLE_MAPPING_NAME, parsedMappings).endObject() ); } @@ -591,7 +591,7 @@ public ClusterState addIndexTemplateV2( if (stringMappings != null) { Map parsedMappings = MapperService.parseMapping(xContentRegistry, stringMappings); if (parsedMappings.size() > 0) { - stringMappings = Strings.toString( + stringMappings = org.opensearch.common.Strings.toString( XContentFactory.jsonBuilder().startObject().field(MapperService.SINGLE_MAPPING_NAME, parsedMappings).endObject() ); } @@ -1502,9 +1502,12 @@ private void validate(String name, @Nullable Settings settings, List ind if (indexPattern.startsWith("_")) { validationErrors.add("index_pattern [" + indexPattern + "] must not start with '_'"); } - if (Strings.validFileNameExcludingAstrix(indexPattern) == false) { + if (org.opensearch.common.Strings.validFileNameExcludingAstrix(indexPattern) == false) { validationErrors.add( - "index_pattern [" + indexPattern + "] must not contain the following characters " + Strings.INVALID_FILENAME_CHARS + "index_pattern [" + + indexPattern + + "] must not contain the following characters " + + org.opensearch.common.Strings.INVALID_FILENAME_CHARS ); } } diff --git a/server/src/main/java/org/opensearch/cluster/metadata/MetadataMappingService.java b/server/src/main/java/org/opensearch/cluster/metadata/MetadataMappingService.java index 665bdf0c4e2ff..9cb1eb6576ac5 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataMappingService.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataMappingService.java @@ -48,11 +48,11 @@ import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.Nullable; import org.opensearch.common.Priority; -import org.opensearch.common.Strings; import org.opensearch.common.compress.CompressedXContent; import org.opensearch.common.inject.Inject; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.io.IOUtils; +import org.opensearch.core.common.Strings; import org.opensearch.index.Index; import org.opensearch.index.IndexService; import org.opensearch.index.mapper.DocumentMapper; diff --git a/server/src/main/java/org/opensearch/cluster/node/DiscoveryNodeFilters.java b/server/src/main/java/org/opensearch/cluster/node/DiscoveryNodeFilters.java index b47ad43d44efb..4fd2905495961 100644 --- a/server/src/main/java/org/opensearch/cluster/node/DiscoveryNodeFilters.java +++ b/server/src/main/java/org/opensearch/cluster/node/DiscoveryNodeFilters.java @@ -33,12 +33,12 @@ package org.opensearch.cluster.node; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.network.InetAddresses; import org.opensearch.common.network.NetworkAddress; import org.opensearch.common.regex.Regex; import org.opensearch.common.settings.Settings; import org.opensearch.common.transport.TransportAddress; +import org.opensearch.core.common.Strings; import java.util.HashMap; import java.util.Map; diff --git a/server/src/main/java/org/opensearch/cluster/node/DiscoveryNodes.java b/server/src/main/java/org/opensearch/cluster/node/DiscoveryNodes.java index 8e6f2045968a1..9c6e0ca9d82f4 100644 --- a/server/src/main/java/org/opensearch/cluster/node/DiscoveryNodes.java +++ b/server/src/main/java/org/opensearch/cluster/node/DiscoveryNodes.java @@ -38,12 +38,12 @@ import org.opensearch.cluster.Diff; import org.opensearch.common.Booleans; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.regex.Regex; import org.opensearch.common.transport.TransportAddress; import org.opensearch.common.util.set.Sets; +import org.opensearch.core.common.Strings; import java.io.IOException; import java.util.ArrayList; diff --git a/server/src/main/java/org/opensearch/cluster/routing/OperationRouting.java b/server/src/main/java/org/opensearch/cluster/routing/OperationRouting.java index 2f8d1387f32b4..d83ed6ce3b827 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/OperationRouting.java +++ b/server/src/main/java/org/opensearch/cluster/routing/OperationRouting.java @@ -38,10 +38,10 @@ import org.opensearch.cluster.node.DiscoveryNodes; import org.opensearch.cluster.routing.allocation.decider.AwarenessAllocationDecider; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; import org.opensearch.index.IndexModule; import org.opensearch.index.IndexNotFoundException; import org.opensearch.index.shard.ShardId; diff --git a/server/src/main/java/org/opensearch/cluster/routing/UnassignedInfo.java b/server/src/main/java/org/opensearch/cluster/routing/UnassignedInfo.java index 64c1c28fe63a0..117cadd648438 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/UnassignedInfo.java +++ b/server/src/main/java/org/opensearch/cluster/routing/UnassignedInfo.java @@ -32,7 +32,7 @@ package org.opensearch.cluster.routing; -import org.opensearch.ExceptionsHelper; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.LegacyESVersion; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.Metadata; @@ -401,7 +401,7 @@ public String getDetails() { if (message == null) { return null; } - return message + (failure == null ? "" : ", failure " + ExceptionsHelper.detailedMessage(failure)); + return message + (failure == null ? "" : ", failure " + BaseExceptionsHelper.detailedMessage(failure)); } /** diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/DiskThresholdMonitor.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/DiskThresholdMonitor.java index b4b0478d71d6f..e6e7034b76627 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/DiskThresholdMonitor.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/DiskThresholdMonitor.java @@ -51,11 +51,11 @@ import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.allocation.decider.DiskThresholdDecider; import org.opensearch.common.Priority; -import org.opensearch.common.Strings; import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.set.Sets; +import org.opensearch.core.common.Strings; import java.util.ArrayList; import java.util.HashSet; diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/FailedShard.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/FailedShard.java index 61ca0f50e93b9..d08dbba3a249d 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/FailedShard.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/FailedShard.java @@ -32,7 +32,7 @@ package org.opensearch.cluster.routing.allocation; -import org.opensearch.ExceptionsHelper; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.common.Nullable; @@ -62,7 +62,7 @@ public String toString() { + ", message [" + message + "], failure [" - + (failure == null ? "null" : ExceptionsHelper.detailedMessage(failure)) + + (failure == null ? "null" : BaseExceptionsHelper.detailedMessage(failure)) + "], markAsStale [" + markAsStale + "]"; diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/NodeAllocationResult.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/NodeAllocationResult.java index d50a474838a6b..ab1d56cbf7a9e 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/NodeAllocationResult.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/NodeAllocationResult.java @@ -32,7 +32,7 @@ package org.opensearch.cluster.routing.allocation; -import org.opensearch.OpenSearchException; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.routing.allocation.decider.Decision; import org.opensearch.common.Nullable; @@ -300,7 +300,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws } if (storeException != null) { builder.startObject("store_exception"); - OpenSearchException.generateThrowableXContent(builder, params, storeException); + BaseExceptionsHelper.generateThrowableXContent(builder, params, storeException); builder.endObject(); } } diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/AwarenessAllocationDecider.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/AwarenessAllocationDecider.java index 3d7ba09c839fc..811dd1249e37d 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/AwarenessAllocationDecider.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/AwarenessAllocationDecider.java @@ -45,11 +45,11 @@ import org.opensearch.cluster.routing.RoutingNode; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.allocation.RoutingAllocation; -import org.opensearch.common.Strings; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Setting.Property; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; import static java.util.Collections.emptyList; diff --git a/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/SameShardAllocationDecider.java b/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/SameShardAllocationDecider.java index 574005cd6b6f3..1680f2d8cad1d 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/SameShardAllocationDecider.java +++ b/server/src/main/java/org/opensearch/cluster/routing/allocation/decider/SameShardAllocationDecider.java @@ -35,11 +35,11 @@ import org.opensearch.cluster.routing.RoutingNode; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.allocation.RoutingAllocation; -import org.opensearch.common.Strings; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Setting.Property; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; /** * An allocation decider that prevents multiple instances of the same shard to diff --git a/server/src/main/java/org/opensearch/common/Strings.java b/server/src/main/java/org/opensearch/common/Strings.java index 64108928b8f41..0bec840a15f40 100644 --- a/server/src/main/java/org/opensearch/common/Strings.java +++ b/server/src/main/java/org/opensearch/common/Strings.java @@ -33,28 +33,18 @@ package org.opensearch.common; import org.apache.lucene.util.BytesRefBuilder; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.OpenSearchException; -import org.opensearch.ExceptionsHelper; import org.opensearch.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.BufferedReader; import java.io.IOException; -import java.io.StringReader; -import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; import java.util.Iterator; -import java.util.List; import java.util.Set; -import java.util.StringTokenizer; -import java.util.TreeSet; -import java.util.function.Supplier; import static java.util.Collections.unmodifiableSet; import static org.opensearch.common.util.set.Sets.newHashSet; @@ -66,206 +56,24 @@ */ public class Strings { - public static final String[] EMPTY_ARRAY = new String[0]; - - 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'); - } - } - } - - /** - * Splits a backslash escaped string on the separator. - *

- * Current backslash escaping supported: - *
\n \t \r \b \f are escaped the same as a Java String - *
Other characters following a backslash are produced verbatim (\c => c) - * - * @param s the string to split - * @param separator the separator to split on - * @param decode decode backslash escaping - */ - public static List splitSmart(String s, String separator, boolean decode) { - ArrayList lst = new ArrayList<>(2); - StringBuilder sb = new StringBuilder(); - int pos = 0, end = s.length(); - while (pos < end) { - if (s.startsWith(separator, pos)) { - if (sb.length() > 0) { - lst.add(sb.toString()); - sb = new StringBuilder(); - } - pos += separator.length(); - continue; - } - - char ch = s.charAt(pos++); - if (ch == '\\') { - if (!decode) sb.append(ch); - if (pos >= end) break; // ERROR, or let it go? - ch = s.charAt(pos++); - if (decode) { - switch (ch) { - case 'n': - ch = '\n'; - break; - case 't': - ch = '\t'; - break; - case 'r': - ch = '\r'; - break; - case 'b': - ch = '\b'; - break; - case 'f': - ch = '\f'; - break; - } - } - } - - sb.append(ch); - } - - if (sb.length() > 0) { - lst.add(sb.toString()); - } - - return lst; - } + public static final String[] EMPTY_ARRAY = org.opensearch.core.common.Strings.EMPTY_ARRAY; // --------------------------------------------------------------------- // General convenience methods for working with Strings // --------------------------------------------------------------------- - /** - * Check that the given CharSequence is neither null nor of length 0. - * Note: Will return true for a CharSequence that purely consists of whitespace. - *

-     * StringUtils.hasLength(null) = false
-     * StringUtils.hasLength("") = false
-     * StringUtils.hasLength(" ") = true
-     * StringUtils.hasLength("Hello") = true
-     * 
- * - * @param str the CharSequence to check (may be null) - * @return true if the CharSequence is not null and has length - * @see #hasText(String) - */ - 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 #hasLength(CharSequence) + * @see org.opensearch.core.common.Strings#hasLength(CharSequence) */ public static boolean hasLength(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. - * - * @param str the String to check (may be null) - * @return true if the String is not null and has length - * @see #hasLength(CharSequence) - */ - public static boolean hasLength(String str) { - return hasLength((CharSequence) str); - } - - /** - * Check that the given CharSequence is either null or of length 0. - * Note: Will return false for a CharSequence that purely consists of whitespace. - *
-     * StringUtils.isEmpty(null) = true
-     * StringUtils.isEmpty("") = true
-     * StringUtils.isEmpty(" ") = false
-     * StringUtils.isEmpty("Hello") = false
-     * 
- * - * @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) { - return !hasLength(str); - } - - /** - * Check whether the given CharSequence has actual text. - * More specifically, returns true if the string not null, - * its length is greater than 0, and it contains at least one non-whitespace character. - *
-     * StringUtils.hasText(null) = false
-     * StringUtils.hasText("") = false
-     * StringUtils.hasText(" ") = false
-     * StringUtils.hasText("12345") = true
-     * StringUtils.hasText(" 12345 ") = true
-     * 
- * - * @param str the CharSequence to check (may be null) - * @return true if the CharSequence is not null, - * its length is greater than 0, and it does not contain whitespace only - * @see java.lang.Character#isWhitespace - */ - public static boolean hasText(CharSequence str) { - if (!hasLength(str)) { - return false; - } - int strLen = str.length(); - for (int i = 0; i < strLen; i++) { - if (!Character.isWhitespace(str.charAt(i))) { - return true; - } - } - return false; - } - - /** - * Check whether the given String has actual text. - * More specifically, returns true if the string not null, - * its length is greater than 0, and it contains at least one non-whitespace character. - * - * @param str the String to check (may be null) - * @return true if the String is not null, its length is - * greater than 0, and it does not contain whitespace only - * @see #hasText(CharSequence) - */ - 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)) { - 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. @@ -284,68 +92,6 @@ public static boolean substringMatch(CharSequence str, int index, CharSequence s return true; } - /** - * Replace all occurrences of a substring within a string with - * another string. - * - * @param inString String to examine - * @param oldPattern String to replace - * @param newPattern String to insert - * @return a String with the replacements - */ - public static String replace(String inString, String oldPattern, String newPattern) { - if (!hasLength(inString) || !hasLength(oldPattern) || newPattern == null) { - return inString; - } - StringBuilder sb = new StringBuilder(); - int pos = 0; // our position in the old string - int index = inString.indexOf(oldPattern); - // the index of an occurrence we've found, or -1 - int patLen = oldPattern.length(); - while (index >= 0) { - sb.append(inString.substring(pos, index)); - sb.append(newPattern); - pos = index + patLen; - index = inString.indexOf(oldPattern, pos); - } - sb.append(inString.substring(pos)); - // remember to append any characters to the right of a match - return sb.toString(); - } - - /** - * Delete all occurrences of the given substring. - * - * @param inString the original String - * @param pattern the pattern to delete all occurrences of - * @return the resulting String - */ - public static String delete(String inString, String pattern) { - return replace(inString, pattern, ""); - } - - /** - * Delete any character in a given String. - * - * @param inString the original String - * @param charsToDelete a set of characters to delete. - * E.g. "az\n" will delete 'a's, 'z's and new lines. - * @return the resulting String - */ - public static String deleteAny(String inString, String charsToDelete) { - if (!hasLength(inString) || !hasLength(charsToDelete)) { - return inString; - } - StringBuilder sb = new StringBuilder(); - for (int i = 0; i < inString.length(); i++) { - char c = inString.charAt(i); - if (charsToDelete.indexOf(c) == -1) { - sb.append(c); - } - } - return sb.toString(); - } - // --------------------------------------------------------------------- // Convenience methods for working with formatted Strings // --------------------------------------------------------------------- @@ -361,32 +107,6 @@ 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 final Set INVALID_FILENAME_CHARS = unmodifiableSet( newHashSet('\\', '/', '*', '?', '"', '<', '>', '|', ' ', ',') ); @@ -411,44 +131,6 @@ public static boolean validFileNameExcludingAstrix(String fileName) { return true; } - /** - * Copy the given Collection into a String array. - * The Collection must contain String elements only. - * - * @param collection the Collection to copy - * @return the String array (null if the passed-in - * Collection was null) - */ - public static String[] toStringArray(Collection collection) { - if (collection == null) { - return null; - } - return collection.toArray(new String[collection.size()]); - } - - /** - * Tokenize the specified string by commas to a set, trimming whitespace and ignoring empty tokens. - * - * @param s the string to tokenize - * @return the set of tokens - */ - public static Set tokenizeByCommaToSet(final String s) { - if (s == null) return Collections.emptySet(); - 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. @@ -460,7 +142,8 @@ public static String[] splitStringByCommaToArray(final String s) { * or null if the delimiter wasn't found in the given input String */ public static String[] split(String toSplit, String delimiter) { - if (!hasLength(toSplit) || !hasLength(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); @@ -472,226 +155,6 @@ public static String[] split(String toSplit, String delimiter) { return new String[] { beforeDelimiter, afterDelimiter }; } - /** - * Tokenize the given String into a String array via a StringTokenizer. - * Trims tokens and omits empty tokens. - *

The given delimiters string is supposed to consist of any number of - * delimiter characters. Each of those characters can be used to separate - * tokens. A delimiter is always a single character; for multi-character - * delimiters, consider using delimitedListToStringArray - * - * @param s the String to tokenize - * @param delimiters the delimiter characters, assembled as String - * (each of those characters is individually considered as delimiter). - * @return an array of the tokens - * @see java.util.StringTokenizer - * @see java.lang.String#trim() - * @see #delimitedListToStringArray - */ - public static String[] tokenizeToStringArray(final String s, final String delimiters) { - if (s == null) { - return EMPTY_ARRAY; - } - return toStringArray(tokenizeToCollection(s, delimiters, ArrayList::new)); - } - - /** - * Tokenizes the specified string to a collection using the specified delimiters as the token delimiters. This method trims whitespace - * from tokens and ignores empty tokens. - * - * @param s the string to tokenize. - * @param delimiters the token delimiters - * @param supplier a collection supplier - * @param the type of the collection - * @return the tokens - * @see java.util.StringTokenizer - */ - private static > T tokenizeToCollection( - final String s, - final String delimiters, - final Supplier supplier - ) { - if (s == null) { - return null; - } - final StringTokenizer tokenizer = new StringTokenizer(s, delimiters); - final T tokens = supplier.get(); - while (tokenizer.hasMoreTokens()) { - final String token = tokenizer.nextToken().trim(); - if (token.length() > 0) { - tokens.add(token); - } - } - 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) - * @return an array of the tokens in the list - * @see #tokenizeToStringArray - */ - public static String[] delimitedListToStringArray(String str, String delimiter) { - return delimitedListToStringArray(str, delimiter, null); - } - - /** - * 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 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); - } - - /** - * Convert a CSV list into an array of Strings. - * - * @param str the input String - * @return an array of Strings, or the empty array in case of empty input - */ - public static String[] commaDelimitedListToStringArray(String str) { - return delimitedListToStringArray(str, ","); - } - - /** - * Convenience method to convert a CSV string list to a set. - * Note that this will suppress duplicates. - * - * @param str the input String - * @return a Set of String entries in the list - */ - public static Set commaDelimitedListToSet(String str) { - Set set = new TreeSet<>(); - String[] tokens = commaDelimitedListToStringArray(str); - set.addAll(Arrays.asList(tokens)); - return set; - } - - /** - * 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); - } - } - } - - /** - * 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'. */ @@ -715,16 +178,6 @@ 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(Object[] array) { - return (array == null || array.length == 0); - } - private Strings() {} public static byte[] toUTF8Bytes(CharSequence charSequence) { @@ -827,7 +280,7 @@ private static String toString(MediaType mediaType, ToXContent toXContent, ToXCo 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.field("stack_trace", BaseExceptionsHelper.stackTrace(e)); builder.endObject(); return toString(builder); } catch (IOException e2) { @@ -871,14 +324,6 @@ public static String cleanTruncate(String s, int length) { return s.substring(0, length); } - public static boolean isNullOrEmpty(@Nullable String s) { - return s == null || s.isEmpty(); - } - - public static String coalesceToEmpty(@Nullable String s) { - return s == null ? "" : s; - } - public static String padStart(String s, int minimumLength, char c) { if (s == null) { throw new NullPointerException("s"); diff --git a/server/src/main/java/org/opensearch/common/Table.java b/server/src/main/java/org/opensearch/common/Table.java index 9c3462e7077cd..da14f628efa0f 100644 --- a/server/src/main/java/org/opensearch/common/Table.java +++ b/server/src/main/java/org/opensearch/common/Table.java @@ -33,6 +33,7 @@ package org.opensearch.common; import org.opensearch.common.time.DateFormatter; +import org.opensearch.core.common.Strings; import java.time.Instant; import java.time.ZoneOffset; diff --git a/server/src/main/java/org/opensearch/common/io/stream/NotSerializableExceptionWrapper.java b/server/src/main/java/org/opensearch/common/io/stream/NotSerializableExceptionWrapper.java index f642dd17ad688..636df65b8c319 100644 --- a/server/src/main/java/org/opensearch/common/io/stream/NotSerializableExceptionWrapper.java +++ b/server/src/main/java/org/opensearch/common/io/stream/NotSerializableExceptionWrapper.java @@ -32,6 +32,7 @@ package org.opensearch.common.io.stream; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.OpenSearchException; import org.opensearch.ExceptionsHelper; import org.opensearch.rest.RestStatus; @@ -53,8 +54,8 @@ public final class NotSerializableExceptionWrapper extends OpenSearchException { private final RestStatus status; public NotSerializableExceptionWrapper(Throwable other) { - super(OpenSearchException.getExceptionName(other) + ": " + other.getMessage(), other.getCause()); - this.name = OpenSearchException.getExceptionName(other); + super(BaseExceptionsHelper.getExceptionName(other) + ": " + other.getMessage(), other.getCause()); + this.name = BaseExceptionsHelper.getExceptionName(other); this.status = ExceptionsHelper.status(other); setStackTrace(other.getStackTrace()); for (Throwable otherSuppressed : other.getSuppressed()) { diff --git a/server/src/main/java/org/opensearch/common/io/stream/StreamInput.java b/server/src/main/java/org/opensearch/common/io/stream/StreamInput.java index 2b51d6c469fcf..1deeb4c60d469 100644 --- a/server/src/main/java/org/opensearch/common/io/stream/StreamInput.java +++ b/server/src/main/java/org/opensearch/common/io/stream/StreamInput.java @@ -47,12 +47,12 @@ import org.opensearch.Version; import org.opensearch.common.CharArrays; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.bytes.BytesArray; import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.settings.SecureString; import org.opensearch.common.text.Text; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.BaseStreamInput; import org.opensearch.core.common.io.stream.BaseWriteable; import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; diff --git a/server/src/main/java/org/opensearch/common/joda/Joda.java b/server/src/main/java/org/opensearch/common/joda/Joda.java index 9ecb3f2236e7c..2691c1e6cf0e9 100644 --- a/server/src/main/java/org/opensearch/common/joda/Joda.java +++ b/server/src/main/java/org/opensearch/common/joda/Joda.java @@ -34,11 +34,11 @@ import org.opensearch.LegacyESVersion; import org.opensearch.Version; -import org.opensearch.common.Strings; import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.time.DateFormatter; import org.opensearch.common.time.FormatNames; import org.opensearch.common.util.LazyInitializable; +import org.opensearch.core.common.Strings; import org.joda.time.Chronology; import org.joda.time.DateTime; import org.joda.time.DateTimeField; diff --git a/server/src/main/java/org/opensearch/common/logging/DeprecatedMessage.java b/server/src/main/java/org/opensearch/common/logging/DeprecatedMessage.java index 8905fb6f391a2..e5843d9333ab5 100644 --- a/server/src/main/java/org/opensearch/common/logging/DeprecatedMessage.java +++ b/server/src/main/java/org/opensearch/common/logging/DeprecatedMessage.java @@ -34,8 +34,8 @@ import java.util.Map; -import org.opensearch.common.Strings; import org.opensearch.common.collect.MapBuilder; +import org.opensearch.core.common.Strings; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; diff --git a/server/src/main/java/org/opensearch/common/logging/HeaderWarning.java b/server/src/main/java/org/opensearch/common/logging/HeaderWarning.java index df5a94ccd34e4..103033ccac312 100644 --- a/server/src/main/java/org/opensearch/common/logging/HeaderWarning.java +++ b/server/src/main/java/org/opensearch/common/logging/HeaderWarning.java @@ -35,6 +35,7 @@ import org.opensearch.Build; import org.opensearch.Version; import org.opensearch.common.util.concurrent.ThreadContext; +import org.opensearch.core.common.logging.LoggerMessageFormat; import org.opensearch.tasks.Task; import java.nio.charset.Charset; diff --git a/server/src/main/java/org/opensearch/common/logging/OpenSearchJsonLayout.java b/server/src/main/java/org/opensearch/common/logging/OpenSearchJsonLayout.java index dac9e299e9b7e..5896aec0ce71c 100644 --- a/server/src/main/java/org/opensearch/common/logging/OpenSearchJsonLayout.java +++ b/server/src/main/java/org/opensearch/common/logging/OpenSearchJsonLayout.java @@ -42,7 +42,7 @@ import org.apache.logging.log4j.core.layout.AbstractStringLayout; import org.apache.logging.log4j.core.layout.ByteBufferDestination; import org.apache.logging.log4j.core.layout.PatternLayout; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import java.nio.charset.Charset; import java.nio.charset.StandardCharsets; diff --git a/server/src/main/java/org/opensearch/common/logging/OpenSearchMessageFieldConverter.java b/server/src/main/java/org/opensearch/common/logging/OpenSearchMessageFieldConverter.java index fc9e523d00dd1..596f8716c1103 100644 --- a/server/src/main/java/org/opensearch/common/logging/OpenSearchMessageFieldConverter.java +++ b/server/src/main/java/org/opensearch/common/logging/OpenSearchMessageFieldConverter.java @@ -39,7 +39,7 @@ import org.apache.logging.log4j.core.pattern.LogEventPatternConverter; import org.apache.logging.log4j.core.pattern.PatternConverter; import org.apache.logging.log4j.util.StringBuilders; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; /** * Pattern converter to populate OpenSearchMessageField in a pattern. diff --git a/server/src/main/java/org/opensearch/common/lucene/Lucene.java b/server/src/main/java/org/opensearch/common/lucene/Lucene.java index d3f93bc8a976e..fb3a27ee0fef2 100644 --- a/server/src/main/java/org/opensearch/common/lucene/Lucene.java +++ b/server/src/main/java/org/opensearch/common/lucene/Lucene.java @@ -102,12 +102,12 @@ import org.opensearch.ExceptionsHelper; import org.opensearch.LegacyESVersion; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.SuppressForbidden; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.lucene.search.TopDocsAndMaxScore; import org.opensearch.common.util.iterable.Iterables; +import org.opensearch.core.common.Strings; import org.opensearch.index.analysis.AnalyzerScope; import org.opensearch.index.analysis.NamedAnalyzer; import org.opensearch.index.fielddata.IndexFieldData; diff --git a/server/src/main/java/org/opensearch/common/lucene/search/MoreLikeThisQuery.java b/server/src/main/java/org/opensearch/common/lucene/search/MoreLikeThisQuery.java index 93d18b3533d5c..b4440b85e0037 100644 --- a/server/src/main/java/org/opensearch/common/lucene/search/MoreLikeThisQuery.java +++ b/server/src/main/java/org/opensearch/common/lucene/search/MoreLikeThisQuery.java @@ -48,7 +48,7 @@ import org.apache.lucene.search.similarities.Similarity; import org.apache.lucene.search.similarities.TFIDFSimilarity; import org.apache.lucene.util.BytesRef; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import java.io.IOException; import java.io.Reader; diff --git a/server/src/main/java/org/opensearch/common/lucene/search/Queries.java b/server/src/main/java/org/opensearch/common/lucene/search/Queries.java index 8b64a45b9db25..fdb4753650531 100644 --- a/server/src/main/java/org/opensearch/common/lucene/search/Queries.java +++ b/server/src/main/java/org/opensearch/common/lucene/search/Queries.java @@ -47,7 +47,7 @@ import org.apache.lucene.search.ScoreMode; import org.apache.lucene.search.Scorer; import org.apache.lucene.search.Weight; -import org.opensearch.OpenSearchException; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.common.Nullable; import org.opensearch.index.mapper.SeqNoFieldMapper; @@ -83,7 +83,7 @@ public static Query newUnmappedFieldsQuery(Collection fields) { } public static Query newLenientFieldQuery(String field, RuntimeException e) { - String message = OpenSearchException.getExceptionName(e) + ":[" + e.getMessage() + "]"; + String message = BaseExceptionsHelper.getExceptionName(e) + ":[" + e.getMessage() + "]"; return Queries.newMatchNoDocsQuery("failed [" + field + "] query, caused by " + message); } diff --git a/server/src/main/java/org/opensearch/common/lucene/search/function/Functions.java b/server/src/main/java/org/opensearch/common/lucene/search/function/Functions.java index ce733bbe78684..a5f4f8ce465b3 100644 --- a/server/src/main/java/org/opensearch/common/lucene/search/function/Functions.java +++ b/server/src/main/java/org/opensearch/common/lucene/search/function/Functions.java @@ -9,7 +9,7 @@ package org.opensearch.common.lucene.search.function; import org.apache.lucene.search.Explanation; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.index.query.AbstractQueryBuilder; import org.opensearch.index.query.functionscore.FunctionScoreQueryBuilder; @@ -27,7 +27,7 @@ private Functions() {} * @return function name wrapped into brackets or empty string */ public static String nameOrEmptyFunc(final String functionName) { - if (!Strings.isNullOrEmpty(functionName)) { + if (Strings.isNullOrEmpty(functionName) == false) { return "(" + AbstractQueryBuilder.NAME_FIELD.getPreferredName() + ": " + functionName + ")"; } else { return ""; @@ -40,7 +40,7 @@ public static String nameOrEmptyFunc(final String functionName) { * @return function name as an argument or empty string */ public static String nameOrEmptyArg(final String functionName) { - if (!Strings.isNullOrEmpty(functionName)) { + if (Strings.isNullOrEmpty(functionName) == false) { return ", " + FunctionScoreQueryBuilder.NAME_FIELD.getPreferredName() + ": " + functionName; } else { return ""; diff --git a/server/src/main/java/org/opensearch/common/lucene/search/function/WeightFactorFunction.java b/server/src/main/java/org/opensearch/common/lucene/search/function/WeightFactorFunction.java index c439b57de41cd..84ef6e308092e 100644 --- a/server/src/main/java/org/opensearch/common/lucene/search/function/WeightFactorFunction.java +++ b/server/src/main/java/org/opensearch/common/lucene/search/function/WeightFactorFunction.java @@ -35,7 +35,7 @@ import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.search.Explanation; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import java.io.IOException; import java.util.Objects; 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 2445f80cf7e28..14716fdff6d2e 100644 --- a/server/src/main/java/org/opensearch/common/regex/Regex.java +++ b/server/src/main/java/org/opensearch/common/regex/Regex.java @@ -35,7 +35,7 @@ import org.apache.lucene.util.automaton.Automata; import org.apache.lucene.util.automaton.Automaton; import org.apache.lucene.util.automaton.Operations; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import java.util.ArrayList; import java.util.List; @@ -122,8 +122,8 @@ public static boolean simpleMatch(String pattern, String str, boolean caseInsens return false; } if (caseInsensitive) { - pattern = Strings.toLowercaseAscii(pattern); - str = Strings.toLowercaseAscii(str); + pattern = org.opensearch.common.Strings.toLowercaseAscii(pattern); + str = org.opensearch.common.Strings.toLowercaseAscii(str); } return simpleMatchWithNormalizedStrings(pattern, str); } diff --git a/server/src/main/java/org/opensearch/common/settings/Settings.java b/server/src/main/java/org/opensearch/common/settings/Settings.java index cc7fbd7ebf88f..efaa07c28dc7d 100644 --- a/server/src/main/java/org/opensearch/common/settings/Settings.java +++ b/server/src/main/java/org/opensearch/common/settings/Settings.java @@ -425,7 +425,7 @@ public List getAsList(String key, List defaultValue, Boolean com if (valueFromPrefix instanceof List) { return Collections.unmodifiableList((List) valueFromPrefix); } else if (commaDelimited) { - String[] strings = Strings.splitStringByCommaToArray(get(key)); + String[] strings = org.opensearch.core.common.Strings.splitStringByCommaToArray(get(key)); if (strings.length > 0) { for (String string : strings) { result.add(string.trim()); @@ -453,7 +453,7 @@ public Map getGroups(String settingPrefix) throws SettingsExce * Returns group settings for the given setting prefix. */ public Map getGroups(String settingPrefix, boolean ignoreNonGrouped) throws SettingsException { - if (!Strings.hasLength(settingPrefix)) { + if (!org.opensearch.core.common.Strings.hasLength(settingPrefix)) { throw new IllegalArgumentException("illegal setting prefix " + settingPrefix); } if (settingPrefix.charAt(settingPrefix.length() - 1) != '.') { @@ -1206,7 +1206,7 @@ public boolean shouldRemoveMissingPlaceholder(String placeholderName) { String value = propertyPlaceholder.replacePlaceholders(Settings.toString(entry.getValue()), placeholderResolver); // if the values exists and has length, we should maintain it in the map // otherwise, the replace process resolved into removing it - if (Strings.hasLength(value)) { + if (org.opensearch.core.common.Strings.hasLength(value)) { entry.setValue(value); } else { entryItr.remove(); diff --git a/server/src/main/java/org/opensearch/common/settings/SettingsFilter.java b/server/src/main/java/org/opensearch/common/settings/SettingsFilter.java index 2bca22f445f62..1079a02f51c55 100644 --- a/server/src/main/java/org/opensearch/common/settings/SettingsFilter.java +++ b/server/src/main/java/org/opensearch/common/settings/SettingsFilter.java @@ -32,8 +32,8 @@ package org.opensearch.common.settings; -import org.opensearch.common.Strings; import org.opensearch.common.regex.Regex; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ToXContent.Params; import org.opensearch.rest.RestRequest; diff --git a/server/src/main/java/org/opensearch/common/time/DateFormatter.java b/server/src/main/java/org/opensearch/common/time/DateFormatter.java index dec08b68644f2..57fee6a33bdb6 100644 --- a/server/src/main/java/org/opensearch/common/time/DateFormatter.java +++ b/server/src/main/java/org/opensearch/common/time/DateFormatter.java @@ -32,8 +32,8 @@ package org.opensearch.common.time; -import org.opensearch.common.Strings; import org.joda.time.DateTime; +import org.opensearch.core.common.Strings; import java.time.Instant; import java.time.ZoneId; diff --git a/server/src/main/java/org/opensearch/common/time/DateFormatters.java b/server/src/main/java/org/opensearch/common/time/DateFormatters.java index 26af6fc40429d..6c8b9282d8a77 100644 --- a/server/src/main/java/org/opensearch/common/time/DateFormatters.java +++ b/server/src/main/java/org/opensearch/common/time/DateFormatters.java @@ -32,10 +32,10 @@ package org.opensearch.common.time; -import org.opensearch.common.Strings; import org.opensearch.common.SuppressForbidden; import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.util.LazyInitializable; +import org.opensearch.core.common.Strings; import java.time.DayOfWeek; import java.time.Instant; diff --git a/server/src/main/java/org/opensearch/common/time/JavaDateFormatter.java b/server/src/main/java/org/opensearch/common/time/JavaDateFormatter.java index 07ea806aa6b8d..07013a3dc75f2 100644 --- a/server/src/main/java/org/opensearch/common/time/JavaDateFormatter.java +++ b/server/src/main/java/org/opensearch/common/time/JavaDateFormatter.java @@ -32,7 +32,7 @@ package org.opensearch.common.time; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import java.text.ParsePosition; import java.time.ZoneId; diff --git a/server/src/main/java/org/opensearch/common/time/JavaDateMathParser.java b/server/src/main/java/org/opensearch/common/time/JavaDateMathParser.java index 1110d07693c85..0536324b6516b 100644 --- a/server/src/main/java/org/opensearch/common/time/JavaDateMathParser.java +++ b/server/src/main/java/org/opensearch/common/time/JavaDateMathParser.java @@ -33,7 +33,7 @@ package org.opensearch.common.time; import org.opensearch.OpenSearchParseException; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import java.time.DayOfWeek; import java.time.Instant; diff --git a/server/src/main/java/org/opensearch/common/util/CollectionUtils.java b/server/src/main/java/org/opensearch/common/util/CollectionUtils.java index 85d897af7479c..2037b2e46488f 100644 --- a/server/src/main/java/org/opensearch/common/util/CollectionUtils.java +++ b/server/src/main/java/org/opensearch/common/util/CollectionUtils.java @@ -32,8 +32,8 @@ package org.opensearch.common.util; -import org.opensearch.common.Strings; import org.opensearch.common.collect.Iterators; +import org.opensearch.core.common.Strings; import java.nio.file.Path; import java.util.AbstractList; diff --git a/server/src/main/java/org/opensearch/common/xcontent/XContentParserUtils.java b/server/src/main/java/org/opensearch/common/xcontent/XContentParserUtils.java index 3dd9cb8863174..5a91d9c2c1394 100644 --- a/server/src/main/java/org/opensearch/common/xcontent/XContentParserUtils.java +++ b/server/src/main/java/org/opensearch/common/xcontent/XContentParserUtils.java @@ -33,8 +33,8 @@ package org.opensearch.common.xcontent; import org.opensearch.common.ParsingException; -import org.opensearch.common.Strings; import org.opensearch.common.bytes.BytesArray; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.core.xcontent.XContentParser.Token; diff --git a/server/src/main/java/org/opensearch/common/xcontent/support/XContentMapValues.java b/server/src/main/java/org/opensearch/common/xcontent/support/XContentMapValues.java index 9170cfaf8eadf..adfa871cbfcbe 100644 --- a/server/src/main/java/org/opensearch/common/xcontent/support/XContentMapValues.java +++ b/server/src/main/java/org/opensearch/common/xcontent/support/XContentMapValues.java @@ -39,9 +39,9 @@ import org.opensearch.OpenSearchParseException; import org.opensearch.common.Booleans; import org.opensearch.common.Numbers; -import org.opensearch.common.Strings; import org.opensearch.common.regex.Regex; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import java.util.ArrayList; import java.util.Arrays; diff --git a/server/src/main/java/org/opensearch/extensions/ExtensionsManager.java b/server/src/main/java/org/opensearch/extensions/ExtensionsManager.java index 76a58ed9ed1b9..827a4a6b6b72d 100644 --- a/server/src/main/java/org/opensearch/extensions/ExtensionsManager.java +++ b/server/src/main/java/org/opensearch/extensions/ExtensionsManager.java @@ -29,12 +29,12 @@ import org.opensearch.cluster.ClusterSettingsResponse; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.settings.Setting; -import org.opensearch.common.Strings; import org.opensearch.common.util.concurrent.AbstractRunnable; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.settings.Settings; import org.opensearch.common.settings.SettingsModule; import org.opensearch.common.transport.TransportAddress; +import org.opensearch.core.common.Strings; import org.opensearch.discovery.InitializeExtensionRequest; import org.opensearch.discovery.InitializeExtensionResponse; diff --git a/server/src/main/java/org/opensearch/extensions/rest/RestInitializeExtensionAction.java b/server/src/main/java/org/opensearch/extensions/rest/RestInitializeExtensionAction.java index 2c593e2717762..f47f342617732 100644 --- a/server/src/main/java/org/opensearch/extensions/rest/RestInitializeExtensionAction.java +++ b/server/src/main/java/org/opensearch/extensions/rest/RestInitializeExtensionAction.java @@ -10,10 +10,10 @@ import org.opensearch.Version; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; import org.opensearch.common.collect.Tuple; import org.opensearch.common.settings.Settings; import org.opensearch.common.xcontent.XContentHelper; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.MediaType; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.extensions.ExtensionDependency; diff --git a/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java b/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java index dcc25faa18bc1..35272d9f54dc6 100644 --- a/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java +++ b/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java @@ -33,8 +33,8 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.OpenSearchTimeoutException; -import org.opensearch.ExceptionsHelper; import org.opensearch.action.ActionListener; import org.opensearch.action.FailedNodeException; import org.opensearch.action.support.nodes.BaseNodeResponse; @@ -251,7 +251,7 @@ protected synchronized void processAsyncFetch(List responses, Listtrue if the index has a custom data path */ public boolean hasCustomDataPath() { - return !Strings.isEmpty(customDataPath()); + return Strings.isEmpty(customDataPath()) == false; } /** diff --git a/server/src/main/java/org/opensearch/index/SearchSlowLog.java b/server/src/main/java/org/opensearch/index/SearchSlowLog.java index 687e2884dcfbe..0b2e3a6b7cbc3 100644 --- a/server/src/main/java/org/opensearch/index/SearchSlowLog.java +++ b/server/src/main/java/org/opensearch/index/SearchSlowLog.java @@ -35,12 +35,12 @@ import com.fasterxml.jackson.core.io.JsonStringEncoder; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -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.core.common.Strings; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.index.shard.SearchOperationListener; import org.opensearch.search.internal.SearchContext; diff --git a/server/src/main/java/org/opensearch/index/analysis/Analysis.java b/server/src/main/java/org/opensearch/index/analysis/Analysis.java index c99ff2f2e0738..0062e4d8fbe05 100644 --- a/server/src/main/java/org/opensearch/index/analysis/Analysis.java +++ b/server/src/main/java/org/opensearch/index/analysis/Analysis.java @@ -69,8 +69,8 @@ import org.apache.lucene.analysis.sv.SwedishAnalyzer; import org.apache.lucene.analysis.th.ThaiAnalyzer; import org.apache.lucene.analysis.tr.TurkishAnalyzer; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; import org.opensearch.env.Environment; import java.io.BufferedReader; diff --git a/server/src/main/java/org/opensearch/index/engine/InternalEngine.java b/server/src/main/java/org/opensearch/index/engine/InternalEngine.java index aad1fbc319341..13f014a06042f 100644 --- a/server/src/main/java/org/opensearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/opensearch/index/engine/InternalEngine.java @@ -66,8 +66,8 @@ import org.apache.lucene.store.LockObtainFailedException; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.InfoStream; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.core.Assertions; -import org.opensearch.ExceptionsHelper; import org.opensearch.action.index.IndexRequest; import org.opensearch.common.Booleans; import org.opensearch.common.Nullable; @@ -313,7 +313,7 @@ public void onFailure(String reason, Exception ex) { } catch (AssertionError e) { // IndexWriter throws AssertionError on init, if asserts are enabled, if any files don't exist, but tests that // randomly throw FNFE/NSFE can also hit this: - if (ExceptionsHelper.stackTrace(e).contains("org.apache.lucene.index.IndexWriter.filesExist")) { + if (BaseExceptionsHelper.stackTrace(e).contains("org.apache.lucene.index.IndexWriter.filesExist")) { throw new EngineCreationFailureException(shardId, "failed to create engine", e); } else { throw e; @@ -2694,7 +2694,7 @@ protected void commitIndexWriter(final IndexWriter writer, final Translog transl * If assertions are enabled, IndexWriter throws AssertionError on commit if any files don't exist, but tests that randomly * throw FileNotFoundException or NoSuchFileException can also hit this. */ - if (ExceptionsHelper.stackTrace(e).contains("org.apache.lucene.index.IndexWriter.filesExist")) { + if (BaseExceptionsHelper.stackTrace(e).contains("org.apache.lucene.index.IndexWriter.filesExist")) { final EngineException engineException = new EngineException(shardId, "failed to commit engine", e); try { failEngine("lucene commit failed", engineException); diff --git a/server/src/main/java/org/opensearch/index/mapper/DocumentParser.java b/server/src/main/java/org/opensearch/index/mapper/DocumentParser.java index 2acffdee49a3a..c5ee5be491095 100644 --- a/server/src/main/java/org/opensearch/index/mapper/DocumentParser.java +++ b/server/src/main/java/org/opensearch/index/mapper/DocumentParser.java @@ -37,12 +37,12 @@ import org.opensearch.OpenSearchParseException; import org.opensearch.Version; import org.opensearch.cluster.metadata.IndexMetadata; -import org.opensearch.common.Strings; import org.opensearch.common.collect.Tuple; import org.opensearch.common.settings.Settings; import org.opensearch.common.time.DateFormatter; import org.opensearch.common.xcontent.LoggingDeprecationHandler; import org.opensearch.common.xcontent.XContentHelper; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.common.xcontent.XContentType; import org.opensearch.index.IndexSettings; 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 5ac254adfe9c6..4905d45adf189 100644 --- a/server/src/main/java/org/opensearch/index/mapper/SourceFieldMapper.java +++ b/server/src/main/java/org/opensearch/index/mapper/SourceFieldMapper.java @@ -39,16 +39,16 @@ import org.apache.lucene.search.Query; import org.apache.lucene.util.BytesRef; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.collect.Tuple; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.util.CollectionUtils; -import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.common.xcontent.XContentFactory; import org.opensearch.common.xcontent.XContentHelper; import org.opensearch.common.xcontent.XContentType; import org.opensearch.common.xcontent.support.XContentMapValues; +import org.opensearch.core.common.Strings; +import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.index.query.QueryShardContext; import org.opensearch.index.query.QueryShardException; import org.opensearch.search.lookup.SearchLookup; diff --git a/server/src/main/java/org/opensearch/index/query/BaseTermQueryBuilder.java b/server/src/main/java/org/opensearch/index/query/BaseTermQueryBuilder.java index f41ef5c734d6d..1a4a9ed185def 100644 --- a/server/src/main/java/org/opensearch/index/query/BaseTermQueryBuilder.java +++ b/server/src/main/java/org/opensearch/index/query/BaseTermQueryBuilder.java @@ -34,9 +34,9 @@ import org.apache.lucene.util.BytesRef; import org.opensearch.core.ParseField; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/index/query/CommonTermsQueryBuilder.java b/server/src/main/java/org/opensearch/index/query/CommonTermsQueryBuilder.java index 5ec1f17c34975..a22dd513c41b1 100644 --- a/server/src/main/java/org/opensearch/index/query/CommonTermsQueryBuilder.java +++ b/server/src/main/java/org/opensearch/index/query/CommonTermsQueryBuilder.java @@ -42,9 +42,9 @@ import org.apache.lucene.util.BytesRefBuilder; import org.opensearch.core.ParseField; import org.opensearch.common.ParsingException; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.index.mapper.MappedFieldType; diff --git a/server/src/main/java/org/opensearch/index/query/ExistsQueryBuilder.java b/server/src/main/java/org/opensearch/index/query/ExistsQueryBuilder.java index 62156d06c1a32..ad8a8da288a32 100644 --- a/server/src/main/java/org/opensearch/index/query/ExistsQueryBuilder.java +++ b/server/src/main/java/org/opensearch/index/query/ExistsQueryBuilder.java @@ -39,10 +39,10 @@ import org.apache.lucene.search.MatchNoDocsQuery; import org.apache.lucene.search.Query; import org.opensearch.common.ParsingException; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.lucene.search.Queries; +import org.opensearch.core.common.Strings; import org.opensearch.core.ParseField; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; diff --git a/server/src/main/java/org/opensearch/index/query/FieldMaskingSpanQueryBuilder.java b/server/src/main/java/org/opensearch/index/query/FieldMaskingSpanQueryBuilder.java index 80b5dc8070ff2..2ff41b5f66152 100644 --- a/server/src/main/java/org/opensearch/index/query/FieldMaskingSpanQueryBuilder.java +++ b/server/src/main/java/org/opensearch/index/query/FieldMaskingSpanQueryBuilder.java @@ -37,9 +37,9 @@ import org.apache.lucene.search.Query; import org.opensearch.core.ParseField; import org.opensearch.common.ParsingException; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.index.mapper.MappedFieldType; diff --git a/server/src/main/java/org/opensearch/index/query/FuzzyQueryBuilder.java b/server/src/main/java/org/opensearch/index/query/FuzzyQueryBuilder.java index f56bfba5c4ca5..aaedc0b92eb47 100644 --- a/server/src/main/java/org/opensearch/index/query/FuzzyQueryBuilder.java +++ b/server/src/main/java/org/opensearch/index/query/FuzzyQueryBuilder.java @@ -37,11 +37,11 @@ import org.apache.lucene.search.Query; import org.opensearch.core.ParseField; import org.opensearch.common.ParsingException; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.unit.Fuzziness; import org.opensearch.common.xcontent.LoggingDeprecationHandler; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.index.mapper.MappedFieldType; diff --git a/server/src/main/java/org/opensearch/index/query/GeoDistanceQueryBuilder.java b/server/src/main/java/org/opensearch/index/query/GeoDistanceQueryBuilder.java index bc52d8fe6a6df..b4d5a8b9ec2c9 100644 --- a/server/src/main/java/org/opensearch/index/query/GeoDistanceQueryBuilder.java +++ b/server/src/main/java/org/opensearch/index/query/GeoDistanceQueryBuilder.java @@ -36,7 +36,6 @@ import org.apache.lucene.search.Query; import org.opensearch.core.ParseField; import org.opensearch.common.ParsingException; -import org.opensearch.common.Strings; import org.opensearch.common.geo.GeoDistance; import org.opensearch.common.geo.GeoPoint; import org.opensearch.common.geo.GeoUtils; @@ -45,6 +44,7 @@ import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.unit.DistanceUnit; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.geometry.Circle; diff --git a/server/src/main/java/org/opensearch/index/query/GeoPolygonQueryBuilder.java b/server/src/main/java/org/opensearch/index/query/GeoPolygonQueryBuilder.java index f5e40fec78c25..c6348d3c2e7be 100644 --- a/server/src/main/java/org/opensearch/index/query/GeoPolygonQueryBuilder.java +++ b/server/src/main/java/org/opensearch/index/query/GeoPolygonQueryBuilder.java @@ -40,11 +40,11 @@ import org.apache.lucene.search.Query; import org.opensearch.core.ParseField; import org.opensearch.common.ParsingException; -import org.opensearch.common.Strings; import org.opensearch.common.geo.GeoPoint; import org.opensearch.common.geo.GeoUtils; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.core.xcontent.XContentParser.Token; diff --git a/server/src/main/java/org/opensearch/index/query/IdsQueryBuilder.java b/server/src/main/java/org/opensearch/index/query/IdsQueryBuilder.java index a77c6e930b3ec..0084077bbaafb 100644 --- a/server/src/main/java/org/opensearch/index/query/IdsQueryBuilder.java +++ b/server/src/main/java/org/opensearch/index/query/IdsQueryBuilder.java @@ -34,12 +34,12 @@ import org.apache.lucene.search.Query; import org.opensearch.Version; -import org.opensearch.core.ParseField; import org.opensearch.common.ParsingException; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.logging.DeprecationLogger; +import org.opensearch.core.common.Strings; +import org.opensearch.core.ParseField; import org.opensearch.core.xcontent.ObjectParser; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; diff --git a/server/src/main/java/org/opensearch/index/query/MatchBoolPrefixQueryBuilder.java b/server/src/main/java/org/opensearch/index/query/MatchBoolPrefixQueryBuilder.java index 7aebf15e780b9..651bcee042fb6 100644 --- a/server/src/main/java/org/opensearch/index/query/MatchBoolPrefixQueryBuilder.java +++ b/server/src/main/java/org/opensearch/index/query/MatchBoolPrefixQueryBuilder.java @@ -35,12 +35,12 @@ import org.apache.lucene.search.FuzzyQuery; import org.apache.lucene.search.Query; import org.opensearch.common.ParsingException; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.lucene.search.Queries; import org.opensearch.common.unit.Fuzziness; import org.opensearch.common.xcontent.LoggingDeprecationHandler; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.index.query.support.QueryParsers; diff --git a/server/src/main/java/org/opensearch/index/query/MatchPhraseQueryBuilder.java b/server/src/main/java/org/opensearch/index/query/MatchPhraseQueryBuilder.java index 5f42825e9d0f7..c584c8af82e60 100644 --- a/server/src/main/java/org/opensearch/index/query/MatchPhraseQueryBuilder.java +++ b/server/src/main/java/org/opensearch/index/query/MatchPhraseQueryBuilder.java @@ -35,9 +35,9 @@ import org.apache.lucene.search.Query; import org.opensearch.core.ParseField; import org.opensearch.common.ParsingException; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.index.search.MatchQuery; diff --git a/server/src/main/java/org/opensearch/index/query/MoreLikeThisQueryBuilder.java b/server/src/main/java/org/opensearch/index/query/MoreLikeThisQueryBuilder.java index 23411a64d80a7..abe352bb48966 100644 --- a/server/src/main/java/org/opensearch/index/query/MoreLikeThisQueryBuilder.java +++ b/server/src/main/java/org/opensearch/index/query/MoreLikeThisQueryBuilder.java @@ -37,6 +37,7 @@ import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.Query; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.OpenSearchParseException; import org.opensearch.ExceptionsHelper; import org.opensearch.Version; @@ -448,7 +449,7 @@ public String toString() { toXContent(builder, EMPTY_PARAMS); return Strings.toString(builder); } catch (Exception e) { - return "{ \"error\" : \"" + ExceptionsHelper.detailedMessage(e) + "\"}"; + return "{ \"error\" : \"" + BaseExceptionsHelper.detailedMessage(e) + "\"}"; } } diff --git a/server/src/main/java/org/opensearch/index/query/MultiMatchQueryBuilder.java b/server/src/main/java/org/opensearch/index/query/MultiMatchQueryBuilder.java index 9ac810f17649e..424e3f7e5ae1c 100644 --- a/server/src/main/java/org/opensearch/index/query/MultiMatchQueryBuilder.java +++ b/server/src/main/java/org/opensearch/index/query/MultiMatchQueryBuilder.java @@ -38,11 +38,11 @@ import org.opensearch.OpenSearchParseException; import org.opensearch.core.ParseField; import org.opensearch.common.ParsingException; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.io.stream.Writeable; import org.opensearch.common.unit.Fuzziness; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.DeprecationHandler; import org.opensearch.common.xcontent.LoggingDeprecationHandler; import org.opensearch.core.xcontent.XContentBuilder; diff --git a/server/src/main/java/org/opensearch/index/query/PrefixQueryBuilder.java b/server/src/main/java/org/opensearch/index/query/PrefixQueryBuilder.java index 7fb264495c67d..4c841b2143109 100644 --- a/server/src/main/java/org/opensearch/index/query/PrefixQueryBuilder.java +++ b/server/src/main/java/org/opensearch/index/query/PrefixQueryBuilder.java @@ -38,11 +38,11 @@ import org.apache.lucene.search.Query; import org.opensearch.LegacyESVersion; import org.opensearch.common.ParsingException; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.xcontent.LoggingDeprecationHandler; import org.opensearch.core.ParseField; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.index.mapper.MappedFieldType; diff --git a/server/src/main/java/org/opensearch/index/query/RangeQueryBuilder.java b/server/src/main/java/org/opensearch/index/query/RangeQueryBuilder.java index 3eb401987e0ba..8bd668b816ae9 100644 --- a/server/src/main/java/org/opensearch/index/query/RangeQueryBuilder.java +++ b/server/src/main/java/org/opensearch/index/query/RangeQueryBuilder.java @@ -37,12 +37,12 @@ import org.apache.lucene.util.BytesRef; import org.opensearch.core.ParseField; import org.opensearch.common.ParsingException; -import org.opensearch.common.Strings; import org.opensearch.common.geo.ShapeRelation; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.time.DateFormatter; import org.opensearch.common.time.DateMathParser; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.index.mapper.FieldNamesFieldMapper; diff --git a/server/src/main/java/org/opensearch/index/query/RegexpFlag.java b/server/src/main/java/org/opensearch/index/query/RegexpFlag.java index 926e00450a0c2..b7e7569f67127 100644 --- a/server/src/main/java/org/opensearch/index/query/RegexpFlag.java +++ b/server/src/main/java/org/opensearch/index/query/RegexpFlag.java @@ -32,7 +32,7 @@ package org.opensearch.index.query; import org.apache.lucene.util.automaton.RegExp; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import java.util.Locale; diff --git a/server/src/main/java/org/opensearch/index/query/RegexpQueryBuilder.java b/server/src/main/java/org/opensearch/index/query/RegexpQueryBuilder.java index 9a450a536480d..4b099995b2797 100644 --- a/server/src/main/java/org/opensearch/index/query/RegexpQueryBuilder.java +++ b/server/src/main/java/org/opensearch/index/query/RegexpQueryBuilder.java @@ -40,11 +40,11 @@ import org.apache.lucene.util.automaton.RegExp; import org.opensearch.LegacyESVersion; import org.opensearch.common.ParsingException; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.lucene.BytesRefs; import org.opensearch.common.xcontent.LoggingDeprecationHandler; +import org.opensearch.core.common.Strings; import org.opensearch.core.ParseField; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; 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 4dfe6a590c8b6..0d6993a42d7d7 100644 --- a/server/src/main/java/org/opensearch/index/query/SimpleQueryStringBuilder.java +++ b/server/src/main/java/org/opensearch/index/query/SimpleQueryStringBuilder.java @@ -37,11 +37,11 @@ import org.apache.lucene.search.Query; import org.opensearch.core.ParseField; import org.opensearch.common.ParsingException; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.lucene.search.Queries; import org.opensearch.common.util.CollectionUtils; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.index.search.QueryParserHelper; diff --git a/server/src/main/java/org/opensearch/index/query/SimpleQueryStringFlag.java b/server/src/main/java/org/opensearch/index/query/SimpleQueryStringFlag.java index 500e41dc6f73f..c989c52ca62b9 100644 --- a/server/src/main/java/org/opensearch/index/query/SimpleQueryStringFlag.java +++ b/server/src/main/java/org/opensearch/index/query/SimpleQueryStringFlag.java @@ -31,7 +31,7 @@ package org.opensearch.index.query; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.index.search.SimpleQueryStringQueryParser; import java.util.Locale; @@ -68,7 +68,7 @@ public int value() { } static int resolveFlags(String flags) { - if (!Strings.hasLength(flags)) { + if (Strings.hasLength(flags) == false) { return ALL.value(); } int magic = NONE.value(); diff --git a/server/src/main/java/org/opensearch/index/query/SpanNearQueryBuilder.java b/server/src/main/java/org/opensearch/index/query/SpanNearQueryBuilder.java index d2ee440ef7536..46cddf8adac4e 100644 --- a/server/src/main/java/org/opensearch/index/query/SpanNearQueryBuilder.java +++ b/server/src/main/java/org/opensearch/index/query/SpanNearQueryBuilder.java @@ -326,7 +326,7 @@ public static class SpanGapQueryBuilder implements SpanQueryBuilder { * @param width The width of the gap introduced */ public SpanGapQueryBuilder(String fieldName, int width) { - if (Strings.isEmpty(fieldName)) { + if (org.opensearch.core.common.Strings.isEmpty(fieldName)) { throw new IllegalArgumentException("[span_gap] field name is null or empty"); } // lucene has not coded any restriction on value of width. diff --git a/server/src/main/java/org/opensearch/index/query/TermsQueryBuilder.java b/server/src/main/java/org/opensearch/index/query/TermsQueryBuilder.java index 028fa4be8a2c8..78c4f30f9479d 100644 --- a/server/src/main/java/org/opensearch/index/query/TermsQueryBuilder.java +++ b/server/src/main/java/org/opensearch/index/query/TermsQueryBuilder.java @@ -42,11 +42,11 @@ import org.opensearch.client.Client; import org.opensearch.common.ParsingException; import org.opensearch.common.SetOnce; -import org.opensearch.common.Strings; import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.common.xcontent.support.XContentMapValues; diff --git a/server/src/main/java/org/opensearch/index/query/WildcardQueryBuilder.java b/server/src/main/java/org/opensearch/index/query/WildcardQueryBuilder.java index fb97c398512db..cf2f49da77128 100644 --- a/server/src/main/java/org/opensearch/index/query/WildcardQueryBuilder.java +++ b/server/src/main/java/org/opensearch/index/query/WildcardQueryBuilder.java @@ -38,11 +38,11 @@ import org.apache.lucene.search.Query; import org.opensearch.LegacyESVersion; import org.opensearch.common.ParsingException; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.xcontent.LoggingDeprecationHandler; import org.opensearch.core.ParseField; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.index.mapper.MappedFieldType; diff --git a/server/src/main/java/org/opensearch/index/query/WrapperQueryBuilder.java b/server/src/main/java/org/opensearch/index/query/WrapperQueryBuilder.java index a9532af8f091f..30d4ce575a499 100644 --- a/server/src/main/java/org/opensearch/index/query/WrapperQueryBuilder.java +++ b/server/src/main/java/org/opensearch/index/query/WrapperQueryBuilder.java @@ -36,11 +36,11 @@ import org.apache.lucene.util.BytesRef; import org.opensearch.core.ParseField; import org.opensearch.common.ParsingException; -import org.opensearch.common.Strings; import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.xcontent.LoggingDeprecationHandler; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.common.xcontent.XContentFactory; import org.opensearch.core.xcontent.XContentParser; diff --git a/server/src/main/java/org/opensearch/index/reindex/BulkByScrollTask.java b/server/src/main/java/org/opensearch/index/reindex/BulkByScrollTask.java index 2fe0fe0259c33..be9cade4925eb 100644 --- a/server/src/main/java/org/opensearch/index/reindex/BulkByScrollTask.java +++ b/server/src/main/java/org/opensearch/index/reindex/BulkByScrollTask.java @@ -32,6 +32,7 @@ package org.opensearch.index.reindex; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.OpenSearchException; import org.opensearch.common.Nullable; import org.opensearch.core.ParseField; @@ -999,7 +1000,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws status.toXContent(builder, params); } else { builder.startObject(); - OpenSearchException.generateThrowableXContent(builder, params, exception); + BaseExceptionsHelper.generateThrowableXContent(builder, params, exception); builder.endObject(); } return builder; diff --git a/server/src/main/java/org/opensearch/index/reindex/ReindexRequest.java b/server/src/main/java/org/opensearch/index/reindex/ReindexRequest.java index b7dcb84e36f32..1f89e2055ffd8 100644 --- a/server/src/main/java/org/opensearch/index/reindex/ReindexRequest.java +++ b/server/src/main/java/org/opensearch/index/reindex/ReindexRequest.java @@ -37,13 +37,13 @@ import org.opensearch.action.index.IndexRequest; import org.opensearch.action.search.SearchRequest; import org.opensearch.core.ParseField; -import org.opensearch.common.Strings; import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.lucene.uid.Versions; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ObjectParser; import org.opensearch.core.xcontent.ToXContentObject; import org.opensearch.core.xcontent.XContentBuilder; diff --git a/server/src/main/java/org/opensearch/index/reindex/ScrollableHitSource.java b/server/src/main/java/org/opensearch/index/reindex/ScrollableHitSource.java index 452c10e481243..8628bd84da35a 100644 --- a/server/src/main/java/org/opensearch/index/reindex/ScrollableHitSource.java +++ b/server/src/main/java/org/opensearch/index/reindex/ScrollableHitSource.java @@ -33,7 +33,7 @@ package org.opensearch.index.reindex; import org.apache.logging.log4j.Logger; -import org.opensearch.OpenSearchException; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.ExceptionsHelper; import org.opensearch.action.ActionListener; import org.opensearch.action.bulk.BackoffPolicy; @@ -137,7 +137,7 @@ public void done(TimeValue extraKeepAlive) { public final void close(Runnable onCompletion) { String scrollId = this.scrollId.get(); - if (Strings.hasLength(scrollId)) { + if (org.opensearch.core.common.Strings.hasLength(scrollId)) { clearScroll(scrollId, () -> cleanup(onCompletion)); } else { cleanup(onCompletion); @@ -489,7 +489,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(REASON_FIELD); { builder.startObject(); - OpenSearchException.generateThrowableXContent(builder, params, reason); + BaseExceptionsHelper.generateThrowableXContent(builder, params, reason); builder.endObject(); } builder.endObject(); diff --git a/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java b/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java index da4e9113143af..4a85b17b5c94a 100644 --- a/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java +++ b/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java @@ -42,7 +42,7 @@ import org.apache.lucene.store.FilterDirectory; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; -import org.opensearch.ExceptionsHelper; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.action.ActionListener; import org.opensearch.action.StepListener; import org.opensearch.cluster.metadata.IndexMetadata; @@ -576,7 +576,7 @@ private void internalRecoverFromStore(IndexShard indexShard) throws IndexShardRe files = Arrays.toString(store.directory().listAll()); } catch (Exception inner) { inner.addSuppressed(e); - files += " (failure=" + ExceptionsHelper.detailedMessage(inner) + ")"; + files += " (failure=" + BaseExceptionsHelper.detailedMessage(inner) + ")"; } if (indexShouldExists) { throw new IndexShardRecoveryException( diff --git a/server/src/main/java/org/opensearch/index/termvectors/TermVectorsService.java b/server/src/main/java/org/opensearch/index/termvectors/TermVectorsService.java index ff75c6e3ddc45..53734460a2d65 100644 --- a/server/src/main/java/org/opensearch/index/termvectors/TermVectorsService.java +++ b/server/src/main/java/org/opensearch/index/termvectors/TermVectorsService.java @@ -46,13 +46,13 @@ import org.opensearch.action.termvectors.TermVectorsRequest; import org.opensearch.action.termvectors.TermVectorsResponse; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.document.DocumentField; import org.opensearch.common.lucene.uid.VersionsAndSeqNoResolver.DocIdAndVersion; import org.opensearch.common.xcontent.XContentHelper; import org.opensearch.common.xcontent.XContentType; import org.opensearch.common.xcontent.support.XContentMapValues; +import org.opensearch.core.common.Strings; import org.opensearch.index.engine.Engine; import org.opensearch.index.get.GetResult; import org.opensearch.index.mapper.DocumentMapperForType; diff --git a/server/src/main/java/org/opensearch/index/translog/Translog.java b/server/src/main/java/org/opensearch/index/translog/Translog.java index fb0be2b6aaf0e..f6de68700dd13 100644 --- a/server/src/main/java/org/opensearch/index/translog/Translog.java +++ b/server/src/main/java/org/opensearch/index/translog/Translog.java @@ -36,7 +36,6 @@ import org.apache.lucene.store.AlreadyClosedException; import org.opensearch.Version; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.UUIDs; import org.opensearch.common.bytes.BytesArray; import org.opensearch.common.bytes.BytesReference; @@ -49,6 +48,7 @@ import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.concurrent.ReleasableLock; import org.opensearch.common.util.io.IOUtils; +import org.opensearch.core.common.Strings; import org.opensearch.index.IndexSettings; import org.opensearch.index.VersionType; import org.opensearch.index.engine.Engine; diff --git a/server/src/main/java/org/opensearch/indices/recovery/MultiFileWriter.java b/server/src/main/java/org/opensearch/indices/recovery/MultiFileWriter.java index ec3986017afac..5364bbbdc51c1 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/MultiFileWriter.java +++ b/server/src/main/java/org/opensearch/indices/recovery/MultiFileWriter.java @@ -37,11 +37,11 @@ import org.apache.lucene.store.IndexOutput; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRefIterator; -import org.opensearch.common.Strings; import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.lease.Releasable; import org.opensearch.common.util.concurrent.AbstractRefCounted; import org.opensearch.common.util.concurrent.ConcurrentCollections; +import org.opensearch.core.common.Strings; import org.opensearch.index.store.Store; import org.opensearch.index.store.StoreFileMetadata; import org.opensearch.indices.replication.common.ReplicationLuceneIndex; diff --git a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java index 66ca923ead027..5c6e3de662be3 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java +++ b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java @@ -36,7 +36,7 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.store.AlreadyClosedException; -import org.opensearch.ExceptionsHelper; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.LegacyESVersion; import org.opensearch.OpenSearchException; import org.opensearch.OpenSearchTimeoutException; @@ -680,7 +680,7 @@ private void onException(Exception e) { e ); } - Throwable cause = ExceptionsHelper.unwrapCause(e); + Throwable cause = BaseExceptionsHelper.unwrapCause(e); if (cause instanceof CancellableThreads.ExecutionCancelledException) { // this can also come from the source wrapped in a RemoteTransportException onGoingRecoveries.fail(recoveryId, new RecoveryFailedException(request, "source has canceled the recovery", cause), false); @@ -691,7 +691,7 @@ private void onException(Exception e) { cause = cause.getCause(); } // do it twice, in case we have double transport exception - cause = ExceptionsHelper.unwrapCause(cause); + cause = BaseExceptionsHelper.unwrapCause(cause); if (cause instanceof RecoveryEngineException) { // unwrap an exception that was thrown as part of the recovery cause = cause.getCause(); diff --git a/server/src/main/java/org/opensearch/indices/recovery/RetryableTransportClient.java b/server/src/main/java/org/opensearch/indices/recovery/RetryableTransportClient.java index 1e378cbeeba3d..3130721d22328 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/RetryableTransportClient.java +++ b/server/src/main/java/org/opensearch/indices/recovery/RetryableTransportClient.java @@ -9,7 +9,7 @@ package org.opensearch.indices.recovery; import org.apache.logging.log4j.Logger; -import org.opensearch.ExceptionsHelper; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.LegacyESVersion; import org.opensearch.action.ActionListener; import org.opensearch.action.ActionListenerResponseHandler; @@ -128,10 +128,10 @@ private static boolean retryableException(Exception e) { if (e instanceof ConnectTransportException) { return true; } else if (e instanceof SendRequestTransportException) { - final Throwable cause = ExceptionsHelper.unwrapCause(e); + final Throwable cause = BaseExceptionsHelper.unwrapCause(e); return cause instanceof ConnectTransportException; } else if (e instanceof RemoteTransportException) { - final Throwable cause = ExceptionsHelper.unwrapCause(e); + final Throwable cause = BaseExceptionsHelper.unwrapCause(e); return cause instanceof CircuitBreakingException || cause instanceof OpenSearchRejectedExecutionException; } return false; 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 a7e0c0ec887ab..2931caa353bc6 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java @@ -11,7 +11,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; -import org.opensearch.ExceptionsHelper; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.action.ActionListener; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.routing.ShardRouting; @@ -444,7 +444,7 @@ public void onResponse(Void o) { @Override public void onFailure(Exception e) { - Throwable cause = ExceptionsHelper.unwrapCause(e); + Throwable cause = BaseExceptionsHelper.unwrapCause(e); if (cause instanceof CancellableThreads.ExecutionCancelledException) { if (onGoingReplications.getTarget(replicationId) != null) { IndexShard indexShard = onGoingReplications.getTarget(replicationId).indexShard(); diff --git a/server/src/main/java/org/opensearch/indices/replication/common/ReplicationTarget.java b/server/src/main/java/org/opensearch/indices/replication/common/ReplicationTarget.java index 815ae29114bae..ef543bdbaf6f2 100644 --- a/server/src/main/java/org/opensearch/indices/replication/common/ReplicationTarget.java +++ b/server/src/main/java/org/opensearch/indices/replication/common/ReplicationTarget.java @@ -10,7 +10,7 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.store.RateLimiter; -import org.opensearch.ExceptionsHelper; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.OpenSearchException; import org.opensearch.action.ActionListener; import org.opensearch.action.support.ChannelActionListener; @@ -176,7 +176,7 @@ public void fail(ReplicationFailedException e, boolean sendShardFailure) { notifyListener(e, sendShardFailure); } finally { try { - cancellableThreads.cancel("failed" + description() + "[" + ExceptionsHelper.stackTrace(e) + "]"); + cancellableThreads.cancel("failed" + description() + "[" + BaseExceptionsHelper.stackTrace(e) + "]"); } finally { // release the initial reference. replication files will be cleaned as soon as ref count goes to zero, potentially now decRef(); diff --git a/server/src/main/java/org/opensearch/ingest/IngestDocument.java b/server/src/main/java/org/opensearch/ingest/IngestDocument.java index ecc5758299e42..e0de0a9488ad9 100644 --- a/server/src/main/java/org/opensearch/ingest/IngestDocument.java +++ b/server/src/main/java/org/opensearch/ingest/IngestDocument.java @@ -32,7 +32,7 @@ package org.opensearch.ingest; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.index.VersionType; import org.opensearch.index.mapper.IdFieldMapper; import org.opensearch.index.mapper.IndexFieldMapper; diff --git a/server/src/main/java/org/opensearch/plugins/ActionPlugin.java b/server/src/main/java/org/opensearch/plugins/ActionPlugin.java index d9792d60ae735..031ac7d068ef9 100644 --- a/server/src/main/java/org/opensearch/plugins/ActionPlugin.java +++ b/server/src/main/java/org/opensearch/plugins/ActionPlugin.java @@ -43,12 +43,12 @@ import org.opensearch.action.support.TransportActions; import org.opensearch.cluster.metadata.IndexNameExpressionResolver; import org.opensearch.cluster.node.DiscoveryNodes; -import org.opensearch.common.Strings; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.IndexScopedSettings; import org.opensearch.common.settings.Settings; import org.opensearch.common.settings.SettingsFilter; import org.opensearch.common.util.concurrent.ThreadContext; +import org.opensearch.core.common.Strings; import org.opensearch.rest.RestController; import org.opensearch.rest.RestHandler; import org.opensearch.rest.RestHeaderDefinition; diff --git a/server/src/main/java/org/opensearch/plugins/PluginInfo.java b/server/src/main/java/org/opensearch/plugins/PluginInfo.java index 61332792dc622..a3fd2fc5ef68b 100644 --- a/server/src/main/java/org/opensearch/plugins/PluginInfo.java +++ b/server/src/main/java/org/opensearch/plugins/PluginInfo.java @@ -34,10 +34,10 @@ import org.opensearch.Version; import org.opensearch.bootstrap.JarHell; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.io.stream.Writeable; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ToXContentObject; import org.opensearch.core.xcontent.XContentBuilder; diff --git a/server/src/main/java/org/opensearch/plugins/PluginsService.java b/server/src/main/java/org/opensearch/plugins/PluginsService.java index 8319d254892a7..3cd226c357367 100644 --- a/server/src/main/java/org/opensearch/plugins/PluginsService.java +++ b/server/src/main/java/org/opensearch/plugins/PluginsService.java @@ -43,13 +43,13 @@ import org.opensearch.Version; import org.opensearch.action.admin.cluster.node.info.PluginsAndModules; import org.opensearch.bootstrap.JarHell; -import org.opensearch.common.Strings; import org.opensearch.common.collect.Tuple; import org.opensearch.common.component.LifecycleComponent; import org.opensearch.common.inject.Module; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Setting.Property; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; import org.opensearch.index.IndexModule; import org.opensearch.node.ReportingService; import org.opensearch.threadpool.ExecutorBuilder; diff --git a/server/src/main/java/org/opensearch/repositories/RepositoriesService.java b/server/src/main/java/org/opensearch/repositories/RepositoriesService.java index 9502ee2962000..9c56d172f2ea1 100644 --- a/server/src/main/java/org/opensearch/repositories/RepositoriesService.java +++ b/server/src/main/java/org/opensearch/repositories/RepositoriesService.java @@ -598,7 +598,7 @@ private Repository createRepository(RepositoryMetadata repositoryMetadata, Map messageSupplier = () -> new ParameterizedMessage( "path: {}, params: {}", @@ -153,8 +156,12 @@ public RestStatus status() { private ToXContent.Params paramsFromRequest(RestRequest restRequest) { ToXContent.Params params = restRequest; - if (params.paramAsBoolean("error_trace", !REST_EXCEPTION_SKIP_STACK_TRACE_DEFAULT) && false == skipStackTrace()) { - params = new ToXContent.DelegatingMapParams(singletonMap(REST_EXCEPTION_SKIP_STACK_TRACE, "false"), params); + if (params.paramAsBoolean("error_trace", !BaseExceptionsHelper.REST_EXCEPTION_SKIP_STACK_TRACE_DEFAULT) + && false == skipStackTrace()) { + params = new ToXContent.DelegatingMapParams( + singletonMap(BaseExceptionsHelper.REST_EXCEPTION_SKIP_STACK_TRACE, "false"), + params + ); } return params; } @@ -166,7 +173,7 @@ protected boolean skipStackTrace() { private void build(XContentBuilder builder, ToXContent.Params params, RestStatus status, boolean detailedErrorsEnabled, Exception e) throws IOException { builder.startObject(); - OpenSearchException.generateFailureXContent(builder, params, e, detailedErrorsEnabled); + BaseOpenSearchException.generateFailureXContent(builder, params, e, detailedErrorsEnabled); builder.field(STATUS, status.getStatus()); builder.endObject(); } diff --git a/server/src/main/java/org/opensearch/rest/DeprecationRestHandler.java b/server/src/main/java/org/opensearch/rest/DeprecationRestHandler.java index e4e0acb2123eb..0cdb08a11b45e 100644 --- a/server/src/main/java/org/opensearch/rest/DeprecationRestHandler.java +++ b/server/src/main/java/org/opensearch/rest/DeprecationRestHandler.java @@ -32,8 +32,8 @@ package org.opensearch.rest; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; import org.opensearch.common.logging.DeprecationLogger; +import org.opensearch.core.common.Strings; import java.util.Objects; diff --git a/server/src/main/java/org/opensearch/rest/RestController.java b/server/src/main/java/org/opensearch/rest/RestController.java index c3ef93abc35d9..e4b10e0babda4 100644 --- a/server/src/main/java/org/opensearch/rest/RestController.java +++ b/server/src/main/java/org/opensearch/rest/RestController.java @@ -38,7 +38,6 @@ import org.opensearch.OpenSearchException; import org.opensearch.client.node.NodeClient; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.breaker.CircuitBreaker; import org.opensearch.common.bytes.BytesArray; import org.opensearch.common.bytes.BytesReference; @@ -46,6 +45,7 @@ import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.path.PathTrie; import org.opensearch.common.util.concurrent.ThreadContext; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.common.util.FeatureFlags; import org.opensearch.common.xcontent.XContentType; diff --git a/server/src/main/java/org/opensearch/rest/RestRequest.java b/server/src/main/java/org/opensearch/rest/RestRequest.java index b4f70e359d618..36961ed3e20da 100644 --- a/server/src/main/java/org/opensearch/rest/RestRequest.java +++ b/server/src/main/java/org/opensearch/rest/RestRequest.java @@ -461,7 +461,7 @@ public String[] paramAsStringArray(String key, String[] defaultValue) { if (value == null) { return defaultValue; } - return Strings.splitStringByCommaToArray(value); + return org.opensearch.core.common.Strings.splitStringByCommaToArray(value); } public String[] paramAsStringArrayOrEmptyIfAll(String key) { diff --git a/server/src/main/java/org/opensearch/rest/RestRequestFilter.java b/server/src/main/java/org/opensearch/rest/RestRequestFilter.java index b173763b3813f..f24c5a15c4304 100644 --- a/server/src/main/java/org/opensearch/rest/RestRequestFilter.java +++ b/server/src/main/java/org/opensearch/rest/RestRequestFilter.java @@ -33,13 +33,13 @@ package org.opensearch.rest; import org.opensearch.OpenSearchException; -import org.opensearch.common.Strings; import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.collect.Tuple; -import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.common.xcontent.XContentHelper; import org.opensearch.common.xcontent.XContentType; import org.opensearch.common.xcontent.support.XContentMapValues; +import org.opensearch.core.common.Strings; +import org.opensearch.core.xcontent.XContentBuilder; import java.io.IOException; import java.util.Map; diff --git a/server/src/main/java/org/opensearch/rest/RestUtils.java b/server/src/main/java/org/opensearch/rest/RestUtils.java index da2a14126d53c..c165fccf70789 100644 --- a/server/src/main/java/org/opensearch/rest/RestUtils.java +++ b/server/src/main/java/org/opensearch/rest/RestUtils.java @@ -33,8 +33,8 @@ package org.opensearch.rest; import org.opensearch.common.Booleans; -import org.opensearch.common.Strings; import org.opensearch.common.path.PathTrie; +import org.opensearch.core.common.Strings; import java.nio.charset.Charset; import java.nio.charset.StandardCharsets; diff --git a/server/src/main/java/org/opensearch/rest/action/RestFieldCapabilitiesAction.java b/server/src/main/java/org/opensearch/rest/action/RestFieldCapabilitiesAction.java index 2c1087327776c..7020c7cab10df 100644 --- a/server/src/main/java/org/opensearch/rest/action/RestFieldCapabilitiesAction.java +++ b/server/src/main/java/org/opensearch/rest/action/RestFieldCapabilitiesAction.java @@ -35,7 +35,7 @@ import org.opensearch.action.fieldcaps.FieldCapabilitiesRequest; import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestAddVotingConfigExclusionAction.java b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestAddVotingConfigExclusionAction.java index f04985886084b..9addbcca27258 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestAddVotingConfigExclusionAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestAddVotingConfigExclusionAction.java @@ -37,8 +37,8 @@ import org.opensearch.action.admin.cluster.configuration.AddVotingConfigExclusionsAction; import org.opensearch.action.admin.cluster.configuration.AddVotingConfigExclusionsRequest; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestToXContentListener; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestCancelTasksAction.java b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestCancelTasksAction.java index 779d172b5d723..379c7ac2fc570 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestCancelTasksAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestCancelTasksAction.java @@ -35,7 +35,7 @@ import org.opensearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest; import org.opensearch.client.node.NodeClient; import org.opensearch.cluster.node.DiscoveryNodes; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.tasks.TaskId; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestClusterHealthAction.java b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestClusterHealthAction.java index 0731ae015d504..6a6f6937ff18a 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestClusterHealthAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestClusterHealthAction.java @@ -38,8 +38,8 @@ import org.opensearch.client.node.NodeClient; import org.opensearch.cluster.health.ClusterHealthStatus; import org.opensearch.common.Priority; -import org.opensearch.common.Strings; import org.opensearch.common.logging.DeprecationLogger; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestStatusToXContentListener; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestClusterRerouteAction.java b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestClusterRerouteAction.java index 6fbe899960b03..a635ca775148f 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestClusterRerouteAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestClusterRerouteAction.java @@ -38,10 +38,10 @@ import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.routing.allocation.command.AllocationCommands; import org.opensearch.core.ParseField; -import org.opensearch.common.Strings; import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.settings.Settings; import org.opensearch.common.settings.SettingsFilter; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ObjectParser; import org.opensearch.core.xcontent.ObjectParser.ValueType; import org.opensearch.rest.BaseRestHandler; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestClusterSearchShardsAction.java b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestClusterSearchShardsAction.java index e1b1389c47891..3555576433104 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestClusterSearchShardsAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestClusterSearchShardsAction.java @@ -36,7 +36,7 @@ import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.Requests; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestToXContentListener; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestClusterStateAction.java b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestClusterStateAction.java index 86f7fef141d78..3cc5fe21a1c72 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestClusterStateAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestClusterStateAction.java @@ -39,10 +39,10 @@ import org.opensearch.client.node.NodeClient; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.Metadata; -import org.opensearch.common.Strings; import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.settings.Settings; import org.opensearch.common.settings.SettingsFilter; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.rest.BaseRestHandler; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestDeleteSnapshotAction.java b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestDeleteSnapshotAction.java index 48f2ad75db4b4..31528dbe6ccf0 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestDeleteSnapshotAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestDeleteSnapshotAction.java @@ -34,8 +34,8 @@ import org.opensearch.action.admin.cluster.snapshots.delete.DeleteSnapshotRequest; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; import org.opensearch.common.logging.DeprecationLogger; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestToXContentListener; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestGetRepositoriesAction.java b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestGetRepositoriesAction.java index 4ef940bd1eb90..a9c2d18645af2 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestGetRepositoriesAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestGetRepositoriesAction.java @@ -34,10 +34,10 @@ import org.opensearch.action.admin.cluster.repositories.get.GetRepositoriesRequest; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.settings.Settings; import org.opensearch.common.settings.SettingsFilter; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestToXContentListener; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestGetSnapshotsAction.java b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestGetSnapshotsAction.java index 9668290af0901..72733475e0cdd 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestGetSnapshotsAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestGetSnapshotsAction.java @@ -34,8 +34,8 @@ import org.opensearch.action.admin.cluster.snapshots.get.GetSnapshotsRequest; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; import org.opensearch.common.logging.DeprecationLogger; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestToXContentListener; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestListTasksAction.java b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestListTasksAction.java index 6804cb4bd6f70..b50c23edb7169 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestListTasksAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestListTasksAction.java @@ -37,8 +37,8 @@ import org.opensearch.action.admin.cluster.node.tasks.list.ListTasksResponse; import org.opensearch.client.node.NodeClient; import org.opensearch.cluster.node.DiscoveryNodes; -import org.opensearch.common.Strings; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.BytesRestResponse; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesHotThreadsAction.java b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesHotThreadsAction.java index 3c9321a8abaa4..443b903a54fb3 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesHotThreadsAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesHotThreadsAction.java @@ -36,8 +36,8 @@ import org.opensearch.action.admin.cluster.node.hotthreads.NodesHotThreadsRequest; import org.opensearch.action.admin.cluster.node.hotthreads.NodesHotThreadsResponse; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.BytesRestResponse; import org.opensearch.rest.RestRequest; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesInfoAction.java b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesInfoAction.java index 534ae087c0e9b..3b83bf9d6f68c 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesInfoAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesInfoAction.java @@ -34,10 +34,10 @@ import org.opensearch.action.admin.cluster.node.info.NodesInfoRequest; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; import org.opensearch.common.settings.SettingsFilter; import org.opensearch.common.util.set.Sets; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestActions.NodesResponseRestListener; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesStatsAction.java b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesStatsAction.java index c0e956fc419d3..66b9afda06eb6 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesStatsAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesStatsAction.java @@ -36,7 +36,7 @@ import org.opensearch.action.admin.indices.stats.CommonStatsFlags; import org.opensearch.action.admin.indices.stats.CommonStatsFlags.Flag; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestActions.NodesResponseRestListener; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesUsageAction.java b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesUsageAction.java index 29186a66df2ad..3b9789cfc73e0 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesUsageAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestNodesUsageAction.java @@ -35,7 +35,7 @@ import org.opensearch.action.admin.cluster.node.usage.NodesUsageRequest; import org.opensearch.action.admin.cluster.node.usage.NodesUsageResponse; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.BytesRestResponse; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestReloadSecureSettingsAction.java b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestReloadSecureSettingsAction.java index 2fb8104927b8d..4233c2bd4c0f6 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestReloadSecureSettingsAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestReloadSecureSettingsAction.java @@ -37,8 +37,8 @@ import org.opensearch.action.admin.cluster.node.reload.NodesReloadSecureSettingsResponse; import org.opensearch.client.node.NodeClient; import org.opensearch.core.ParseField; -import org.opensearch.common.Strings; import org.opensearch.common.settings.SecureString; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ObjectParser; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.rest.BaseRestHandler; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestSnapshotsStatusAction.java b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestSnapshotsStatusAction.java index 7f527095f754d..b2d4fff14a7f1 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestSnapshotsStatusAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/cluster/RestSnapshotsStatusAction.java @@ -34,8 +34,8 @@ import org.opensearch.action.admin.cluster.snapshots.status.SnapshotsStatusRequest; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; import org.opensearch.common.logging.DeprecationLogger; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestToXContentListener; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestAddIndexBlockAction.java b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestAddIndexBlockAction.java index c10a6e82cc156..04327d4324d52 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestAddIndexBlockAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestAddIndexBlockAction.java @@ -36,8 +36,8 @@ import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.node.NodeClient; import org.opensearch.cluster.metadata.IndexMetadata; -import org.opensearch.common.Strings; import org.opensearch.common.logging.DeprecationLogger; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestToXContentListener; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestClearIndicesCacheAction.java b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestClearIndicesCacheAction.java index ee1d5a97d00c0..616a6ffd63433 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestClearIndicesCacheAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestClearIndicesCacheAction.java @@ -35,7 +35,7 @@ import org.opensearch.action.admin.indices.cache.clear.ClearIndicesCacheRequest; import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestToXContentListener; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestCloseIndexAction.java b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestCloseIndexAction.java index 756c83560ef2e..465fff02b22d2 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestCloseIndexAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestCloseIndexAction.java @@ -36,8 +36,8 @@ import org.opensearch.action.support.ActiveShardCount; import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; import org.opensearch.common.logging.DeprecationLogger; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestToXContentListener; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestDataStreamsStatsAction.java b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestDataStreamsStatsAction.java index 741be5c5ffc9c..e3e549fa2c587 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestDataStreamsStatsAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestDataStreamsStatsAction.java @@ -35,7 +35,7 @@ import org.opensearch.action.admin.indices.datastream.DataStreamsStatsAction; import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestToXContentListener; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestDeleteDataStreamAction.java b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestDeleteDataStreamAction.java index 2665609511676..cebebc63fde29 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestDeleteDataStreamAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestDeleteDataStreamAction.java @@ -33,7 +33,7 @@ import org.opensearch.action.admin.indices.datastream.DeleteDataStreamAction; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestToXContentListener; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestDeleteIndexAction.java b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestDeleteIndexAction.java index 4b6ee17a8cb48..6c422f55918a2 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestDeleteIndexAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestDeleteIndexAction.java @@ -35,8 +35,8 @@ import org.opensearch.action.admin.indices.delete.DeleteIndexRequest; import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; import org.opensearch.common.logging.DeprecationLogger; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestToXContentListener; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestFlushAction.java b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestFlushAction.java index 03c91bced293f..59daaef33e4de 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestFlushAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestFlushAction.java @@ -35,7 +35,7 @@ import org.opensearch.action.admin.indices.flush.FlushRequest; import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestToXContentListener; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestForceMergeAction.java b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestForceMergeAction.java index eefe284d572ba..06f1d5f46f90b 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestForceMergeAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestForceMergeAction.java @@ -36,8 +36,8 @@ import org.opensearch.action.admin.indices.forcemerge.ForceMergeRequest; import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; import org.opensearch.common.logging.DeprecationLogger; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestToXContentListener; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestGetAliasesAction.java b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestGetAliasesAction.java index 214bc4ba418dd..15605a32e4189 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestGetAliasesAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestGetAliasesAction.java @@ -38,8 +38,8 @@ import org.opensearch.client.node.NodeClient; import org.opensearch.cluster.metadata.AliasMetadata; import org.opensearch.cluster.metadata.Metadata; -import org.opensearch.common.Strings; import org.opensearch.common.regex.Regex; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.rest.BaseRestHandler; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestGetDataStreamsAction.java b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestGetDataStreamsAction.java index 4b447d1ac337a..2c7a8398a34c6 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestGetDataStreamsAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestGetDataStreamsAction.java @@ -33,7 +33,7 @@ import org.opensearch.action.admin.indices.datastream.GetDataStreamAction; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestToXContentListener; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestGetFieldMappingAction.java b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestGetFieldMappingAction.java index 5acf1aa0d42ab..ca48b4de2f609 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestGetFieldMappingAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestGetFieldMappingAction.java @@ -39,8 +39,8 @@ import org.opensearch.action.admin.indices.mapping.get.GetFieldMappingsResponse.FieldMappingMetadata; import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; import org.opensearch.common.logging.DeprecationLogger; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.BytesRestResponse; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestGetIndexTemplateAction.java b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestGetIndexTemplateAction.java index 11378ddcbf01e..6fe18926cab43 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestGetIndexTemplateAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestGetIndexTemplateAction.java @@ -35,9 +35,9 @@ import org.opensearch.action.admin.indices.template.get.GetIndexTemplatesRequest; import org.opensearch.action.admin.indices.template.get.GetIndexTemplatesResponse; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.RestStatus; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestGetIndicesAction.java b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestGetIndicesAction.java index 1fb20fae25c27..7c62d125ef43a 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestGetIndicesAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestGetIndicesAction.java @@ -35,9 +35,9 @@ import org.opensearch.action.admin.indices.get.GetIndexRequest; import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestToXContentListener; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestGetMappingAction.java b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestGetMappingAction.java index 3d244265c4169..c237877de0a46 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestGetMappingAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestGetMappingAction.java @@ -39,8 +39,8 @@ import org.opensearch.action.admin.indices.mapping.get.GetMappingsResponse; import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.BytesRestResponse; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestGetSettingsAction.java b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestGetSettingsAction.java index 2e1047a0945b9..827d315173881 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestGetSettingsAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestGetSettingsAction.java @@ -35,8 +35,8 @@ import org.opensearch.action.admin.indices.settings.get.GetSettingsRequest; import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; import org.opensearch.common.logging.DeprecationLogger; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestToXContentListener; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestIndexDeleteAliasesAction.java b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestIndexDeleteAliasesAction.java index 14078a9041684..830a3db599df7 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestIndexDeleteAliasesAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestIndexDeleteAliasesAction.java @@ -34,8 +34,8 @@ import org.opensearch.action.admin.indices.alias.IndicesAliasesRequest; import org.opensearch.action.admin.indices.alias.IndicesAliasesRequest.AliasActions; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; import org.opensearch.common.logging.DeprecationLogger; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestToXContentListener; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestIndexPutAliasAction.java b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestIndexPutAliasAction.java index 22f91af486d42..9253763c53fb8 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestIndexPutAliasAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestIndexPutAliasAction.java @@ -34,8 +34,8 @@ import org.opensearch.action.admin.indices.alias.IndicesAliasesRequest; import org.opensearch.action.admin.indices.alias.IndicesAliasesRequest.AliasActions; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; import org.opensearch.common.logging.DeprecationLogger; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestIndicesSegmentsAction.java b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestIndicesSegmentsAction.java index 3906c48295b4f..f263158abcb7b 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestIndicesSegmentsAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestIndicesSegmentsAction.java @@ -35,7 +35,7 @@ import org.opensearch.action.admin.indices.segments.IndicesSegmentsRequest; import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestToXContentListener; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestIndicesShardStoresAction.java b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestIndicesShardStoresAction.java index 041f48113527f..e80fd1519d507 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestIndicesShardStoresAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestIndicesShardStoresAction.java @@ -37,7 +37,7 @@ import org.opensearch.action.admin.indices.shards.IndicesShardStoresResponse; import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.BytesRestResponse; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestIndicesStatsAction.java b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestIndicesStatsAction.java index ba1407b36bfd9..d370fa074aaed 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestIndicesStatsAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestIndicesStatsAction.java @@ -37,7 +37,7 @@ import org.opensearch.action.admin.indices.stats.IndicesStatsRequest; import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestToXContentListener; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestOpenIndexAction.java b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestOpenIndexAction.java index caf05abf7f816..34c1f0b725ec3 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestOpenIndexAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestOpenIndexAction.java @@ -38,8 +38,8 @@ import org.opensearch.action.support.ActiveShardCount; import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; import org.opensearch.common.logging.DeprecationLogger; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestToXContentListener; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestPutIndexTemplateAction.java b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestPutIndexTemplateAction.java index 3ed39c293df7a..5de999df87829 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestPutIndexTemplateAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestPutIndexTemplateAction.java @@ -34,9 +34,9 @@ import org.opensearch.action.admin.indices.template.put.PutIndexTemplateRequest; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.xcontent.XContentHelper; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestToXContentListener; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestPutMappingAction.java b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestPutMappingAction.java index a1d3b13bdb02b..75609468036da 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestPutMappingAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestPutMappingAction.java @@ -35,9 +35,9 @@ import org.opensearch.action.admin.indices.mapping.put.PutMappingRequest; import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.xcontent.XContentHelper; +import org.opensearch.core.common.Strings; import org.opensearch.index.mapper.MapperService; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestRecoveryAction.java b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestRecoveryAction.java index 33c7b239e27ce..768e31d854200 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestRecoveryAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestRecoveryAction.java @@ -35,7 +35,7 @@ import org.opensearch.action.admin.indices.recovery.RecoveryRequest; import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestToXContentListener; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestRefreshAction.java b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestRefreshAction.java index 2dc4c280956b5..2d26d09e09e6e 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestRefreshAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestRefreshAction.java @@ -36,7 +36,7 @@ import org.opensearch.action.admin.indices.refresh.RefreshResponse; import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.RestStatus; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestResolveIndexAction.java b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestResolveIndexAction.java index eee9804abec3b..2feca44c95a85 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestResolveIndexAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestResolveIndexAction.java @@ -35,7 +35,7 @@ import org.opensearch.action.admin.indices.resolve.ResolveIndexAction; import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestToXContentListener; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestSyncedFlushAction.java b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestSyncedFlushAction.java index 8d093d7bd9e27..99d5d706f60c8 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestSyncedFlushAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestSyncedFlushAction.java @@ -36,8 +36,8 @@ import org.opensearch.action.admin.indices.flush.FlushResponse; import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; import org.opensearch.common.logging.DeprecationLogger; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.BytesRestResponse; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestUpdateSettingsAction.java b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestUpdateSettingsAction.java index ef95f33dc4bf2..b229fa2eccb2c 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestUpdateSettingsAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestUpdateSettingsAction.java @@ -35,9 +35,9 @@ import org.opensearch.action.admin.indices.settings.put.UpdateSettingsRequest; import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestToXContentListener; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestUpgradeAction.java b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestUpgradeAction.java index 1c925f030f582..4f53fde259454 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestUpgradeAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestUpgradeAction.java @@ -35,7 +35,7 @@ import org.opensearch.action.admin.indices.upgrade.post.UpgradeRequest; import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestToXContentListener; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestUpgradeStatusAction.java b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestUpgradeStatusAction.java index a05b52b708770..79c8ada440df6 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestUpgradeStatusAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestUpgradeStatusAction.java @@ -35,7 +35,7 @@ import org.opensearch.action.admin.indices.upgrade.get.UpgradeStatusRequest; import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestToXContentListener; diff --git a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestValidateQueryAction.java b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestValidateQueryAction.java index bfd44c30ec59f..3e064bc0a170a 100644 --- a/server/src/main/java/org/opensearch/rest/action/admin/indices/RestValidateQueryAction.java +++ b/server/src/main/java/org/opensearch/rest/action/admin/indices/RestValidateQueryAction.java @@ -38,7 +38,7 @@ import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.node.NodeClient; import org.opensearch.common.ParsingException; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.BytesRestResponse; diff --git a/server/src/main/java/org/opensearch/rest/action/cat/RestAliasAction.java b/server/src/main/java/org/opensearch/rest/action/cat/RestAliasAction.java index 6df5dc268d666..4600dddbb361d 100644 --- a/server/src/main/java/org/opensearch/rest/action/cat/RestAliasAction.java +++ b/server/src/main/java/org/opensearch/rest/action/cat/RestAliasAction.java @@ -36,8 +36,8 @@ import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.node.NodeClient; import org.opensearch.cluster.metadata.AliasMetadata; -import org.opensearch.common.Strings; import org.opensearch.common.Table; +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/rest/action/cat/RestAllocationAction.java b/server/src/main/java/org/opensearch/rest/action/cat/RestAllocationAction.java index f8d720cc69957..6a2be1efc0b86 100644 --- a/server/src/main/java/org/opensearch/rest/action/cat/RestAllocationAction.java +++ b/server/src/main/java/org/opensearch/rest/action/cat/RestAllocationAction.java @@ -42,10 +42,10 @@ import org.opensearch.client.node.NodeClient; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.routing.ShardRouting; -import org.opensearch.common.Strings; import org.opensearch.common.Table; import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.unit.ByteSizeValue; +import org.opensearch.core.common.Strings; import org.opensearch.rest.RestRequest; import org.opensearch.rest.RestResponse; import org.opensearch.rest.action.RestActionListener; diff --git a/server/src/main/java/org/opensearch/rest/action/cat/RestCatRecoveryAction.java b/server/src/main/java/org/opensearch/rest/action/cat/RestCatRecoveryAction.java index f18ace993599d..4f95e10ae3622 100644 --- a/server/src/main/java/org/opensearch/rest/action/cat/RestCatRecoveryAction.java +++ b/server/src/main/java/org/opensearch/rest/action/cat/RestCatRecoveryAction.java @@ -39,10 +39,10 @@ import org.opensearch.client.node.NodeClient; import org.opensearch.cluster.routing.RecoverySource; import org.opensearch.cluster.routing.RecoverySource.SnapshotRecoverySource; -import org.opensearch.common.Strings; import org.opensearch.common.Table; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.xcontent.XContentOpenSearchExtension; +import org.opensearch.core.common.Strings; import org.opensearch.indices.recovery.RecoveryState; import org.opensearch.rest.RestRequest; import org.opensearch.rest.RestResponse; diff --git a/server/src/main/java/org/opensearch/rest/action/cat/RestCatSegmentReplicationAction.java b/server/src/main/java/org/opensearch/rest/action/cat/RestCatSegmentReplicationAction.java index 0130f9cd14c36..52890274d4198 100644 --- a/server/src/main/java/org/opensearch/rest/action/cat/RestCatSegmentReplicationAction.java +++ b/server/src/main/java/org/opensearch/rest/action/cat/RestCatSegmentReplicationAction.java @@ -13,11 +13,11 @@ import org.opensearch.action.admin.indices.replication.SegmentReplicationStatsResponse; import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; import org.opensearch.common.Table; import org.opensearch.common.unit.ByteSizeValue; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.xcontent.XContentOpenSearchExtension; +import org.opensearch.core.common.Strings; import org.opensearch.index.SegmentReplicationPerGroupStats; import org.opensearch.index.SegmentReplicationShardStats; import org.opensearch.indices.replication.SegmentReplicationState; diff --git a/server/src/main/java/org/opensearch/rest/action/cat/RestCountAction.java b/server/src/main/java/org/opensearch/rest/action/cat/RestCountAction.java index a35c0047a03bf..9c054ffe1bcc7 100644 --- a/server/src/main/java/org/opensearch/rest/action/cat/RestCountAction.java +++ b/server/src/main/java/org/opensearch/rest/action/cat/RestCountAction.java @@ -37,8 +37,8 @@ import org.opensearch.action.search.SearchRequest; import org.opensearch.action.search.SearchResponse; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; import org.opensearch.common.Table; +import org.opensearch.core.common.Strings; import org.opensearch.index.query.QueryBuilder; import org.opensearch.rest.RestRequest; import org.opensearch.rest.RestResponse; diff --git a/server/src/main/java/org/opensearch/rest/action/cat/RestIndicesAction.java b/server/src/main/java/org/opensearch/rest/action/cat/RestIndicesAction.java index 5454d16830f91..9c66d8bcb3edf 100644 --- a/server/src/main/java/org/opensearch/rest/action/cat/RestIndicesAction.java +++ b/server/src/main/java/org/opensearch/rest/action/cat/RestIndicesAction.java @@ -51,11 +51,11 @@ import org.opensearch.cluster.health.ClusterHealthStatus; import org.opensearch.cluster.health.ClusterIndexHealth; import org.opensearch.cluster.metadata.IndexMetadata; -import org.opensearch.common.Strings; import org.opensearch.common.Table; import org.opensearch.common.settings.Settings; import org.opensearch.common.time.DateFormatter; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import org.opensearch.index.IndexSettings; import org.opensearch.rest.RestRequest; import org.opensearch.rest.RestResponse; diff --git a/server/src/main/java/org/opensearch/rest/action/cat/RestSegmentsAction.java b/server/src/main/java/org/opensearch/rest/action/cat/RestSegmentsAction.java index 69378b2af95fc..68087d77fd326 100644 --- a/server/src/main/java/org/opensearch/rest/action/cat/RestSegmentsAction.java +++ b/server/src/main/java/org/opensearch/rest/action/cat/RestSegmentsAction.java @@ -41,9 +41,9 @@ import org.opensearch.action.admin.indices.segments.ShardSegments; import org.opensearch.client.node.NodeClient; 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.index.engine.Segment; import org.opensearch.rest.RestRequest; import org.opensearch.rest.RestResponse; diff --git a/server/src/main/java/org/opensearch/rest/action/cat/RestShardsAction.java b/server/src/main/java/org/opensearch/rest/action/cat/RestShardsAction.java index ca6cfb6eab1f4..cadc6ef730350 100644 --- a/server/src/main/java/org/opensearch/rest/action/cat/RestShardsAction.java +++ b/server/src/main/java/org/opensearch/rest/action/cat/RestShardsAction.java @@ -41,10 +41,10 @@ import org.opensearch.client.node.NodeClient; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.UnassignedInfo; -import org.opensearch.common.Strings; import org.opensearch.common.Table; import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import org.opensearch.index.cache.query.QueryCacheStats; import org.opensearch.index.engine.CommitStats; import org.opensearch.index.engine.Engine; diff --git a/server/src/main/java/org/opensearch/rest/action/cat/RestTable.java b/server/src/main/java/org/opensearch/rest/action/cat/RestTable.java index 69f1e6b3228cf..cb4b2ca281348 100644 --- a/server/src/main/java/org/opensearch/rest/action/cat/RestTable.java +++ b/server/src/main/java/org/opensearch/rest/action/cat/RestTable.java @@ -33,7 +33,6 @@ package org.opensearch.rest.action.cat; import org.opensearch.common.Booleans; -import org.opensearch.common.Strings; import org.opensearch.common.Table; import org.opensearch.common.io.Streams; import org.opensearch.common.io.UTF8StreamWriter; @@ -42,6 +41,7 @@ import org.opensearch.common.unit.ByteSizeValue; import org.opensearch.common.unit.SizeValue; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.common.xcontent.XContentType; import org.opensearch.rest.BytesRestResponse; diff --git a/server/src/main/java/org/opensearch/rest/action/document/RestGetAction.java b/server/src/main/java/org/opensearch/rest/action/document/RestGetAction.java index 87bec22ac2d73..2abb3135319e5 100644 --- a/server/src/main/java/org/opensearch/rest/action/document/RestGetAction.java +++ b/server/src/main/java/org/opensearch/rest/action/document/RestGetAction.java @@ -35,7 +35,7 @@ import org.opensearch.action.get.GetRequest; import org.opensearch.action.get.GetResponse; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.index.VersionType; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; diff --git a/server/src/main/java/org/opensearch/rest/action/document/RestMultiGetAction.java b/server/src/main/java/org/opensearch/rest/action/document/RestMultiGetAction.java index 4b9e1f07df59a..3190a016b613a 100644 --- a/server/src/main/java/org/opensearch/rest/action/document/RestMultiGetAction.java +++ b/server/src/main/java/org/opensearch/rest/action/document/RestMultiGetAction.java @@ -34,8 +34,8 @@ import org.opensearch.action.get.MultiGetRequest; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; diff --git a/server/src/main/java/org/opensearch/rest/action/document/RestMultiTermVectorsAction.java b/server/src/main/java/org/opensearch/rest/action/document/RestMultiTermVectorsAction.java index c453094f33120..67618ed8e5c2b 100644 --- a/server/src/main/java/org/opensearch/rest/action/document/RestMultiTermVectorsAction.java +++ b/server/src/main/java/org/opensearch/rest/action/document/RestMultiTermVectorsAction.java @@ -35,7 +35,7 @@ import org.opensearch.action.termvectors.MultiTermVectorsRequest; import org.opensearch.action.termvectors.TermVectorsRequest; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestToXContentListener; diff --git a/server/src/main/java/org/opensearch/rest/action/document/RestTermVectorsAction.java b/server/src/main/java/org/opensearch/rest/action/document/RestTermVectorsAction.java index 8dbfe84ea48fb..b7b75104adaad 100644 --- a/server/src/main/java/org/opensearch/rest/action/document/RestTermVectorsAction.java +++ b/server/src/main/java/org/opensearch/rest/action/document/RestTermVectorsAction.java @@ -34,7 +34,7 @@ import org.opensearch.action.termvectors.TermVectorsRequest; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.index.VersionType; import org.opensearch.rest.BaseRestHandler; diff --git a/server/src/main/java/org/opensearch/rest/action/ingest/RestGetPipelineAction.java b/server/src/main/java/org/opensearch/rest/action/ingest/RestGetPipelineAction.java index 75efe29872285..c6bdece0a7534 100644 --- a/server/src/main/java/org/opensearch/rest/action/ingest/RestGetPipelineAction.java +++ b/server/src/main/java/org/opensearch/rest/action/ingest/RestGetPipelineAction.java @@ -34,8 +34,8 @@ import org.opensearch.action.ingest.GetPipelineRequest; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; import org.opensearch.common.logging.DeprecationLogger; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestStatusToXContentListener; diff --git a/server/src/main/java/org/opensearch/rest/action/search/RestClearScrollAction.java b/server/src/main/java/org/opensearch/rest/action/search/RestClearScrollAction.java index b9f57feab4296..04cfaacef5e76 100644 --- a/server/src/main/java/org/opensearch/rest/action/search/RestClearScrollAction.java +++ b/server/src/main/java/org/opensearch/rest/action/search/RestClearScrollAction.java @@ -34,7 +34,7 @@ import org.opensearch.action.search.ClearScrollRequest; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestStatusToXContentListener; diff --git a/server/src/main/java/org/opensearch/rest/action/search/RestCountAction.java b/server/src/main/java/org/opensearch/rest/action/search/RestCountAction.java index 9ed818f2d7de2..acedd5a884596 100644 --- a/server/src/main/java/org/opensearch/rest/action/search/RestCountAction.java +++ b/server/src/main/java/org/opensearch/rest/action/search/RestCountAction.java @@ -36,7 +36,7 @@ import org.opensearch.action.search.SearchResponse; import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.index.query.QueryBuilder; import org.opensearch.rest.BaseRestHandler; diff --git a/server/src/main/java/org/opensearch/rest/action/search/RestCreatePitAction.java b/server/src/main/java/org/opensearch/rest/action/search/RestCreatePitAction.java index 9439670880015..2134087fc627b 100644 --- a/server/src/main/java/org/opensearch/rest/action/search/RestCreatePitAction.java +++ b/server/src/main/java/org/opensearch/rest/action/search/RestCreatePitAction.java @@ -11,8 +11,8 @@ import org.opensearch.action.search.CreatePitRequest; import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestStatusToXContentListener; diff --git a/server/src/main/java/org/opensearch/rest/action/search/RestExplainAction.java b/server/src/main/java/org/opensearch/rest/action/search/RestExplainAction.java index bf29e0edfd2ab..f63b54c0443d7 100644 --- a/server/src/main/java/org/opensearch/rest/action/search/RestExplainAction.java +++ b/server/src/main/java/org/opensearch/rest/action/search/RestExplainAction.java @@ -34,7 +34,7 @@ import org.opensearch.action.explain.ExplainRequest; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.index.query.QueryBuilder; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; diff --git a/server/src/main/java/org/opensearch/rest/action/search/RestGetSearchPipelineAction.java b/server/src/main/java/org/opensearch/rest/action/search/RestGetSearchPipelineAction.java index bcbf248afc5e8..f377988d457b3 100644 --- a/server/src/main/java/org/opensearch/rest/action/search/RestGetSearchPipelineAction.java +++ b/server/src/main/java/org/opensearch/rest/action/search/RestGetSearchPipelineAction.java @@ -10,7 +10,7 @@ import org.opensearch.action.search.GetSearchPipelineRequest; import org.opensearch.client.node.NodeClient; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.RestRequest; import org.opensearch.rest.action.RestStatusToXContentListener; diff --git a/server/src/main/java/org/opensearch/rest/action/search/RestMultiSearchAction.java b/server/src/main/java/org/opensearch/rest/action/search/RestMultiSearchAction.java index 7e206c2e5d15d..190818ce08d0d 100644 --- a/server/src/main/java/org/opensearch/rest/action/search/RestMultiSearchAction.java +++ b/server/src/main/java/org/opensearch/rest/action/search/RestMultiSearchAction.java @@ -38,13 +38,13 @@ import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.node.NodeClient; import org.opensearch.common.CheckedBiConsumer; -import org.opensearch.common.Strings; import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.collect.Tuple; import org.opensearch.common.io.stream.NamedWriteableRegistry; import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContent; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.common.xcontent.XContentType; diff --git a/server/src/main/java/org/opensearch/rest/action/search/RestSearchAction.java b/server/src/main/java/org/opensearch/rest/action/search/RestSearchAction.java index d624e1bf826b2..34a9ad6ed39eb 100644 --- a/server/src/main/java/org/opensearch/rest/action/search/RestSearchAction.java +++ b/server/src/main/java/org/opensearch/rest/action/search/RestSearchAction.java @@ -40,8 +40,8 @@ import org.opensearch.action.support.IndicesOptions; import org.opensearch.client.node.NodeClient; import org.opensearch.common.Booleans; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.NamedWriteableRegistry; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.index.query.QueryBuilder; import org.opensearch.rest.BaseRestHandler; diff --git a/server/src/main/java/org/opensearch/script/ScriptService.java b/server/src/main/java/org/opensearch/script/ScriptService.java index a7de3e2f354d3..11e72ee6a8b66 100644 --- a/server/src/main/java/org/opensearch/script/ScriptService.java +++ b/server/src/main/java/org/opensearch/script/ScriptService.java @@ -47,13 +47,13 @@ import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.service.ClusterManagerTaskThrottler; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.Strings; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Setting.Property; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.io.IOUtils; +import org.opensearch.core.common.Strings; import java.io.Closeable; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/DateHistogramInterval.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/DateHistogramInterval.java index b520741ae53d9..c49a2ac496768 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/DateHistogramInterval.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/DateHistogramInterval.java @@ -33,11 +33,11 @@ package org.opensearch.search.aggregations.bucket.histogram; import org.opensearch.common.Rounding; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.io.stream.Writeable; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.XContentBuilder; diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/DateIntervalWrapper.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/DateIntervalWrapper.java index 65c085b22e66d..222ae59600c65 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/DateIntervalWrapper.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/DateIntervalWrapper.java @@ -36,12 +36,12 @@ import org.opensearch.LegacyESVersion; import org.opensearch.common.Rounding; import org.opensearch.common.Rounding.DateTimeUnit; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.io.stream.Writeable; import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ObjectParser; import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.XContentBuilder; diff --git a/server/src/main/java/org/opensearch/search/aggregations/metrics/TopHitsAggregationBuilder.java b/server/src/main/java/org/opensearch/search/aggregations/metrics/TopHitsAggregationBuilder.java index b34e68f174dd2..4be0de116154b 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/metrics/TopHitsAggregationBuilder.java +++ b/server/src/main/java/org/opensearch/search/aggregations/metrics/TopHitsAggregationBuilder.java @@ -35,9 +35,9 @@ import org.opensearch.LegacyESVersion; import org.opensearch.common.Nullable; import org.opensearch.common.ParsingException; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.index.IndexSettings; diff --git a/server/src/main/java/org/opensearch/search/aggregations/pipeline/MovFnPipelineAggregationBuilder.java b/server/src/main/java/org/opensearch/search/aggregations/pipeline/MovFnPipelineAggregationBuilder.java index 670508ae7ef82..157b9bd958ff4 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/pipeline/MovFnPipelineAggregationBuilder.java +++ b/server/src/main/java/org/opensearch/search/aggregations/pipeline/MovFnPipelineAggregationBuilder.java @@ -33,10 +33,10 @@ package org.opensearch.search.aggregations.pipeline; import org.opensearch.LegacyESVersion; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.core.ParseField; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ConstructingObjectParser; import org.opensearch.core.xcontent.ObjectParser; import org.opensearch.core.xcontent.XContentBuilder; diff --git a/server/src/main/java/org/opensearch/search/aggregations/support/AggregationPath.java b/server/src/main/java/org/opensearch/search/aggregations/support/AggregationPath.java index 2bdb561879baa..4cfd2070211d7 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/support/AggregationPath.java +++ b/server/src/main/java/org/opensearch/search/aggregations/support/AggregationPath.java @@ -32,7 +32,7 @@ package org.opensearch.search.aggregations.support; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.search.aggregations.AggregationExecutionException; import org.opensearch.search.aggregations.Aggregator; import org.opensearch.search.aggregations.Aggregator.BucketComparator; diff --git a/server/src/main/java/org/opensearch/search/aggregations/support/MultiTermsValuesSourceConfig.java b/server/src/main/java/org/opensearch/search/aggregations/support/MultiTermsValuesSourceConfig.java index 4e6beefd1ee88..644c6796b3cfb 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/support/MultiTermsValuesSourceConfig.java +++ b/server/src/main/java/org/opensearch/search/aggregations/support/MultiTermsValuesSourceConfig.java @@ -9,9 +9,9 @@ package org.opensearch.search.aggregations.support; import org.opensearch.core.ParseField; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ObjectParser; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; diff --git a/server/src/main/java/org/opensearch/search/aggregations/support/MultiValuesSourceFieldConfig.java b/server/src/main/java/org/opensearch/search/aggregations/support/MultiValuesSourceFieldConfig.java index 65e5bff1d4280..1f469cdf82e75 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/support/MultiValuesSourceFieldConfig.java +++ b/server/src/main/java/org/opensearch/search/aggregations/support/MultiValuesSourceFieldConfig.java @@ -33,11 +33,11 @@ package org.opensearch.search.aggregations.support; import org.opensearch.LegacyESVersion; -import org.opensearch.common.Strings; import org.opensearch.common.TriFunction; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.core.ParseField; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ObjectParser; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.index.query.AbstractQueryBuilder; diff --git a/server/src/main/java/org/opensearch/search/builder/SearchSourceBuilder.java b/server/src/main/java/org/opensearch/search/builder/SearchSourceBuilder.java index 9bb07cbeefffc..ee47078a0d34c 100644 --- a/server/src/main/java/org/opensearch/search/builder/SearchSourceBuilder.java +++ b/server/src/main/java/org/opensearch/search/builder/SearchSourceBuilder.java @@ -38,20 +38,20 @@ import org.opensearch.common.Booleans; import org.opensearch.common.Nullable; import org.opensearch.common.util.FeatureFlags; -import org.opensearch.core.ParseField; import org.opensearch.common.ParsingException; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.io.stream.Writeable; import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.xcontent.XContentHelper; +import org.opensearch.common.xcontent.XContentType; +import org.opensearch.core.common.Strings; +import org.opensearch.core.ParseField; import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.ToXContentObject; import org.opensearch.core.xcontent.XContentBuilder; -import org.opensearch.common.xcontent.XContentHelper; import org.opensearch.core.xcontent.XContentParser; -import org.opensearch.common.xcontent.XContentType; import org.opensearch.index.query.QueryBuilder; import org.opensearch.index.query.QueryRewriteContext; import org.opensearch.index.query.Rewriteable; diff --git a/server/src/main/java/org/opensearch/search/collapse/CollapseBuilder.java b/server/src/main/java/org/opensearch/search/collapse/CollapseBuilder.java index fcad490a39bf2..dc83a023d5bf7 100644 --- a/server/src/main/java/org/opensearch/search/collapse/CollapseBuilder.java +++ b/server/src/main/java/org/opensearch/search/collapse/CollapseBuilder.java @@ -33,10 +33,10 @@ import org.opensearch.core.ParseField; import org.opensearch.common.ParsingException; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.io.stream.Writeable; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ObjectParser; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.ToXContentObject; diff --git a/server/src/main/java/org/opensearch/search/fetch/StoredFieldsContext.java b/server/src/main/java/org/opensearch/search/fetch/StoredFieldsContext.java index 2d42e7871b601..94df87201cf27 100644 --- a/server/src/main/java/org/opensearch/search/fetch/StoredFieldsContext.java +++ b/server/src/main/java/org/opensearch/search/fetch/StoredFieldsContext.java @@ -33,10 +33,10 @@ package org.opensearch.search.fetch; import org.opensearch.common.ParsingException; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.io.stream.Writeable; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.rest.RestRequest; diff --git a/server/src/main/java/org/opensearch/search/fetch/subphase/FetchSourceContext.java b/server/src/main/java/org/opensearch/search/fetch/subphase/FetchSourceContext.java index d5cbf6fe09636..4e4a582384d16 100644 --- a/server/src/main/java/org/opensearch/search/fetch/subphase/FetchSourceContext.java +++ b/server/src/main/java/org/opensearch/search/fetch/subphase/FetchSourceContext.java @@ -35,10 +35,10 @@ import org.opensearch.common.Booleans; import org.opensearch.core.ParseField; import org.opensearch.common.ParsingException; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.io.stream.Writeable; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ToXContentObject; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; diff --git a/server/src/main/java/org/opensearch/search/fetch/subphase/highlight/UnifiedHighlighter.java b/server/src/main/java/org/opensearch/search/fetch/subphase/highlight/UnifiedHighlighter.java index 5efaa7c9f766b..ce20fd0b74bb9 100644 --- a/server/src/main/java/org/opensearch/search/fetch/subphase/highlight/UnifiedHighlighter.java +++ b/server/src/main/java/org/opensearch/search/fetch/subphase/highlight/UnifiedHighlighter.java @@ -46,8 +46,8 @@ import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.CollectionUtil; import org.opensearch.common.CheckedSupplier; -import org.opensearch.common.Strings; import org.opensearch.common.text.Text; +import org.opensearch.core.common.Strings; import org.opensearch.index.mapper.DocumentMapper; import org.opensearch.index.mapper.IdFieldMapper; import org.opensearch.index.mapper.MappedFieldType; diff --git a/server/src/main/java/org/opensearch/search/internal/AliasFilter.java b/server/src/main/java/org/opensearch/search/internal/AliasFilter.java index b0d8b3cf47bec..d50911759aa44 100644 --- a/server/src/main/java/org/opensearch/search/internal/AliasFilter.java +++ b/server/src/main/java/org/opensearch/search/internal/AliasFilter.java @@ -32,10 +32,10 @@ package org.opensearch.search.internal; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.io.stream.Writeable; +import org.opensearch.core.common.Strings; import org.opensearch.index.query.QueryBuilder; import org.opensearch.index.query.QueryRewriteContext; import org.opensearch.index.query.Rewriteable; diff --git a/server/src/main/java/org/opensearch/search/internal/ShardSearchRequest.java b/server/src/main/java/org/opensearch/search/internal/ShardSearchRequest.java index 9a4c14c0e054c..3ffdcb0bebeff 100644 --- a/server/src/main/java/org/opensearch/search/internal/ShardSearchRequest.java +++ b/server/src/main/java/org/opensearch/search/internal/ShardSearchRequest.java @@ -44,12 +44,12 @@ import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.CheckedFunction; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.index.Index; import org.opensearch.index.query.BoolQueryBuilder; diff --git a/server/src/main/java/org/opensearch/search/query/QuerySearchRequest.java b/server/src/main/java/org/opensearch/search/query/QuerySearchRequest.java index ca74942decb50..6d1e977fa36e4 100644 --- a/server/src/main/java/org/opensearch/search/query/QuerySearchRequest.java +++ b/server/src/main/java/org/opensearch/search/query/QuerySearchRequest.java @@ -38,9 +38,9 @@ import org.opensearch.action.search.SearchShardTask; import org.opensearch.action.support.IndicesOptions; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import org.opensearch.search.dfs.AggregatedDfs; import org.opensearch.search.internal.ShardSearchContextId; import org.opensearch.search.internal.ShardSearchRequest; diff --git a/server/src/main/java/org/opensearch/search/slice/SliceBuilder.java b/server/src/main/java/org/opensearch/search/slice/SliceBuilder.java index 9f61a2c445b98..59bf5acc14d96 100644 --- a/server/src/main/java/org/opensearch/search/slice/SliceBuilder.java +++ b/server/src/main/java/org/opensearch/search/slice/SliceBuilder.java @@ -133,7 +133,7 @@ public void writeTo(StreamOutput out) throws IOException { } private SliceBuilder setField(String field) { - if (Strings.isEmpty(field)) { + if (org.opensearch.core.common.Strings.isEmpty(field)) { throw new IllegalArgumentException("field name is null or empty"); } this.field = field; diff --git a/server/src/main/java/org/opensearch/search/suggest/phrase/DirectCandidateGeneratorBuilder.java b/server/src/main/java/org/opensearch/search/suggest/phrase/DirectCandidateGeneratorBuilder.java index afd46f1a8164c..dc75088d4c85b 100644 --- a/server/src/main/java/org/opensearch/search/suggest/phrase/DirectCandidateGeneratorBuilder.java +++ b/server/src/main/java/org/opensearch/search/suggest/phrase/DirectCandidateGeneratorBuilder.java @@ -40,7 +40,7 @@ import org.apache.lucene.search.spell.StringDistance; import org.apache.lucene.search.spell.SuggestMode; import org.apache.lucene.util.automaton.LevenshteinAutomata; -import org.opensearch.ExceptionsHelper; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.core.ParseField; import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; @@ -510,7 +510,7 @@ public String toString() { toXContent(builder, EMPTY_PARAMS); return Strings.toString(builder); } catch (Exception e) { - return "{ \"error\" : \"" + ExceptionsHelper.detailedMessage(e) + "\"}"; + return "{ \"error\" : \"" + BaseExceptionsHelper.detailedMessage(e) + "\"}"; } } diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java index b8afdfb0c0e8b..93bb1db39df73 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotsService.java @@ -651,7 +651,7 @@ public ClusterState execute(ClusterState currentState) { "No indices in the source snapshot [" + sourceSnapshotId + "] matched requested pattern [" - + Strings.arrayToCommaDelimitedString(request.indices()) + + org.opensearch.core.common.Strings.arrayToCommaDelimitedString(request.indices()) + "]" ); } @@ -940,7 +940,7 @@ private static void validate(String repositoryName, String snapshotName, Cluster } private static void validate(final String repositoryName, final String snapshotName) { - if (Strings.hasLength(snapshotName) == false) { + if (org.opensearch.core.common.Strings.hasLength(snapshotName) == false) { throw new InvalidSnapshotNameException(repositoryName, snapshotName, "cannot be empty"); } if (snapshotName.contains(" ")) { @@ -2185,7 +2185,7 @@ public void deleteSnapshots(final DeleteSnapshotRequest request, final ActionLis logger.info( () -> new ParameterizedMessage( "deleting snapshots [{}] from repository [{}]", - Strings.arrayToCommaDelimitedString(snapshotNames), + org.opensearch.core.common.Strings.arrayToCommaDelimitedString(snapshotNames), repoName ) ); diff --git a/server/src/main/java/org/opensearch/tasks/TaskCancellationService.java b/server/src/main/java/org/opensearch/tasks/TaskCancellationService.java index 4073a48bcde0d..b7165cf0bba82 100644 --- a/server/src/main/java/org/opensearch/tasks/TaskCancellationService.java +++ b/server/src/main/java/org/opensearch/tasks/TaskCancellationService.java @@ -34,7 +34,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.opensearch.ExceptionsHelper; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.LegacyESVersion; import org.opensearch.OpenSearchSecurityException; import org.opensearch.action.ActionListener; @@ -157,7 +157,7 @@ public void handleResponse(TransportResponse.Empty response) { @Override public void handleException(TransportException exp) { - assert ExceptionsHelper.unwrapCause(exp) instanceof OpenSearchSecurityException == false; + assert BaseExceptionsHelper.unwrapCause(exp) instanceof OpenSearchSecurityException == false; logger.warn("Cannot send ban for tasks with the parent [{}] to the node [{}]", taskId, node); groupedListener.onFailure(exp); } @@ -173,7 +173,7 @@ private void removeBanOnNodes(CancellableTask task, Collection ch transportService.sendRequest(node, BAN_PARENT_ACTION_NAME, request, new EmptyTransportResponseHandler(ThreadPool.Names.SAME) { @Override public void handleException(TransportException exp) { - assert ExceptionsHelper.unwrapCause(exp) instanceof OpenSearchSecurityException == false; + assert BaseExceptionsHelper.unwrapCause(exp) instanceof OpenSearchSecurityException == false; logger.info("failed to remove the parent ban for task {} on node {}", request.parentTaskId, node); } }); diff --git a/server/src/main/java/org/opensearch/tasks/TaskId.java b/server/src/main/java/org/opensearch/tasks/TaskId.java index 6e51f73659053..e1f198dba14ad 100644 --- a/server/src/main/java/org/opensearch/tasks/TaskId.java +++ b/server/src/main/java/org/opensearch/tasks/TaskId.java @@ -71,7 +71,7 @@ private TaskId() { } public TaskId(String taskId) { - if (Strings.hasLength(taskId) && "unset".equals(taskId) == false) { + if (org.opensearch.core.common.Strings.hasLength(taskId) && "unset".equals(taskId) == false) { String[] s = Strings.split(taskId, ":"); if (s == null || s.length != 2) { throw new IllegalArgumentException("malformed task id " + taskId); diff --git a/server/src/main/java/org/opensearch/tasks/TaskManager.java b/server/src/main/java/org/opensearch/tasks/TaskManager.java index 6b00a3038c4bf..e402aa88d11a8 100644 --- a/server/src/main/java/org/opensearch/tasks/TaskManager.java +++ b/server/src/main/java/org/opensearch/tasks/TaskManager.java @@ -37,6 +37,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.core.Assertions; import org.opensearch.ExceptionsHelper; import org.opensearch.OpenSearchException; @@ -369,7 +370,7 @@ public void storeResult(Task task, Exception e try { taskResult = task.result(localNode, error); } catch (IOException ex) { - logger.warn(() -> new ParameterizedMessage("couldn't store error {}", ExceptionsHelper.detailedMessage(error)), ex); + logger.warn(() -> new ParameterizedMessage("couldn't store error {}", BaseExceptionsHelper.detailedMessage(error)), ex); listener.onFailure(ex); return; } @@ -381,7 +382,7 @@ public void onResponse(Void aVoid) { @Override public void onFailure(Exception e) { - logger.warn(() -> new ParameterizedMessage("couldn't store error {}", ExceptionsHelper.detailedMessage(error)), e); + logger.warn(() -> new ParameterizedMessage("couldn't store error {}", BaseExceptionsHelper.detailedMessage(error)), e); listener.onFailure(e); } }); diff --git a/server/src/main/java/org/opensearch/tasks/TaskResult.java b/server/src/main/java/org/opensearch/tasks/TaskResult.java index 2387de173c924..11ea9a9a82861 100644 --- a/server/src/main/java/org/opensearch/tasks/TaskResult.java +++ b/server/src/main/java/org/opensearch/tasks/TaskResult.java @@ -31,7 +31,7 @@ package org.opensearch.tasks; -import org.opensearch.OpenSearchException; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.client.Requests; import org.opensearch.common.Nullable; import org.opensearch.core.ParseField; @@ -240,7 +240,7 @@ public int hashCode() { private static BytesReference toXContent(Exception error) throws IOException { try (XContentBuilder builder = XContentFactory.contentBuilder(Requests.INDEX_CONTENT_TYPE)) { builder.startObject(); - OpenSearchException.generateThrowableXContent(builder, ToXContent.EMPTY_PARAMS, error); + BaseExceptionsHelper.generateThrowableXContent(builder, ToXContent.EMPTY_PARAMS, error); builder.endObject(); return BytesReference.bytes(builder); } diff --git a/server/src/main/java/org/opensearch/tasks/TaskResultsService.java b/server/src/main/java/org/opensearch/tasks/TaskResultsService.java index 1feb115cb585a..329fc55431bc8 100644 --- a/server/src/main/java/org/opensearch/tasks/TaskResultsService.java +++ b/server/src/main/java/org/opensearch/tasks/TaskResultsService.java @@ -34,7 +34,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; -import org.opensearch.ExceptionsHelper; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.OpenSearchException; import org.opensearch.ResourceAlreadyExistsException; import org.opensearch.action.ActionListener; @@ -126,7 +126,7 @@ public void onResponse(CreateIndexResponse result) { @Override public void onFailure(Exception e) { - if (ExceptionsHelper.unwrapCause(e) instanceof ResourceAlreadyExistsException) { + if (BaseExceptionsHelper.unwrapCause(e) instanceof ResourceAlreadyExistsException) { // we have the index, do it try { doStoreResult(taskResult, listener); diff --git a/server/src/main/java/org/opensearch/transport/ProxyConnectionStrategy.java b/server/src/main/java/org/opensearch/transport/ProxyConnectionStrategy.java index 42b84b947e766..710f40bd19a53 100644 --- a/server/src/main/java/org/opensearch/transport/ProxyConnectionStrategy.java +++ b/server/src/main/java/org/opensearch/transport/ProxyConnectionStrategy.java @@ -39,7 +39,6 @@ import org.opensearch.cluster.ClusterName; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodeRole; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.io.stream.Writeable; @@ -47,6 +46,7 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.transport.TransportAddress; import org.opensearch.common.util.concurrent.CountDown; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/transport/RemoteConnectionStrategy.java b/server/src/main/java/org/opensearch/transport/RemoteConnectionStrategy.java index 15aebd5f71123..902b8f8367336 100644 --- a/server/src/main/java/org/opensearch/transport/RemoteConnectionStrategy.java +++ b/server/src/main/java/org/opensearch/transport/RemoteConnectionStrategy.java @@ -39,12 +39,12 @@ import org.opensearch.action.ActionListener; import org.opensearch.action.support.ContextPreservingActionListener; import org.opensearch.cluster.node.DiscoveryNode; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.Writeable; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.AbstractRunnable; +import org.opensearch.core.common.Strings; import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; import org.opensearch.threadpool.ThreadPool; diff --git a/server/src/main/java/org/opensearch/transport/SniffConnectionStrategy.java b/server/src/main/java/org/opensearch/transport/SniffConnectionStrategy.java index a9bba4fe44874..981df456e9394 100644 --- a/server/src/main/java/org/opensearch/transport/SniffConnectionStrategy.java +++ b/server/src/main/java/org/opensearch/transport/SniffConnectionStrategy.java @@ -44,7 +44,6 @@ import org.opensearch.cluster.node.DiscoveryNodeRole; import org.opensearch.common.Booleans; import org.opensearch.common.SetOnce; -import org.opensearch.common.Strings; import org.opensearch.common.UUIDs; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; @@ -53,6 +52,7 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.transport.TransportAddress; import org.opensearch.common.util.concurrent.ThreadContext; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.common.util.io.IOUtils; import org.opensearch.threadpool.ThreadPool; diff --git a/server/src/main/java/org/opensearch/transport/TcpTransport.java b/server/src/main/java/org/opensearch/transport/TcpTransport.java index 7aedebb946e58..07bf9162d9355 100644 --- a/server/src/main/java/org/opensearch/transport/TcpTransport.java +++ b/server/src/main/java/org/opensearch/transport/TcpTransport.java @@ -42,7 +42,6 @@ import org.opensearch.action.support.ThreadedActionListener; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.common.Booleans; -import org.opensearch.common.Strings; import org.opensearch.common.breaker.CircuitBreaker; import org.opensearch.common.bytes.BytesArray; import org.opensearch.common.bytes.BytesReference; @@ -66,6 +65,7 @@ import org.opensearch.common.util.PageCacheRecycler; import org.opensearch.common.util.concurrent.ConcurrentCollections; import org.opensearch.common.util.concurrent.CountDown; +import org.opensearch.core.common.Strings; import org.opensearch.indices.breaker.CircuitBreakerService; import org.opensearch.monitor.jvm.JvmInfo; import org.opensearch.node.Node; diff --git a/server/src/main/java/org/opensearch/transport/TransportService.java b/server/src/main/java/org/opensearch/transport/TransportService.java index 27493176465b8..43630203091c7 100644 --- a/server/src/main/java/org/opensearch/transport/TransportService.java +++ b/server/src/main/java/org/opensearch/transport/TransportService.java @@ -44,7 +44,6 @@ import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodes; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.component.AbstractLifecycleComponent; import org.opensearch.common.geo.GeoPoint; import org.opensearch.common.io.stream.StreamInput; @@ -59,9 +58,10 @@ import org.opensearch.common.transport.TransportAddress; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.AbstractRunnable; -import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; import org.opensearch.common.util.concurrent.ThreadContext; import org.opensearch.common.util.io.IOUtils; +import org.opensearch.core.common.Strings; +import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; import org.opensearch.node.NodeClosedException; import org.opensearch.node.ReportingService; import org.opensearch.script.JodaCompatibleZonedDateTime; diff --git a/server/src/test/java/org/opensearch/ExceptionsHelperTests.java b/server/src/test/java/org/opensearch/ExceptionsHelperTests.java index 4c65eadf46e89..395c5ec14c8a3 100644 --- a/server/src/test/java/org/opensearch/ExceptionsHelperTests.java +++ b/server/src/test/java/org/opensearch/ExceptionsHelperTests.java @@ -114,9 +114,9 @@ public void testStatus() { } public void testSummaryMessage() { - assertThat(ExceptionsHelper.summaryMessage(new IllegalArgumentException("illegal")), equalTo("Invalid argument")); - assertThat(ExceptionsHelper.summaryMessage(new JsonParseException(null, "illegal")), equalTo("Failed to parse JSON")); - assertThat(ExceptionsHelper.summaryMessage(new OpenSearchRejectedExecutionException("rejected")), equalTo("Too many requests")); + assertThat(BaseExceptionsHelper.summaryMessage(new IllegalArgumentException("illegal")), equalTo("Invalid argument")); + assertThat(BaseExceptionsHelper.summaryMessage(new JsonParseException(null, "illegal")), equalTo("Failed to parse JSON")); + assertThat(BaseExceptionsHelper.summaryMessage(new OpenSearchRejectedExecutionException("rejected")), equalTo("Too many requests")); } public void testGroupBy() { diff --git a/server/src/test/java/org/opensearch/OpenSearchExceptionTests.java b/server/src/test/java/org/opensearch/OpenSearchExceptionTests.java index a0bd8202abbbe..d584e9130e20e 100644 --- a/server/src/test/java/org/opensearch/OpenSearchExceptionTests.java +++ b/server/src/test/java/org/opensearch/OpenSearchExceptionTests.java @@ -127,9 +127,9 @@ public void testGuessRootCause() { "foo", new OpenSearchException("bar", new IndexNotFoundException("foo", new RuntimeException("foobar"))) ); - OpenSearchException[] rootCauses = exception.guessRootCauses(); + BaseOpenSearchException[] rootCauses = exception.guessRootCauses(); assertEquals(rootCauses.length, 1); - assertEquals(OpenSearchException.getExceptionName(rootCauses[0]), "index_not_found_exception"); + assertEquals(BaseExceptionsHelper.getExceptionName(rootCauses[0]), "index_not_found_exception"); assertEquals("no such index [foo]", rootCauses[0].getMessage()); ShardSearchFailure failure = new ShardSearchFailure( new ParsingException(1, 2, "foobar", null), @@ -147,14 +147,14 @@ public void testGuessRootCause() { if (randomBoolean()) { rootCauses = (randomBoolean() ? new RemoteTransportException("remoteboom", ex) : ex).guessRootCauses(); } else { - rootCauses = OpenSearchException.guessRootCauses(randomBoolean() ? new RemoteTransportException("remoteboom", ex) : ex); + rootCauses = BaseOpenSearchException.guessRootCauses(randomBoolean() ? new RemoteTransportException("remoteboom", ex) : ex); } - assertEquals("parsing_exception", OpenSearchException.getExceptionName(rootCauses[0])); + assertEquals("parsing_exception", BaseExceptionsHelper.getExceptionName(rootCauses[0])); assertEquals("foobar", rootCauses[0].getMessage()); OpenSearchException oneLevel = new OpenSearchException("foo", new RuntimeException("foobar")); rootCauses = oneLevel.guessRootCauses(); - assertEquals("exception", OpenSearchException.getExceptionName(rootCauses[0])); + assertEquals("exception", BaseExceptionsHelper.getExceptionName(rootCauses[0])); assertEquals("foo", rootCauses[0].getMessage()); } { @@ -175,32 +175,32 @@ public void testGuessRootCause() { "all shards failed", new ShardSearchFailure[] { failure, failure1, failure2 } ); - final OpenSearchException[] rootCauses = ex.guessRootCauses(); + final BaseOpenSearchException[] rootCauses = ex.guessRootCauses(); assertEquals(rootCauses.length, 2); - assertEquals(OpenSearchException.getExceptionName(rootCauses[0]), "parsing_exception"); + assertEquals(BaseExceptionsHelper.getExceptionName(rootCauses[0]), "parsing_exception"); assertEquals(rootCauses[0].getMessage(), "foobar"); assertEquals(1, ((ParsingException) rootCauses[0]).getLineNumber()); assertEquals(2, ((ParsingException) rootCauses[0]).getColumnNumber()); - assertEquals("query_shard_exception", OpenSearchException.getExceptionName(rootCauses[1])); - assertEquals("foo1", rootCauses[1].getIndex().getName()); + assertEquals("query_shard_exception", BaseExceptionsHelper.getExceptionName(rootCauses[1])); + assertEquals("foo1", rootCauses[1].getIndexName()); assertEquals("foobar", rootCauses[1].getMessage()); } { - final OpenSearchException[] foobars = OpenSearchException.guessRootCauses(new IllegalArgumentException("foobar")); + final BaseOpenSearchException[] foobars = BaseOpenSearchException.guessRootCauses(new IllegalArgumentException("foobar")); assertEquals(foobars.length, 1); - assertThat(foobars[0], instanceOf(OpenSearchException.class)); + assertThat(foobars[0], instanceOf(BaseOpenSearchException.class)); assertEquals("foobar", foobars[0].getMessage()); assertEquals(IllegalArgumentException.class, foobars[0].getCause().getClass()); assertEquals("illegal_argument_exception", foobars[0].getExceptionName()); } { - final OpenSearchException[] foobars = OpenSearchException.guessRootCauses( + final BaseOpenSearchException[] foobars = BaseOpenSearchException.guessRootCauses( new RemoteTransportException("abc", new IllegalArgumentException("foobar")) ); assertEquals(foobars.length, 1); - assertThat(foobars[0], instanceOf(OpenSearchException.class)); + assertThat(foobars[0], instanceOf(BaseOpenSearchException.class)); assertEquals("foobar", foobars[0].getMessage()); assertEquals(IllegalArgumentException.class, foobars[0].getCause().getClass()); assertEquals("illegal_argument_exception", foobars[0].getExceptionName()); @@ -209,9 +209,9 @@ public void testGuessRootCause() { { XContentParseException inner = new XContentParseException(null, "inner"); XContentParseException outer = new XContentParseException(null, "outer", inner); - final OpenSearchException[] causes = OpenSearchException.guessRootCauses(outer); + final BaseOpenSearchException[] causes = BaseOpenSearchException.guessRootCauses(outer); assertEquals(causes.length, 1); - assertThat(causes[0], instanceOf(OpenSearchException.class)); + assertThat(causes[0], instanceOf(BaseOpenSearchException.class)); assertEquals("inner", causes[0].getMessage()); assertEquals("x_content_parse_exception", causes[0].getExceptionName()); } @@ -219,9 +219,9 @@ public void testGuessRootCause() { { OpenSearchException inner = new OpenSearchException("inner"); XContentParseException outer = new XContentParseException(null, "outer", inner); - final OpenSearchException[] causes = OpenSearchException.guessRootCauses(outer); + final BaseOpenSearchException[] causes = BaseOpenSearchException.guessRootCauses(outer); assertEquals(causes.length, 1); - assertThat(causes[0], instanceOf(OpenSearchException.class)); + assertThat(causes[0], instanceOf(BaseOpenSearchException.class)); assertEquals("inner", causes[0].getMessage()); assertEquals("exception", causes[0].getExceptionName()); } @@ -409,7 +409,7 @@ public void testToXContent() throws IOException { // Test the same exception but with the "rest.exception.stacktrace.skip" parameter disabled: the stack_trace must be present // in the JSON. Since the stack can be large, it only checks the beginning of the JSON. ToXContent.Params params = new ToXContent.MapParams( - Collections.singletonMap(OpenSearchException.REST_EXCEPTION_SKIP_STACK_TRACE, "false") + Collections.singletonMap(BaseExceptionsHelper.REST_EXCEPTION_SKIP_STACK_TRACE, "false") ); String actual; try (XContentBuilder builder = XContentBuilder.builder(XContentType.JSON.xContent())) { @@ -451,7 +451,7 @@ public void testGenerateThrowableToXContent() throws IOException { OpenSearchException ex = new RemoteTransportException("foobar", new FileNotFoundException("foo not found")); String toXContentString = Strings.toString(XContentType.JSON, ex); String throwableString = Strings.toString(XContentType.JSON, (builder, params) -> { - OpenSearchException.generateThrowableXContent(builder, params, ex); + BaseExceptionsHelper.generateThrowableXContent(builder, params, ex); return builder; }); @@ -740,7 +740,7 @@ public void testThrowableToAndFromXContent() throws IOException { } BytesReference throwableBytes = toShuffledXContent((builder, params) -> { - OpenSearchException.generateThrowableXContent(builder, params, throwable); + BaseExceptionsHelper.generateThrowableXContent(builder, params, throwable); return builder; }, xContent.mediaType(), ToXContent.EMPTY_PARAMS, randomBoolean()); @@ -756,7 +756,7 @@ public void testThrowableToAndFromXContent() throws IOException { if (suppressedCount > 0) { XContentBuilder builder = XContentBuilder.builder(xContent); builder.startObject(); - OpenSearchException.generateThrowableXContent(builder, ToXContent.EMPTY_PARAMS, throwable); + BaseExceptionsHelper.generateThrowableXContent(builder, ToXContent.EMPTY_PARAMS, throwable); builder.endObject(); throwableBytes = BytesReference.bytes(builder); try (XContentParser parser = createParser(xContent, throwableBytes)) { @@ -772,7 +772,7 @@ public void testUnknownFailureToAndFromXContent() throws IOException { BytesReference failureBytes = toShuffledXContent((builder, params) -> { // Prints a null failure using generateFailureXContent() - OpenSearchException.generateFailureXContent(builder, params, null, randomBoolean()); + BaseOpenSearchException.generateFailureXContent(builder, params, null, randomBoolean()); return builder; }, xContent.mediaType(), ToXContent.EMPTY_PARAMS, randomBoolean()); @@ -796,7 +796,7 @@ public void testFailureToAndFromXContentWithNoDetails() throws IOException { final Exception failure = (Exception) randomExceptions().v1(); BytesReference failureBytes = toShuffledXContent((builder, params) -> { - OpenSearchException.generateFailureXContent(builder, params, failure, false); + BaseOpenSearchException.generateFailureXContent(builder, params, failure, false); return builder; }, xContent.mediaType(), ToXContent.EMPTY_PARAMS, randomBoolean()); @@ -814,8 +814,8 @@ public void testFailureToAndFromXContentWithNoDetails() throws IOException { } assertNotNull(parsedFailure); - String reason = ExceptionsHelper.summaryMessage(failure); - assertEquals(OpenSearchException.buildMessage("exception", reason, null), parsedFailure.getMessage()); + String reason = BaseExceptionsHelper.summaryMessage(failure); + assertEquals(BaseOpenSearchException.buildMessage("exception", reason, null), parsedFailure.getMessage()); assertEquals(0, parsedFailure.getHeaders().size()); assertEquals(0, parsedFailure.getMetadata().size()); assertNull(parsedFailure.getCause()); @@ -950,7 +950,7 @@ public void testFailureToAndFromXContentWithDetails() throws IOException { Exception finalFailure = failure; BytesReference failureBytes = toShuffledXContent((builder, params) -> { - OpenSearchException.generateFailureXContent(builder, params, finalFailure, true); + BaseOpenSearchException.generateFailureXContent(builder, params, finalFailure, true); return builder; }, xContent.mediaType(), ToXContent.EMPTY_PARAMS, randomBoolean()); @@ -983,7 +983,7 @@ private static void assertToXContentAsJson(ToXContent e, String expectedJson) th private static void assertExceptionAsJson(Exception e, String expectedJson) throws IOException { assertToXContentAsJson((builder, params) -> { - OpenSearchException.generateThrowableXContent(builder, params, e); + BaseExceptionsHelper.generateThrowableXContent(builder, params, e); return builder; }, expectedJson); } diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/configuration/AddVotingConfigExclusionsRequestTests.java b/server/src/test/java/org/opensearch/action/admin/cluster/configuration/AddVotingConfigExclusionsRequestTests.java index 54b51944999d4..529ed8c1b62cc 100644 --- a/server/src/test/java/org/opensearch/action/admin/cluster/configuration/AddVotingConfigExclusionsRequestTests.java +++ b/server/src/test/java/org/opensearch/action/admin/cluster/configuration/AddVotingConfigExclusionsRequestTests.java @@ -40,8 +40,8 @@ import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodeRole; import org.opensearch.cluster.node.DiscoveryNodes.Builder; -import org.opensearch.common.Strings; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import org.opensearch.test.OpenSearchTestCase; import java.io.IOException; diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/configuration/TransportAddVotingConfigExclusionsActionTests.java b/server/src/test/java/org/opensearch/action/admin/cluster/configuration/TransportAddVotingConfigExclusionsActionTests.java index 40a041e9b94e3..56c9f70dc5575 100644 --- a/server/src/test/java/org/opensearch/action/admin/cluster/configuration/TransportAddVotingConfigExclusionsActionTests.java +++ b/server/src/test/java/org/opensearch/action/admin/cluster/configuration/TransportAddVotingConfigExclusionsActionTests.java @@ -49,12 +49,12 @@ import org.opensearch.cluster.node.DiscoveryNodes.Builder; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.SetOnce; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.ThreadContext; +import org.opensearch.core.common.Strings; import org.opensearch.test.OpenSearchTestCase; import org.opensearch.test.transport.MockTransport; import org.opensearch.threadpool.TestThreadPool; diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/configuration/VotingConfigExclusionsHelperTests.java b/server/src/test/java/org/opensearch/action/admin/cluster/configuration/VotingConfigExclusionsHelperTests.java index f33781064345d..55528cfb71a6a 100644 --- a/server/src/test/java/org/opensearch/action/admin/cluster/configuration/VotingConfigExclusionsHelperTests.java +++ b/server/src/test/java/org/opensearch/action/admin/cluster/configuration/VotingConfigExclusionsHelperTests.java @@ -17,8 +17,8 @@ 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.unit.TimeValue; +import org.opensearch.core.common.Strings; import org.opensearch.test.OpenSearchTestCase; import java.util.Set; diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/health/ClusterHealthRequestTests.java b/server/src/test/java/org/opensearch/action/admin/cluster/health/ClusterHealthRequestTests.java index db1497f8dee98..9844cd8be2e34 100644 --- a/server/src/test/java/org/opensearch/action/admin/cluster/health/ClusterHealthRequestTests.java +++ b/server/src/test/java/org/opensearch/action/admin/cluster/health/ClusterHealthRequestTests.java @@ -37,9 +37,9 @@ import org.opensearch.action.support.IndicesOptions; import org.opensearch.cluster.health.ClusterHealthStatus; import org.opensearch.common.Priority; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.core.common.Strings; import org.opensearch.test.OpenSearchTestCase; import org.opensearch.test.VersionUtils; diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/shards/ClusterSearchShardsResponseTests.java b/server/src/test/java/org/opensearch/action/admin/cluster/shards/ClusterSearchShardsResponseTests.java index 3c2e34ebb9ed5..fe82527fde3a1 100644 --- a/server/src/test/java/org/opensearch/action/admin/cluster/shards/ClusterSearchShardsResponseTests.java +++ b/server/src/test/java/org/opensearch/action/admin/cluster/shards/ClusterSearchShardsResponseTests.java @@ -37,13 +37,13 @@ import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.ShardRoutingState; import org.opensearch.cluster.routing.TestShardRouting; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.io.stream.NamedWriteableAwareStreamInput; import org.opensearch.common.io.stream.NamedWriteableRegistry; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.settings.Settings; import org.opensearch.common.transport.TransportAddress; +import org.opensearch.core.common.Strings; import org.opensearch.index.query.RandomQueryBuilder; import org.opensearch.index.shard.ShardId; import org.opensearch.search.SearchModule; diff --git a/server/src/test/java/org/opensearch/action/admin/indices/datastream/DeleteDataStreamRequestTests.java b/server/src/test/java/org/opensearch/action/admin/indices/datastream/DeleteDataStreamRequestTests.java index 6284903f0bf3f..008046aa83dfe 100644 --- a/server/src/test/java/org/opensearch/action/admin/indices/datastream/DeleteDataStreamRequestTests.java +++ b/server/src/test/java/org/opensearch/action/admin/indices/datastream/DeleteDataStreamRequestTests.java @@ -41,10 +41,10 @@ import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.metadata.MetadataDeleteIndexService; -import org.opensearch.common.Strings; import org.opensearch.common.collect.Tuple; import org.opensearch.common.io.stream.Writeable; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; import org.opensearch.index.Index; import org.opensearch.snapshots.Snapshot; import org.opensearch.snapshots.SnapshotId; diff --git a/server/src/test/java/org/opensearch/action/admin/indices/resolve/ResolveIndexResponseTests.java b/server/src/test/java/org/opensearch/action/admin/indices/resolve/ResolveIndexResponseTests.java index a9b99df5c4ce3..c792ed0e0439f 100644 --- a/server/src/test/java/org/opensearch/action/admin/indices/resolve/ResolveIndexResponseTests.java +++ b/server/src/test/java/org/opensearch/action/admin/indices/resolve/ResolveIndexResponseTests.java @@ -36,8 +36,8 @@ import org.opensearch.action.admin.indices.resolve.ResolveIndexAction.ResolvedDataStream; import org.opensearch.action.admin.indices.resolve.ResolveIndexAction.ResolvedIndex; import org.opensearch.action.admin.indices.resolve.ResolveIndexAction.Response; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.Writeable; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ConstructingObjectParser; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.test.AbstractSerializingTestCase; diff --git a/server/src/test/java/org/opensearch/action/admin/indices/resolve/ResolveIndexTests.java b/server/src/test/java/org/opensearch/action/admin/indices/resolve/ResolveIndexTests.java index 0ad31fec57472..8fbe46be856ba 100644 --- a/server/src/test/java/org/opensearch/action/admin/indices/resolve/ResolveIndexTests.java +++ b/server/src/test/java/org/opensearch/action/admin/indices/resolve/ResolveIndexTests.java @@ -45,10 +45,10 @@ import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.IndexNameExpressionResolver; import org.opensearch.cluster.metadata.Metadata; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; import org.opensearch.common.time.DateFormatter; import org.opensearch.common.util.concurrent.ThreadContext; +import org.opensearch.core.common.Strings; import org.opensearch.test.OpenSearchTestCase; import java.time.Clock; diff --git a/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionTookTests.java b/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionTookTests.java index 713b506f14299..2361b69e9b82c 100644 --- a/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionTookTests.java +++ b/server/src/test/java/org/opensearch/action/bulk/TransportBulkActionTookTests.java @@ -48,11 +48,11 @@ import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodeRole; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.concurrent.AtomicArray; import org.opensearch.common.util.concurrent.ThreadContext; import org.opensearch.common.xcontent.XContentType; +import org.opensearch.core.common.Strings; import org.opensearch.index.IndexNotFoundException; import org.opensearch.index.IndexingPressureService; import org.opensearch.indices.SystemIndices; diff --git a/server/src/test/java/org/opensearch/action/search/CanMatchPreFilterSearchPhaseTests.java b/server/src/test/java/org/opensearch/action/search/CanMatchPreFilterSearchPhaseTests.java index cb2ac328002c5..2e3ff166a6a53 100644 --- a/server/src/test/java/org/opensearch/action/search/CanMatchPreFilterSearchPhaseTests.java +++ b/server/src/test/java/org/opensearch/action/search/CanMatchPreFilterSearchPhaseTests.java @@ -38,8 +38,8 @@ import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.routing.GroupShardsIterator; -import org.opensearch.common.Strings; import org.opensearch.common.util.concurrent.OpenSearchExecutors; +import org.opensearch.core.common.Strings; import org.opensearch.index.shard.ShardId; import org.opensearch.search.SearchPhaseResult; import org.opensearch.search.SearchService; diff --git a/server/src/test/java/org/opensearch/action/search/MultiSearchResponseTests.java b/server/src/test/java/org/opensearch/action/search/MultiSearchResponseTests.java index 4a0857244021b..e26a3cb4731da 100644 --- a/server/src/test/java/org/opensearch/action/search/MultiSearchResponseTests.java +++ b/server/src/test/java/org/opensearch/action/search/MultiSearchResponseTests.java @@ -32,7 +32,7 @@ package org.opensearch.action.search; import org.opensearch.OpenSearchException; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.search.internal.InternalSearchResponse; diff --git a/server/src/test/java/org/opensearch/action/search/SearchAsyncActionTests.java b/server/src/test/java/org/opensearch/action/search/SearchAsyncActionTests.java index 277f2f1dee0bf..cf838682aa717 100644 --- a/server/src/test/java/org/opensearch/action/search/SearchAsyncActionTests.java +++ b/server/src/test/java/org/opensearch/action/search/SearchAsyncActionTests.java @@ -40,9 +40,9 @@ import org.opensearch.cluster.routing.RecoverySource; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.UnassignedInfo; -import org.opensearch.common.Strings; import org.opensearch.common.UUIDs; import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.core.common.Strings; import org.opensearch.index.Index; import org.opensearch.index.shard.ShardId; import org.opensearch.search.SearchPhaseResult; diff --git a/server/src/test/java/org/opensearch/action/search/SearchPhaseControllerTests.java b/server/src/test/java/org/opensearch/action/search/SearchPhaseControllerTests.java index abcf1efe56122..a007fe0fb44f8 100644 --- a/server/src/test/java/org/opensearch/action/search/SearchPhaseControllerTests.java +++ b/server/src/test/java/org/opensearch/action/search/SearchPhaseControllerTests.java @@ -45,7 +45,6 @@ import org.apache.lucene.search.grouping.CollapseTopFieldDocs; import org.apache.lucene.util.BytesRef; import org.opensearch.action.OriginalIndices; -import org.opensearch.common.Strings; import org.opensearch.common.UUIDs; import org.opensearch.common.breaker.CircuitBreaker; import org.opensearch.common.breaker.CircuitBreakingException; @@ -59,6 +58,7 @@ import org.opensearch.common.util.concurrent.AtomicArray; import org.opensearch.common.util.concurrent.OpenSearchExecutors; import org.opensearch.common.util.concurrent.OpenSearchThreadPoolExecutor; +import org.opensearch.core.common.Strings; import org.opensearch.index.shard.ShardId; import org.opensearch.search.DocValueFormat; import org.opensearch.search.SearchHit; diff --git a/server/src/test/java/org/opensearch/action/search/SearchQueryThenFetchAsyncActionTests.java b/server/src/test/java/org/opensearch/action/search/SearchQueryThenFetchAsyncActionTests.java index 4a23c4ec18951..4e351e1424cd0 100644 --- a/server/src/test/java/org/opensearch/action/search/SearchQueryThenFetchAsyncActionTests.java +++ b/server/src/test/java/org/opensearch/action/search/SearchQueryThenFetchAsyncActionTests.java @@ -41,12 +41,12 @@ import org.opensearch.action.OriginalIndices; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.routing.GroupShardsIterator; -import org.opensearch.common.Strings; import org.opensearch.common.breaker.CircuitBreaker; import org.opensearch.common.breaker.NoopCircuitBreaker; import org.opensearch.common.lucene.search.TopDocsAndMaxScore; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.OpenSearchExecutors; +import org.opensearch.core.common.Strings; import org.opensearch.index.shard.ShardId; import org.opensearch.search.DocValueFormat; import org.opensearch.search.SearchPhaseResult; diff --git a/server/src/test/java/org/opensearch/action/search/SearchRequestTests.java b/server/src/test/java/org/opensearch/action/search/SearchRequestTests.java index 19544af63944c..9df709150314b 100644 --- a/server/src/test/java/org/opensearch/action/search/SearchRequestTests.java +++ b/server/src/test/java/org/opensearch/action/search/SearchRequestTests.java @@ -36,9 +36,9 @@ import org.opensearch.Version; import org.opensearch.action.ActionRequestValidationException; import org.opensearch.action.support.IndicesOptions; -import org.opensearch.common.Strings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.ArrayUtils; +import org.opensearch.core.common.Strings; import org.opensearch.index.query.QueryBuilders; import org.opensearch.search.AbstractSearchTestCase; import org.opensearch.search.Scroll; diff --git a/server/src/test/java/org/opensearch/action/search/TransportSearchActionTests.java b/server/src/test/java/org/opensearch/action/search/TransportSearchActionTests.java index 136e1879881f8..51d9a06c9ac43 100644 --- a/server/src/test/java/org/opensearch/action/search/TransportSearchActionTests.java +++ b/server/src/test/java/org/opensearch/action/search/TransportSearchActionTests.java @@ -52,10 +52,10 @@ import org.opensearch.cluster.routing.ShardRoutingState; import org.opensearch.cluster.routing.TestShardRouting; import org.opensearch.common.SetOnce; -import org.opensearch.common.Strings; import org.opensearch.common.collect.Tuple; import org.opensearch.common.settings.Settings; import org.opensearch.common.transport.TransportAddress; +import org.opensearch.core.common.Strings; import org.opensearch.index.Index; import org.opensearch.index.query.InnerHitBuilder; import org.opensearch.index.query.MatchAllQueryBuilder; diff --git a/server/src/test/java/org/opensearch/client/AbstractClientHeadersTestCase.java b/server/src/test/java/org/opensearch/client/AbstractClientHeadersTestCase.java index 92a88aa7940ee..f40227beb59b7 100644 --- a/server/src/test/java/org/opensearch/client/AbstractClientHeadersTestCase.java +++ b/server/src/test/java/org/opensearch/client/AbstractClientHeadersTestCase.java @@ -32,7 +32,7 @@ package org.opensearch.client; -import org.opensearch.ExceptionsHelper; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.action.ActionType; import org.opensearch.action.ActionListener; import org.opensearch.action.admin.cluster.reroute.ClusterRerouteAction; @@ -244,7 +244,7 @@ public Throwable unwrap(Throwable t, Class exceptionType) { } if (counter++ > 10) { // dear god, if we got more than 10 levels down, WTF? just bail - fail("Exception cause unwrapping ran for 10 levels: " + ExceptionsHelper.stackTrace(t)); + fail("Exception cause unwrapping ran for 10 levels: " + BaseExceptionsHelper.stackTrace(t)); return null; } result = result.getCause(); diff --git a/server/src/test/java/org/opensearch/cluster/metadata/IndexNameExpressionResolverTests.java b/server/src/test/java/org/opensearch/cluster/metadata/IndexNameExpressionResolverTests.java index 52a2663f173a7..e79839de7a292 100644 --- a/server/src/test/java/org/opensearch/cluster/metadata/IndexNameExpressionResolverTests.java +++ b/server/src/test/java/org/opensearch/cluster/metadata/IndexNameExpressionResolverTests.java @@ -45,9 +45,9 @@ import org.opensearch.cluster.ClusterName; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexMetadata.State; -import org.opensearch.common.Strings; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.concurrent.ThreadContext; +import org.opensearch.core.common.Strings; import org.opensearch.index.Index; import org.opensearch.index.IndexNotFoundException; import org.opensearch.index.IndexSettings; diff --git a/server/src/test/java/org/opensearch/cluster/metadata/MetadataIndexStateServiceTests.java b/server/src/test/java/org/opensearch/cluster/metadata/MetadataIndexStateServiceTests.java index 502436dc66fb7..a334d20563450 100644 --- a/server/src/test/java/org/opensearch/cluster/metadata/MetadataIndexStateServiceTests.java +++ b/server/src/test/java/org/opensearch/cluster/metadata/MetadataIndexStateServiceTests.java @@ -55,9 +55,9 @@ import org.opensearch.cluster.routing.UnassignedInfo; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.collect.Tuple; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; import org.opensearch.index.Index; import org.opensearch.index.IndexNotFoundException; import org.opensearch.index.shard.ShardId; diff --git a/server/src/test/java/org/opensearch/common/StringsTests.java b/server/src/test/java/org/opensearch/common/StringsTests.java index 6663b16b5d122..50f7be8be170d 100644 --- a/server/src/test/java/org/opensearch/common/StringsTests.java +++ b/server/src/test/java/org/opensearch/common/StringsTests.java @@ -32,7 +32,6 @@ package org.opensearch.common; -import org.opensearch.common.util.set.Sets; import org.opensearch.common.xcontent.XContentType; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.ToXContentObject; @@ -125,17 +124,4 @@ public void testToStringToXContentWithOrWithoutParams() { containsString("\"color_from_param\":\"blue\"") ); } - - public void testSplitStringToSet() { - assertEquals(Strings.tokenizeByCommaToSet(null), Sets.newHashSet()); - assertEquals(Strings.tokenizeByCommaToSet(""), Sets.newHashSet()); - assertEquals(Strings.tokenizeByCommaToSet("a,b,c"), Sets.newHashSet("a", "b", "c")); - assertEquals(Strings.tokenizeByCommaToSet("a, b, c"), Sets.newHashSet("a", "b", "c")); - assertEquals(Strings.tokenizeByCommaToSet(" a , b, c "), Sets.newHashSet("a", "b", "c")); - assertEquals(Strings.tokenizeByCommaToSet("aa, bb, cc"), Sets.newHashSet("aa", "bb", "cc")); - assertEquals(Strings.tokenizeByCommaToSet(" a "), Sets.newHashSet("a")); - assertEquals(Strings.tokenizeByCommaToSet(" a "), Sets.newHashSet("a")); - assertEquals(Strings.tokenizeByCommaToSet(" aa "), Sets.newHashSet("aa")); - assertEquals(Strings.tokenizeByCommaToSet(" "), Sets.newHashSet()); - } } diff --git a/server/src/test/java/org/opensearch/common/geo/GeoJsonParserTests.java b/server/src/test/java/org/opensearch/common/geo/GeoJsonParserTests.java index 05ad2b41fe5b5..9ab0f41f33762 100644 --- a/server/src/test/java/org/opensearch/common/geo/GeoJsonParserTests.java +++ b/server/src/test/java/org/opensearch/common/geo/GeoJsonParserTests.java @@ -33,11 +33,11 @@ package org.opensearch.common.geo; import org.opensearch.common.Strings; -import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.common.xcontent.XContentFactory; +import org.opensearch.common.xcontent.json.JsonXContent; +import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParseException; import org.opensearch.core.xcontent.XContentParser; -import org.opensearch.common.xcontent.json.JsonXContent; import org.opensearch.geometry.Circle; import org.opensearch.geometry.Geometry; import org.opensearch.geometry.GeometryCollection; diff --git a/server/src/test/java/org/opensearch/http/CorsHandlerTests.java b/server/src/test/java/org/opensearch/http/CorsHandlerTests.java index 08464db73fd78..7b38d7039dc0e 100644 --- a/server/src/test/java/org/opensearch/http/CorsHandlerTests.java +++ b/server/src/test/java/org/opensearch/http/CorsHandlerTests.java @@ -32,10 +32,10 @@ package org.opensearch.http; -import org.opensearch.common.Strings; import org.opensearch.common.bytes.BytesArray; import org.opensearch.common.settings.Settings; import org.opensearch.common.settings.SettingsException; +import org.opensearch.core.common.Strings; import org.opensearch.rest.RestRequest; import org.opensearch.rest.RestStatus; import org.opensearch.test.OpenSearchTestCase; @@ -50,7 +50,7 @@ import java.util.regex.PatternSyntaxException; import java.util.stream.Collectors; -import static org.opensearch.common.Strings.collectionToDelimitedString; +import static org.opensearch.core.common.Strings.collectionToDelimitedString; import static org.opensearch.http.HttpTransportSettings.SETTING_CORS_ALLOW_CREDENTIALS; import static org.opensearch.http.HttpTransportSettings.SETTING_CORS_ALLOW_HEADERS; import static org.opensearch.http.HttpTransportSettings.SETTING_CORS_ALLOW_METHODS; diff --git a/server/src/test/java/org/opensearch/index/engine/InternalEngineTests.java b/server/src/test/java/org/opensearch/index/engine/InternalEngineTests.java index a80ee734a5bd2..d74bc965f12cb 100644 --- a/server/src/test/java/org/opensearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/opensearch/index/engine/InternalEngineTests.java @@ -97,7 +97,6 @@ import org.opensearch.common.CheckedRunnable; import org.opensearch.common.Randomness; import org.opensearch.common.SetOnce; -import org.opensearch.common.Strings; import org.opensearch.common.TriFunction; import org.opensearch.common.UUIDs; import org.opensearch.common.bytes.BytesArray; @@ -120,6 +119,7 @@ import org.opensearch.common.util.concurrent.ReleasableLock; import org.opensearch.common.xcontent.XContentType; import org.opensearch.common.util.io.IOUtils; +import org.opensearch.core.common.Strings; import org.opensearch.index.IndexSettings; import org.opensearch.index.VersionType; import org.opensearch.index.codec.CodecService; diff --git a/server/src/test/java/org/opensearch/index/fielddata/AbstractFieldDataImplTestCase.java b/server/src/test/java/org/opensearch/index/fielddata/AbstractFieldDataImplTestCase.java index 9be0a738441e5..1ffacf98a6836 100644 --- a/server/src/test/java/org/opensearch/index/fielddata/AbstractFieldDataImplTestCase.java +++ b/server/src/test/java/org/opensearch/index/fielddata/AbstractFieldDataImplTestCase.java @@ -41,7 +41,7 @@ import org.apache.lucene.search.SortField; import org.apache.lucene.search.TopFieldDocs; import org.apache.lucene.util.BytesRef; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.search.MultiValueMode; import java.io.IOException; diff --git a/server/src/test/java/org/opensearch/index/mapper/DynamicMappingTests.java b/server/src/test/java/org/opensearch/index/mapper/DynamicMappingTests.java index 6759ed05e9735..222cdf9f7684c 100644 --- a/server/src/test/java/org/opensearch/index/mapper/DynamicMappingTests.java +++ b/server/src/test/java/org/opensearch/index/mapper/DynamicMappingTests.java @@ -37,7 +37,6 @@ import org.opensearch.common.xcontent.XContentType; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.common.xcontent.XContentFactory; -import org.opensearch.common.xcontent.XContentType; import java.io.IOException; import java.time.Instant; diff --git a/server/src/test/java/org/opensearch/index/query/InnerHitBuilderTests.java b/server/src/test/java/org/opensearch/index/query/InnerHitBuilderTests.java index bdfc03838dcea..d252d76fa19b2 100644 --- a/server/src/test/java/org/opensearch/index/query/InnerHitBuilderTests.java +++ b/server/src/test/java/org/opensearch/index/query/InnerHitBuilderTests.java @@ -31,16 +31,16 @@ package org.opensearch.index.query; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.io.stream.NamedWriteableRegistry; 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.xcontent.NamedXContentRegistry; import org.opensearch.core.xcontent.ToXContent; 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.script.Script; import org.opensearch.script.ScriptType; import org.opensearch.search.SearchModule; diff --git a/server/src/test/java/org/opensearch/index/reindex/BulkByScrollTaskStatusOrExceptionTests.java b/server/src/test/java/org/opensearch/index/reindex/BulkByScrollTaskStatusOrExceptionTests.java index 53c1824e849d0..8e28580fc9296 100644 --- a/server/src/test/java/org/opensearch/index/reindex/BulkByScrollTaskStatusOrExceptionTests.java +++ b/server/src/test/java/org/opensearch/index/reindex/BulkByScrollTaskStatusOrExceptionTests.java @@ -33,7 +33,7 @@ package org.opensearch.index.reindex; import org.opensearch.OpenSearchException; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.test.AbstractXContentTestCase; diff --git a/server/src/test/java/org/opensearch/index/reindex/BulkByScrollTaskStatusTests.java b/server/src/test/java/org/opensearch/index/reindex/BulkByScrollTaskStatusTests.java index 465307c11be5e..8d2c24ff5320d 100644 --- a/server/src/test/java/org/opensearch/index/reindex/BulkByScrollTaskStatusTests.java +++ b/server/src/test/java/org/opensearch/index/reindex/BulkByScrollTaskStatusTests.java @@ -36,9 +36,9 @@ import org.opensearch.OpenSearchException; import org.opensearch.Version; import org.opensearch.common.Randomness; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.unit.TimeValue; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.test.AbstractXContentTestCase; diff --git a/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java b/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java index b698b1a30ce20..199769b29a263 100644 --- a/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java +++ b/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java @@ -477,7 +477,7 @@ public static class TestFilterDirectory extends FilterDirectory { /** * Sole constructor, typically called from sub-classes. * - * @param in + * @param in input directory */ public TestFilterDirectory(Directory in) { super(in); diff --git a/server/src/test/java/org/opensearch/lucene/search/uhighlight/CustomUnifiedHighlighterTests.java b/server/src/test/java/org/opensearch/lucene/search/uhighlight/CustomUnifiedHighlighterTests.java index ba4d16c87bed1..ac1c67eda0d91 100644 --- a/server/src/test/java/org/opensearch/lucene/search/uhighlight/CustomUnifiedHighlighterTests.java +++ b/server/src/test/java/org/opensearch/lucene/search/uhighlight/CustomUnifiedHighlighterTests.java @@ -63,8 +63,8 @@ import org.apache.lucene.search.uhighlight.UnifiedHighlighter; import org.apache.lucene.store.Directory; import org.apache.lucene.tests.index.RandomIndexWriter; -import org.opensearch.common.Strings; import org.opensearch.common.lucene.search.MultiPhrasePrefixQuery; +import org.opensearch.core.common.Strings; import org.opensearch.test.OpenSearchTestCase; import java.text.BreakIterator; diff --git a/server/src/test/java/org/opensearch/monitor/jvm/JvmStatsTests.java b/server/src/test/java/org/opensearch/monitor/jvm/JvmStatsTests.java index 41a0891b73aa8..6a7d3afdc5dad 100644 --- a/server/src/test/java/org/opensearch/monitor/jvm/JvmStatsTests.java +++ b/server/src/test/java/org/opensearch/monitor/jvm/JvmStatsTests.java @@ -32,8 +32,8 @@ package org.opensearch.monitor.jvm; -import org.opensearch.common.Strings; import org.opensearch.common.unit.ByteSizeValue; +import org.opensearch.core.common.Strings; import org.opensearch.test.OpenSearchTestCase; import java.io.IOException; diff --git a/server/src/test/java/org/opensearch/rest/BytesRestResponseTests.java b/server/src/test/java/org/opensearch/rest/BytesRestResponseTests.java index ddca4288353ff..0e58e8ce719f6 100644 --- a/server/src/test/java/org/opensearch/rest/BytesRestResponseTests.java +++ b/server/src/test/java/org/opensearch/rest/BytesRestResponseTests.java @@ -32,7 +32,7 @@ package org.opensearch.rest; -import org.opensearch.ExceptionsHelper; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.OpenSearchStatusException; import org.opensearch.OpenSearchException; import org.opensearch.ResourceAlreadyExistsException; @@ -183,7 +183,7 @@ public void testConvert() throws IOException { + "\"failed_shards\":[{\"shard\":1,\"index\":\"foo\",\"node\":\"node_1\",\"reason\":{\"type\":\"parsing_exception\"," + "\"reason\":\"foobar\",\"line\":1,\"col\":2}}]},\"status\":400}"; assertEquals(expected.trim(), text.trim()); - String stackTrace = ExceptionsHelper.stackTrace(ex); + String stackTrace = BaseExceptionsHelper.stackTrace(ex); assertTrue(stackTrace.contains("Caused by: ParsingException[foobar]")); } diff --git a/server/src/test/java/org/opensearch/rest/action/admin/cluster/RestAddVotingConfigExclusionActionTests.java b/server/src/test/java/org/opensearch/rest/action/admin/cluster/RestAddVotingConfigExclusionActionTests.java index 629bb3ca5f358..750519ab006d4 100644 --- a/server/src/test/java/org/opensearch/rest/action/admin/cluster/RestAddVotingConfigExclusionActionTests.java +++ b/server/src/test/java/org/opensearch/rest/action/admin/cluster/RestAddVotingConfigExclusionActionTests.java @@ -33,7 +33,7 @@ package org.opensearch.rest.action.admin.cluster; import org.opensearch.action.admin.cluster.configuration.AddVotingConfigExclusionsRequest; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.rest.RestRequest; import org.opensearch.test.rest.FakeRestRequest; import org.opensearch.test.rest.RestActionTestCase; diff --git a/server/src/test/java/org/opensearch/search/SearchServiceTests.java b/server/src/test/java/org/opensearch/search/SearchServiceTests.java index af9c616cc2d8d..666b2b002e1e3 100644 --- a/server/src/test/java/org/opensearch/search/SearchServiceTests.java +++ b/server/src/test/java/org/opensearch/search/SearchServiceTests.java @@ -55,13 +55,13 @@ import org.opensearch.action.support.IndicesOptions; import org.opensearch.action.support.PlainActionFuture; import org.opensearch.action.support.WriteRequest; -import org.opensearch.common.Strings; import org.opensearch.common.UUIDs; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.FeatureFlags; +import org.opensearch.core.common.Strings; import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.index.Index; diff --git a/server/src/test/java/org/opensearch/search/aggregations/AggregatorFactoriesTests.java b/server/src/test/java/org/opensearch/search/aggregations/AggregatorFactoriesTests.java index 4a67ae838764a..6e83739ac8771 100644 --- a/server/src/test/java/org/opensearch/search/aggregations/AggregatorFactoriesTests.java +++ b/server/src/test/java/org/opensearch/search/aggregations/AggregatorFactoriesTests.java @@ -34,16 +34,16 @@ import org.opensearch.action.ActionListener; import org.opensearch.common.ParsingException; import org.opensearch.common.SetOnce; -import org.opensearch.common.Strings; import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.settings.Settings; -import org.opensearch.core.xcontent.NamedXContentRegistry; -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.common.xcontent.json.JsonXContent; +import org.opensearch.core.common.Strings; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParser; import org.opensearch.env.Environment; import org.opensearch.index.query.MatchAllQueryBuilder; import org.opensearch.index.query.QueryBuilder; diff --git a/server/src/test/java/org/opensearch/search/aggregations/pipeline/DerivativeAggregatorTests.java b/server/src/test/java/org/opensearch/search/aggregations/pipeline/DerivativeAggregatorTests.java index 88628cd44c721..801f057d8888d 100644 --- a/server/src/test/java/org/opensearch/search/aggregations/pipeline/DerivativeAggregatorTests.java +++ b/server/src/test/java/org/opensearch/search/aggregations/pipeline/DerivativeAggregatorTests.java @@ -41,7 +41,7 @@ import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.Query; import org.apache.lucene.store.Directory; -import org.opensearch.ExceptionsHelper; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.action.search.SearchPhaseExecutionException; import org.opensearch.common.CheckedConsumer; import org.opensearch.index.mapper.DateFieldMapper; @@ -635,7 +635,7 @@ public void testSingleValueAggDerivative_invalidPath() throws IOException { executeTestCase(query, aggBuilder, history -> {}); fail("Expected an Exception but didn't get one"); } catch (Exception e) { - Throwable cause = ExceptionsHelper.unwrapCause(e); + Throwable cause = BaseExceptionsHelper.unwrapCause(e); if (cause == null) { throw e; } else if (cause instanceof SearchPhaseExecutionException) { diff --git a/server/src/test/java/org/opensearch/search/fetch/subphase/FetchSourcePhaseTests.java b/server/src/test/java/org/opensearch/search/fetch/subphase/FetchSourcePhaseTests.java index 2da4b12587dbd..7356e64ed329b 100644 --- a/server/src/test/java/org/opensearch/search/fetch/subphase/FetchSourcePhaseTests.java +++ b/server/src/test/java/org/opensearch/search/fetch/subphase/FetchSourcePhaseTests.java @@ -34,10 +34,10 @@ import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.memory.MemoryIndex; -import org.opensearch.common.Strings; import org.opensearch.common.bytes.BytesReference; -import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.common.xcontent.XContentFactory; +import org.opensearch.core.common.Strings; +import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.search.SearchHit; import org.opensearch.search.fetch.FetchContext; import org.opensearch.search.fetch.FetchSubPhase.HitContext; diff --git a/server/src/test/java/org/opensearch/search/query/QuerySearchResultTests.java b/server/src/test/java/org/opensearch/search/query/QuerySearchResultTests.java index 6b89eb92065b1..6e612c4ceb830 100644 --- a/server/src/test/java/org/opensearch/search/query/QuerySearchResultTests.java +++ b/server/src/test/java/org/opensearch/search/query/QuerySearchResultTests.java @@ -40,13 +40,13 @@ import org.opensearch.action.OriginalIndices; import org.opensearch.action.OriginalIndicesTests; import org.opensearch.action.search.SearchRequest; -import org.opensearch.common.Strings; import org.opensearch.common.UUIDs; import org.opensearch.common.io.stream.NamedWriteableAwareStreamInput; import org.opensearch.common.io.stream.NamedWriteableRegistry; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.lucene.search.TopDocsAndMaxScore; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; import org.opensearch.index.shard.ShardId; import org.opensearch.search.DocValueFormat; import org.opensearch.search.SearchModule; diff --git a/server/src/test/java/org/opensearch/search/slice/SliceBuilderTests.java b/server/src/test/java/org/opensearch/search/slice/SliceBuilderTests.java index 5a3031ba34a2b..692080b592502 100644 --- a/server/src/test/java/org/opensearch/search/slice/SliceBuilderTests.java +++ b/server/src/test/java/org/opensearch/search/slice/SliceBuilderTests.java @@ -54,13 +54,13 @@ import org.opensearch.cluster.routing.ShardIterator; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.Nullable; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.NamedWriteableRegistry; import org.opensearch.common.settings.Settings; -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.core.common.Strings; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParser; import org.opensearch.index.IndexSettings; import org.opensearch.index.fielddata.IndexNumericFieldData; import org.opensearch.index.mapper.MappedFieldType; diff --git a/server/src/test/java/org/opensearch/tasks/CancelTasksResponseTests.java b/server/src/test/java/org/opensearch/tasks/CancelTasksResponseTests.java index 44b3274d04f97..546ee937e3754 100644 --- a/server/src/test/java/org/opensearch/tasks/CancelTasksResponseTests.java +++ b/server/src/test/java/org/opensearch/tasks/CancelTasksResponseTests.java @@ -36,7 +36,7 @@ import org.opensearch.action.TaskOperationFailure; import org.opensearch.action.admin.cluster.node.tasks.cancel.CancelTasksResponse; 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.xcontent.ToXContent; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.test.AbstractXContentTestCase; diff --git a/server/src/test/java/org/opensearch/tasks/ListTasksResponseTests.java b/server/src/test/java/org/opensearch/tasks/ListTasksResponseTests.java index 92cffbd8cdb10..5da0727d23fed 100644 --- a/server/src/test/java/org/opensearch/tasks/ListTasksResponseTests.java +++ b/server/src/test/java/org/opensearch/tasks/ListTasksResponseTests.java @@ -36,7 +36,7 @@ import org.opensearch.action.FailedNodeException; 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.xcontent.ToXContent; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.test.AbstractXContentTestCase; diff --git a/server/src/test/java/org/opensearch/transport/RemoteClusterConnectionTests.java b/server/src/test/java/org/opensearch/transport/RemoteClusterConnectionTests.java index 86b40b3f84536..74084757e3dd7 100644 --- a/server/src/test/java/org/opensearch/transport/RemoteClusterConnectionTests.java +++ b/server/src/test/java/org/opensearch/transport/RemoteClusterConnectionTests.java @@ -694,7 +694,7 @@ private static Settings buildSniffSettings(String clusterAlias, List see builder.put(RemoteConnectionStrategy.REMOTE_CONNECTION_MODE.getConcreteSettingForNamespace(clusterAlias).getKey(), "sniff"); builder.put( SniffConnectionStrategy.REMOTE_CLUSTER_SEEDS.getConcreteSettingForNamespace(clusterAlias).getKey(), - Strings.collectionToCommaDelimitedString(seedNodes) + org.opensearch.core.common.Strings.collectionToCommaDelimitedString(seedNodes) ); return builder.build(); } diff --git a/server/src/test/java/org/opensearch/transport/RemoteClusterServiceTests.java b/server/src/test/java/org/opensearch/transport/RemoteClusterServiceTests.java index 8b26fa42e7b76..52bf7aa08fe67 100644 --- a/server/src/test/java/org/opensearch/transport/RemoteClusterServiceTests.java +++ b/server/src/test/java/org/opensearch/transport/RemoteClusterServiceTests.java @@ -38,12 +38,12 @@ import org.opensearch.action.support.PlainActionFuture; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodeRole; -import org.opensearch.common.Strings; import org.opensearch.common.settings.AbstractScopedSettings; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.io.IOUtils; +import org.opensearch.core.common.Strings; import org.opensearch.test.OpenSearchTestCase; import org.opensearch.test.transport.MockTransportService; import org.opensearch.threadpool.TestThreadPool; diff --git a/server/src/test/java/org/opensearch/transport/SniffConnectionStrategyTests.java b/server/src/test/java/org/opensearch/transport/SniffConnectionStrategyTests.java index 409bc327cb095..975da2de82ae6 100644 --- a/server/src/test/java/org/opensearch/transport/SniffConnectionStrategyTests.java +++ b/server/src/test/java/org/opensearch/transport/SniffConnectionStrategyTests.java @@ -42,13 +42,13 @@ 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.collect.Tuple; import org.opensearch.common.settings.AbstractScopedSettings; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; import org.opensearch.common.transport.TransportAddress; +import org.opensearch.core.common.Strings; import org.opensearch.test.OpenSearchTestCase; import org.opensearch.test.VersionUtils; import org.opensearch.test.transport.MockTransportService; diff --git a/server/src/test/java/org/opensearch/transport/TransportActionProxyTests.java b/server/src/test/java/org/opensearch/transport/TransportActionProxyTests.java index 2bbbab1b2ccbd..64125863c3c5f 100644 --- a/server/src/test/java/org/opensearch/transport/TransportActionProxyTests.java +++ b/server/src/test/java/org/opensearch/transport/TransportActionProxyTests.java @@ -31,8 +31,8 @@ package org.opensearch.transport; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.OpenSearchException; -import org.opensearch.ExceptionsHelper; import org.opensearch.Version; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.common.io.stream.StreamInput; @@ -196,7 +196,7 @@ public void handleResponse(SimpleTestResponse response) { @Override public void handleException(TransportException exp) { try { - Throwable cause = ExceptionsHelper.unwrapCause(exp); + Throwable cause = BaseExceptionsHelper.unwrapCause(exp); assertEquals("greetings from TS_C", cause.getMessage()); } finally { latch.countDown(); diff --git a/test/fixtures/gcs-fixture/src/main/java/fixture/gcs/GoogleCloudStorageHttpHandler.java b/test/fixtures/gcs-fixture/src/main/java/fixture/gcs/GoogleCloudStorageHttpHandler.java index af3faa46e2169..a3741668465fa 100644 --- a/test/fixtures/gcs-fixture/src/main/java/fixture/gcs/GoogleCloudStorageHttpHandler.java +++ b/test/fixtures/gcs-fixture/src/main/java/fixture/gcs/GoogleCloudStorageHttpHandler.java @@ -36,7 +36,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; -import org.opensearch.common.Strings; import org.opensearch.common.SuppressForbidden; import org.opensearch.common.UUIDs; import org.opensearch.common.bytes.BytesArray; @@ -45,6 +44,7 @@ import org.opensearch.common.collect.Tuple; import org.opensearch.common.io.Streams; import org.opensearch.common.regex.Regex; +import org.opensearch.core.common.Strings; import org.opensearch.rest.RestStatus; import org.opensearch.rest.RestUtils; diff --git a/test/framework/src/main/java/org/opensearch/bootstrap/BootstrapForTesting.java b/test/framework/src/main/java/org/opensearch/bootstrap/BootstrapForTesting.java index 428b0988d4cce..d46fcb224e8c2 100644 --- a/test/framework/src/main/java/org/opensearch/bootstrap/BootstrapForTesting.java +++ b/test/framework/src/main/java/org/opensearch/bootstrap/BootstrapForTesting.java @@ -37,8 +37,8 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.tests.util.LuceneTestCase; import org.opensearch.common.Booleans; -import org.opensearch.common.Strings; import org.opensearch.common.SuppressForbidden; +import org.opensearch.core.common.Strings; import org.opensearch.core.util.FileSystemUtils; import org.opensearch.common.io.PathUtils; import org.opensearch.common.network.IfConfig; diff --git a/test/framework/src/main/java/org/opensearch/repositories/AbstractThirdPartyRepositoryTestCase.java b/test/framework/src/main/java/org/opensearch/repositories/AbstractThirdPartyRepositoryTestCase.java index 8ddaba8067b2d..5c28d9f83a4ee 100644 --- a/test/framework/src/main/java/org/opensearch/repositories/AbstractThirdPartyRepositoryTestCase.java +++ b/test/framework/src/main/java/org/opensearch/repositories/AbstractThirdPartyRepositoryTestCase.java @@ -36,13 +36,13 @@ import org.opensearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse; import org.opensearch.action.admin.cluster.snapshots.delete.DeleteSnapshotRequest; import org.opensearch.action.support.PlainActionFuture; -import org.opensearch.common.Strings; import org.opensearch.common.blobstore.BlobMetadata; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.blobstore.BlobStore; import org.opensearch.common.blobstore.support.PlainBlobMetadata; import org.opensearch.common.settings.SecureSettings; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; import org.opensearch.repositories.blobstore.BlobStoreRepository; import org.opensearch.repositories.blobstore.BlobStoreTestUtil; import org.opensearch.snapshots.SnapshotState; diff --git a/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java b/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java index 465101ef164a1..c164bccd21c37 100644 --- a/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java +++ b/test/framework/src/main/java/org/opensearch/repositories/blobstore/BlobStoreTestUtil.java @@ -47,13 +47,13 @@ import org.opensearch.cluster.node.DiscoveryNodes; import org.opensearch.cluster.service.ClusterApplierService; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.Strings; import org.opensearch.common.blobstore.BlobContainer; import org.opensearch.common.blobstore.BlobMetadata; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.blobstore.BlobStore; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.xcontent.LoggingDeprecationHandler; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.common.xcontent.XContentType; diff --git a/test/framework/src/main/java/org/opensearch/repositories/blobstore/OpenSearchMockAPIBasedRepositoryIntegTestCase.java b/test/framework/src/main/java/org/opensearch/repositories/blobstore/OpenSearchMockAPIBasedRepositoryIntegTestCase.java index 85ff45b80fad6..cfeb4f4fee8f2 100644 --- a/test/framework/src/main/java/org/opensearch/repositories/blobstore/OpenSearchMockAPIBasedRepositoryIntegTestCase.java +++ b/test/framework/src/main/java/org/opensearch/repositories/blobstore/OpenSearchMockAPIBasedRepositoryIntegTestCase.java @@ -42,11 +42,11 @@ import org.apache.lucene.tests.util.LuceneTestCase; import org.opensearch.action.admin.indices.forcemerge.ForceMergeResponse; import org.opensearch.cluster.metadata.IndexMetadata; -import org.opensearch.common.Strings; import org.opensearch.common.SuppressForbidden; import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.network.InetAddresses; import org.opensearch.common.settings.Settings; +import org.opensearch.core.common.Strings; import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.Repository; import org.opensearch.repositories.RepositoryMissingException; diff --git a/test/framework/src/main/java/org/opensearch/test/AbstractSerializingTestCase.java b/test/framework/src/main/java/org/opensearch/test/AbstractSerializingTestCase.java index aa6187cba5327..152691bdb38a3 100644 --- a/test/framework/src/main/java/org/opensearch/test/AbstractSerializingTestCase.java +++ b/test/framework/src/main/java/org/opensearch/test/AbstractSerializingTestCase.java @@ -32,8 +32,8 @@ package org.opensearch.test; -import org.opensearch.common.Strings; import org.opensearch.common.io.stream.Writeable; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; diff --git a/test/framework/src/main/java/org/opensearch/test/AbstractXContentTestCase.java b/test/framework/src/main/java/org/opensearch/test/AbstractXContentTestCase.java index b0abe854d3ce7..d2dfa4ef628e5 100644 --- a/test/framework/src/main/java/org/opensearch/test/AbstractXContentTestCase.java +++ b/test/framework/src/main/java/org/opensearch/test/AbstractXContentTestCase.java @@ -35,8 +35,8 @@ import org.opensearch.common.CheckedBiConsumer; import org.opensearch.common.CheckedBiFunction; import org.opensearch.common.CheckedFunction; -import org.opensearch.common.Strings; import org.opensearch.common.bytes.BytesReference; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.XContent; import org.opensearch.core.xcontent.XContentBuilder; diff --git a/test/framework/src/main/java/org/opensearch/test/InternalTestCluster.java b/test/framework/src/main/java/org/opensearch/test/InternalTestCluster.java index f67924a2a7719..d0727e6a01a99 100644 --- a/test/framework/src/main/java/org/opensearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/opensearch/test/InternalTestCluster.java @@ -68,10 +68,8 @@ import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.Nullable; import org.opensearch.common.Randomness; -import org.opensearch.common.Strings; import org.opensearch.common.breaker.CircuitBreaker; import org.opensearch.common.component.LifecycleListener; -import org.opensearch.core.util.FileSystemUtils; import org.opensearch.common.io.stream.NamedWriteableRegistry; import org.opensearch.common.lease.Releasables; import org.opensearch.common.settings.MockSecureSettings; @@ -87,6 +85,8 @@ import org.opensearch.common.util.concurrent.ThreadContext; import org.opensearch.common.util.set.Sets; import org.opensearch.common.util.io.IOUtils; +import org.opensearch.core.common.Strings; +import org.opensearch.core.util.FileSystemUtils; import org.opensearch.env.Environment; import org.opensearch.env.NodeEnvironment; import org.opensearch.env.ShardLockObtainFailedException; 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 737342f85ae2a..e061055882fba 100644 --- a/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java +++ b/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java @@ -40,8 +40,8 @@ import org.apache.lucene.search.Sort; import org.apache.lucene.search.TotalHits; import org.apache.lucene.tests.util.LuceneTestCase; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.OpenSearchException; -import org.opensearch.ExceptionsHelper; import org.opensearch.action.ActionListener; import org.opensearch.action.DocWriteResponse; import org.opensearch.action.admin.cluster.health.ClusterHealthRequest; @@ -99,7 +99,6 @@ import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.Nullable; import org.opensearch.common.Priority; -import org.opensearch.common.Strings; import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.collect.Tuple; import org.opensearch.common.io.stream.NamedWriteableRegistry; @@ -114,16 +113,17 @@ import org.opensearch.common.unit.ByteSizeUnit; import org.opensearch.common.unit.ByteSizeValue; import org.opensearch.common.unit.TimeValue; -import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; import org.opensearch.common.util.concurrent.ThreadContext; -import org.opensearch.core.xcontent.NamedXContentRegistry; -import org.opensearch.core.xcontent.ToXContent; -import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.common.xcontent.XContentHelper; -import org.opensearch.core.xcontent.XContentParser; import org.opensearch.common.xcontent.XContentType; import org.opensearch.common.xcontent.smile.SmileXContent; import org.opensearch.common.util.io.IOUtils; +import org.opensearch.core.common.Strings; +import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParser; import org.opensearch.env.Environment; import org.opensearch.env.TestEnvironment; import org.opensearch.http.HttpInfo; @@ -1648,7 +1648,7 @@ public void indexRandom(boolean forceRefresh, boolean dummyDocuments, boolean ma } final List actualErrors = new ArrayList<>(); for (Tuple tuple : errors) { - Throwable t = ExceptionsHelper.unwrapCause(tuple.v2()); + Throwable t = BaseExceptionsHelper.unwrapCause(tuple.v2()); if (t instanceof OpenSearchRejectedExecutionException) { logger.debug("Error indexing doc: " + t.getMessage() + ", reindexing."); tuple.v1().execute().actionGet(); // re-index if rejected diff --git a/test/framework/src/main/java/org/opensearch/test/OpenSearchSingleNodeTestCase.java b/test/framework/src/main/java/org/opensearch/test/OpenSearchSingleNodeTestCase.java index 3d03c3d4b04c9..bf797ef6b310b 100644 --- a/test/framework/src/main/java/org/opensearch/test/OpenSearchSingleNodeTestCase.java +++ b/test/framework/src/main/java/org/opensearch/test/OpenSearchSingleNodeTestCase.java @@ -45,14 +45,14 @@ import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.routing.allocation.DiskThresholdSettings; 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.BigArrays; import org.opensearch.common.util.concurrent.OpenSearchExecutors; +import org.opensearch.common.util.io.IOUtils; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.core.xcontent.XContentBuilder; -import org.opensearch.common.util.io.IOUtils; import org.opensearch.env.Environment; import org.opensearch.env.NodeEnvironment; import org.opensearch.index.Index; diff --git a/test/framework/src/main/java/org/opensearch/test/junit/listeners/ReproduceInfoPrinter.java b/test/framework/src/main/java/org/opensearch/test/junit/listeners/ReproduceInfoPrinter.java index b2d0705b937e2..1b119f2c34dde 100644 --- a/test/framework/src/main/java/org/opensearch/test/junit/listeners/ReproduceInfoPrinter.java +++ b/test/framework/src/main/java/org/opensearch/test/junit/listeners/ReproduceInfoPrinter.java @@ -35,8 +35,8 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.lucene.util.Constants; -import org.opensearch.common.Strings; import org.opensearch.common.SuppressForbidden; +import org.opensearch.core.common.Strings; import org.opensearch.test.OpenSearchIntegTestCase; import org.opensearch.test.OpenSearchTestCase; import org.opensearch.test.rest.yaml.OpenSearchClientYamlSuiteTestCase; diff --git a/test/framework/src/main/java/org/opensearch/test/rest/OpenSearchRestTestCase.java b/test/framework/src/main/java/org/opensearch/test/rest/OpenSearchRestTestCase.java index 4664220dcb2d0..56567d21f3847 100644 --- a/test/framework/src/main/java/org/opensearch/test/rest/OpenSearchRestTestCase.java +++ b/test/framework/src/main/java/org/opensearch/test/rest/OpenSearchRestTestCase.java @@ -1073,10 +1073,10 @@ protected static boolean aliasExists(String index, String alias) throws IOExcept @SuppressWarnings("unchecked") protected static Map getAlias(final String index, final String alias) throws IOException { String endpoint = "/_alias"; - if (false == Strings.isEmpty(index)) { + if (false == org.opensearch.core.common.Strings.isEmpty(index)) { endpoint = index + endpoint; } - if (false == Strings.isEmpty(alias)) { + if (false == org.opensearch.core.common.Strings.isEmpty(alias)) { endpoint = endpoint + "/" + alias; } Map getAliasResponse = getAsMap(endpoint); diff --git a/test/framework/src/main/java/org/opensearch/test/rest/yaml/OpenSearchClientYamlSuiteTestCase.java b/test/framework/src/main/java/org/opensearch/test/rest/yaml/OpenSearchClientYamlSuiteTestCase.java index ab03955e4a17d..6ee7cb9b88b77 100644 --- a/test/framework/src/main/java/org/opensearch/test/rest/yaml/OpenSearchClientYamlSuiteTestCase.java +++ b/test/framework/src/main/java/org/opensearch/test/rest/yaml/OpenSearchClientYamlSuiteTestCase.java @@ -299,7 +299,7 @@ private static void addSuite(Path root, Path file, Map> files) private static String[] resolvePathsProperty(String propertyName, String defaultValue) { String property = System.getProperty(propertyName); - if (!Strings.hasLength(property)) { + if (!org.opensearch.core.common.Strings.hasLength(property)) { return defaultValue == null ? Strings.EMPTY_ARRAY : new String[] { defaultValue }; } else { return property.split(PATHS_SEPARATOR); diff --git a/test/framework/src/main/java/org/opensearch/test/rest/yaml/Stash.java b/test/framework/src/main/java/org/opensearch/test/rest/yaml/Stash.java index 8a064f42269b8..4f5dd35602256 100644 --- a/test/framework/src/main/java/org/opensearch/test/rest/yaml/Stash.java +++ b/test/framework/src/main/java/org/opensearch/test/rest/yaml/Stash.java @@ -34,7 +34,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.XContentBuilder; diff --git a/test/framework/src/main/java/org/opensearch/test/rest/yaml/section/DoSection.java b/test/framework/src/main/java/org/opensearch/test/rest/yaml/section/DoSection.java index f527ac1039e28..3d353484fbfa9 100644 --- a/test/framework/src/main/java/org/opensearch/test/rest/yaml/section/DoSection.java +++ b/test/framework/src/main/java/org/opensearch/test/rest/yaml/section/DoSection.java @@ -40,9 +40,9 @@ import org.opensearch.client.Node; import org.opensearch.client.NodeSelector; import org.opensearch.common.ParsingException; -import org.opensearch.common.Strings; import org.opensearch.common.collect.Tuple; import org.opensearch.common.logging.HeaderWarning; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.DeprecationHandler; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.core.xcontent.XContentLocation; diff --git a/test/framework/src/main/java/org/opensearch/test/rest/yaml/section/SkipSection.java b/test/framework/src/main/java/org/opensearch/test/rest/yaml/section/SkipSection.java index f96dcfa3e0246..172c509eab0c0 100644 --- a/test/framework/src/main/java/org/opensearch/test/rest/yaml/section/SkipSection.java +++ b/test/framework/src/main/java/org/opensearch/test/rest/yaml/section/SkipSection.java @@ -33,7 +33,7 @@ import org.opensearch.Version; import org.opensearch.common.ParsingException; -import org.opensearch.common.Strings; +import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.XContentParser; import org.opensearch.test.VersionUtils; import org.opensearch.test.rest.yaml.Features; @@ -117,10 +117,10 @@ public static SkipSection parse(XContentParser parser) throws IOException { parser.nextToken(); - if (!Strings.hasLength(version) && features.isEmpty()) { + if (Strings.hasLength(version) == false && features.isEmpty() == true) { throw new ParsingException(parser.getTokenLocation(), "version or features is mandatory within skip section"); } - if (Strings.hasLength(version) && !Strings.hasLength(reason)) { + if (Strings.hasLength(version) == true && Strings.hasLength(reason) == false) { throw new ParsingException(parser.getTokenLocation(), "reason is mandatory within skip version section"); } return new SkipSection(version, features, reason); diff --git a/test/framework/src/main/java/org/opensearch/transport/AbstractSimpleTransportTestCase.java b/test/framework/src/main/java/org/opensearch/transport/AbstractSimpleTransportTestCase.java index 71f71e7b4a842..2a8a9217fdf98 100644 --- a/test/framework/src/main/java/org/opensearch/transport/AbstractSimpleTransportTestCase.java +++ b/test/framework/src/main/java/org/opensearch/transport/AbstractSimpleTransportTestCase.java @@ -39,6 +39,7 @@ import org.apache.logging.log4j.util.Supplier; import org.apache.lucene.util.CollectionUtil; import org.apache.lucene.util.Constants; +import org.opensearch.BaseExceptionsHelper; import org.opensearch.LegacyESVersion; import org.opensearch.OpenSearchException; import org.opensearch.ExceptionsHelper; @@ -558,7 +559,7 @@ public void onRequestSent( serviceA.submitRequest(nodeB, ACTION, TransportRequest.Empty.INSTANCE, EmptyTransportResponseHandler.INSTANCE_SAME).get(); } catch (ExecutionException e) { assertThat(e.getCause(), instanceOf(OpenSearchException.class)); - assertThat(ExceptionsHelper.unwrapCause(e.getCause()).getMessage(), equalTo("simulated")); + assertThat(BaseExceptionsHelper.unwrapCause(e.getCause()).getMessage(), equalTo("simulated")); } // use assert busy as callbacks are called on a different thread @@ -577,7 +578,7 @@ public void onRequestSent( serviceB.submitRequest(nodeA, ACTION, TransportRequest.Empty.INSTANCE, EmptyTransportResponseHandler.INSTANCE_SAME).get(); } catch (ExecutionException e) { assertThat(e.getCause(), instanceOf(OpenSearchException.class)); - assertThat(ExceptionsHelper.unwrapCause(e.getCause()).getMessage(), equalTo("simulated")); + assertThat(BaseExceptionsHelper.unwrapCause(e.getCause()).getMessage(), equalTo("simulated")); } // use assert busy as callbacks are called on a different thread @@ -597,7 +598,7 @@ public void onRequestSent( serviceA.submitRequest(nodeA, ACTION, TransportRequest.Empty.INSTANCE, EmptyTransportResponseHandler.INSTANCE_SAME).get(); } catch (ExecutionException e) { assertThat(e.getCause(), instanceOf(OpenSearchException.class)); - assertThat(ExceptionsHelper.unwrapCause(e.getCause()).getMessage(), equalTo("simulated")); + assertThat(BaseExceptionsHelper.unwrapCause(e.getCause()).getMessage(), equalTo("simulated")); } // use assert busy as callbacks are called on a different thread @@ -1651,7 +1652,7 @@ public void handleResponse(StringMessageResponse response) { @Override public void handleException(TransportException exp) { - Throwable cause = ExceptionsHelper.unwrapCause(exp); + Throwable cause = BaseExceptionsHelper.unwrapCause(exp); assertThat(cause, instanceOf(ConnectTransportException.class)); assertThat(((ConnectTransportException) cause).node(), equalTo(nodeA)); } @@ -1662,7 +1663,7 @@ public void handleException(TransportException exp) { res.txGet(); fail("exception should be thrown"); } catch (Exception e) { - Throwable cause = ExceptionsHelper.unwrapCause(e); + Throwable cause = BaseExceptionsHelper.unwrapCause(e); assertThat(cause, instanceOf(ConnectTransportException.class)); assertThat(((ConnectTransportException) cause).node(), equalTo(nodeA)); }