From 7d1515d85274bbd3c8f136313ac2ce34258786e5 Mon Sep 17 00:00:00 2001 From: Andriy Redko Date: Mon, 13 Nov 2023 16:45:04 -0500 Subject: [PATCH] [BWC and API enforcement] Decorate the existing APIs with proper annotations (part 3) Signed-off-by: Andriy Redko --- .../java/org/opensearch/common/Explicit.java | 5 +- .../opensearch/common/lease/Releasable.java | 5 +- .../core/action/ActionResponse.java | 2 + .../core/common/io/stream/Writeable.java | 11 +++- .../core/common/util/ByteArray.java | 4 +- .../breaker/AllCircuitBreakerStats.java | 4 +- .../breaker/CircuitBreakerService.java | 8 +-- .../indices/breaker/CircuitBreakerStats.java | 4 +- .../core/xcontent/ObjectParser.java | 4 +- ...emoteStoreBackpressureAndResiliencyIT.java | 4 +- .../org/opensearch/action/AliasesRequest.java | 5 +- .../opensearch/action/OriginalIndices.java | 4 +- .../ClusterAllocationExplanation.java | 4 +- .../snapshots/status/SnapshotStatus.java | 4 +- .../admin/indices/rollover/RolloverInfo.java | 4 +- .../action/search/ParsedScrollId.java | 5 +- .../action/search/SearchContextIdForNode.java | 4 +- .../opensearch/action/search/SearchPhase.java | 4 +- .../action/search/SearchPhaseContext.java | 4 +- .../SearchRequestOperationsListener.java | 4 +- .../action/search/SearchShardIterator.java | 4 +- .../action/search/SearchShardTask.java | 4 +- .../action/search/ShardSearchFailure.java | 4 +- .../action/support/ActiveShardCount.java | 4 +- .../action/support/RetryableAction.java | 4 +- .../PendingReplicationActions.java | 4 +- .../support/replication/ReplicationMode.java | 5 +- .../replication/ReplicationResponse.java | 10 +++- .../org/opensearch/cluster/ClusterState.java | 10 +++- .../cluster/ClusterStateApplier.java | 4 +- .../cluster/ClusterStateListener.java | 5 +- .../cluster/ClusterStateTaskConfig.java | 4 +- .../cluster/ClusterStateTaskExecutor.java | 10 +++- .../cluster/ClusterStateTaskListener.java | 4 +- .../cluster/ClusterStateUpdateTask.java | 4 +- .../LocalNodeClusterManagerListener.java | 5 +- .../cluster/LocalNodeMasterListener.java | 5 +- .../org/opensearch/cluster/NamedDiff.java | 4 +- .../cluster/NodeConnectionsService.java | 4 +- .../opensearch/cluster/RestoreInProgress.java | 13 +++-- .../cluster/SnapshotsInProgress.java | 4 +- .../cluster/TimeoutClusterStateListener.java | 4 +- .../ack/ClusterStateUpdateResponse.java | 5 +- .../cluster/block/ClusterBlockException.java | 4 +- .../cluster/block/ClusterBlocks.java | 7 ++- .../block/IndexCreateBlockException.java | 4 +- .../coordination/ClusterStatePublisher.java | 7 ++- .../coordination/CoordinationMetadata.java | 13 +++-- .../coordination/PersistedStateStats.java | 4 +- .../DecommissionAttributeMetadata.java | 4 +- .../cluster/metadata/ComponentTemplate.java | 4 +- .../metadata/ComposableIndexTemplate.java | 7 ++- .../cluster/metadata/DiffableStringMap.java | 4 +- .../cluster/metadata/IndexAbstraction.java | 10 +++- .../cluster/metadata/IndexGraveyard.java | 10 +++- .../cluster/metadata/IndexMetadata.java | 9 ++- .../opensearch/cluster/metadata/Metadata.java | 13 +++-- .../opensearch/cluster/metadata/Template.java | 4 +- .../metadata/WeightedRoutingMetadata.java | 7 +-- .../cluster/node/DiscoveryNode.java | 22 -------- .../cluster/node/DiscoveryNodeFilters.java | 7 ++- .../cluster/routing/AllocationId.java | 4 +- .../cluster/routing/GroupShardsIterator.java | 4 +- .../cluster/routing/IndexRoutingTable.java | 7 ++- .../routing/IndexShardRoutingTable.java | 7 ++- .../cluster/routing/OperationRouting.java | 4 +- .../cluster/routing/RecoverySource.java | 6 +- .../cluster/routing/RerouteService.java | 4 +- .../routing/RoutingChangesObserver.java | 5 +- .../cluster/routing/RoutingNode.java | 4 +- .../cluster/routing/RoutingNodes.java | 10 +++- .../cluster/routing/RoutingTable.java | 7 ++- .../cluster/routing/ShardIterator.java | 4 +- .../routing/ShardMovementStrategy.java | 4 +- .../cluster/routing/ShardRouting.java | 4 +- .../cluster/routing/ShardsIterator.java | 4 +- .../AllocateUnassignedDecision.java | 4 +- .../allocation/AllocationDecision.java | 4 +- .../routing/allocation/MoveDecision.java | 4 +- .../allocation/NodeAllocationResult.java | 7 ++- .../routing/allocation/RoutingAllocation.java | 7 ++- .../allocation/ShardAllocationDecision.java | 4 +- .../allocation/command/AllocationCommand.java | 4 +- .../command/AllocationCommands.java | 4 +- .../allocation/decider/AllocationDecider.java | 4 +- .../decider/AllocationDeciders.java | 4 +- .../cluster/service/ClusterApplier.java | 4 ++ .../service/ClusterApplierService.java | 4 +- .../service/ClusterManagerService.java | 4 +- .../ClusterManagerThrottlingStats.java | 4 ++ .../cluster/service/ClusterService.java | 4 +- .../cluster/service/ClusterStateStats.java | 4 +- .../cluster/service/MasterService.java | 4 +- .../java/org/opensearch/common/Rounding.java | 13 +++-- .../org/opensearch/common/cache/Cache.java | 7 ++- .../opensearch/common/cache/CacheLoader.java | 5 +- .../common/cache/RemovalNotification.java | 5 +- .../common/cache/RemovalReason.java | 5 +- .../common/compress/CompressedXContent.java | 4 +- .../common/concurrent/GatedCloseable.java | 4 +- .../common/io/stream/DelayableWriteable.java | 7 ++- .../index/OpenSearchDirectoryReader.java | 4 +- .../lucene/search/TopDocsAndMaxScore.java | 4 +- .../lucene/uid/VersionsAndSeqNoResolver.java | 4 +- .../opensearch/common/time/DateFormatter.java | 4 +- .../common/time/DateMathParser.java | 4 +- .../org/opensearch/common/util/BigArrays.java | 2 + .../common/util/CancellableThreads.java | 15 +++-- .../org/opensearch/common/util/Countable.java | 5 +- .../opensearch/common/util/DoubleArray.java | 4 +- .../opensearch/common/util/FloatArray.java | 4 +- .../org/opensearch/common/util/IntArray.java | 4 +- .../org/opensearch/common/util/LongArray.java | 4 +- .../opensearch/common/util/ObjectArray.java | 4 +- .../util/concurrent/AsyncIOProcessor.java | 4 +- .../common/util/concurrent/AtomicArray.java | 4 +- .../common/util/concurrent/ThreadContext.java | 7 ++- .../ThreadContextStatePropagator.java | 5 ++ .../org/opensearch/env/NodeEnvironment.java | 3 + .../gateway/WriteStateException.java | 5 +- .../org/opensearch/index/IndexService.java | 12 ++-- .../org/opensearch/index/IndexWarmer.java | 7 ++- .../index/IndexingPressureService.java | 4 +- .../index/analysis/AnalysisRegistry.java | 4 +- .../index/analysis/AnalyzerProvider.java | 4 +- .../index/analysis/CharFilterFactory.java | 5 +- .../index/analysis/IndexAnalyzers.java | 4 +- .../index/analysis/TokenFilterFactory.java | 4 +- .../index/analysis/TokenizerFactory.java | 4 +- .../opensearch/index/cache/IndexCache.java | 4 +- .../index/cache/bitset/BitsetFilterCache.java | 7 ++- .../cache/bitset/ShardBitsetFilterCache.java | 4 +- .../index/cache/query/QueryCache.java | 4 +- .../cache/request/ShardRequestCache.java | 4 +- .../org/opensearch/index/engine/Engine.java | 55 +++++++++++++------ .../opensearch/index/engine/EngineConfig.java | 7 ++- .../index/engine/EngineException.java | 4 +- .../index/engine/EngineFactory.java | 5 +- .../index/engine/NRTReplicationEngine.java | 4 +- .../index/engine/SafeCommitInfo.java | 4 +- .../index/fielddata/DocValueBits.java | 5 +- .../index/fielddata/IndexFieldData.java | 13 ++++- .../index/fielddata/IndexFieldDataCache.java | 7 ++- .../index/fielddata/LeafFieldData.java | 4 +- .../index/fielddata/NumericDoubleValues.java | 4 +- .../index/fielddata/ShardFieldData.java | 4 +- .../fielddata/SortedNumericDoubleValues.java | 4 +- .../opensearch/index/get/ShardGetService.java | 4 +- .../opensearch/index/mapper/ContentPath.java | 5 +- .../index/mapper/DocValueFetcher.java | 4 +- .../index/mapper/DocumentMapper.java | 9 ++- .../index/mapper/DocumentMapperForType.java | 3 + .../index/mapper/DocumentMapperParser.java | 4 +- .../index/mapper/DynamicTemplate.java | 7 ++- .../opensearch/index/mapper/FieldMapper.java | 10 +++- .../index/mapper/FieldTypeLookup.java | 2 + .../index/mapper/IdFieldMapper.java | 4 +- .../index/mapper/IndexFieldMapper.java | 4 +- .../index/mapper/MappedFieldType.java | 7 ++- .../org/opensearch/index/mapper/Mapper.java | 16 ++++-- .../index/mapper/MapperParsingException.java | 4 +- .../index/mapper/MapperService.java | 5 +- .../org/opensearch/index/mapper/Mapping.java | 4 +- .../index/mapper/MappingLookup.java | 6 +- .../index/mapper/MetadataFieldMapper.java | 10 +++- .../opensearch/index/mapper/ObjectMapper.java | 10 +++- .../index/mapper/ParametrizedFieldMapper.java | 17 ++++-- .../opensearch/index/mapper/ParseContext.java | 7 ++- .../index/mapper/ParsedDocument.java | 4 +- .../index/mapper/RangeFieldMapper.java | 2 +- .../index/mapper/RootObjectMapper.java | 4 +- .../index/mapper/RoutingFieldMapper.java | 4 +- .../index/mapper/SeqNoFieldMapper.java | 4 +- .../index/mapper/SourceFieldMapper.java | 4 +- .../index/mapper/SourceToParse.java | 4 +- .../index/mapper/TextSearchInfo.java | 7 ++- .../opensearch/index/mapper/ValueFetcher.java | 4 +- .../index/query/InnerHitBuilder.java | 4 ++ .../opensearch/index/query/ParsedQuery.java | 4 +- .../index/query/QueryShardContext.java | 4 +- .../opensearch/index/query/Rewriteable.java | 4 +- .../index/query/support/NestedScope.java | 4 +- .../RemoteStoreStatsTrackerFactory.java | 4 +- .../index/seqno/ReplicationTracker.java | 7 ++- .../index/seqno/RetentionLeaseSyncer.java | 2 + .../index/seqno/SequenceNumbers.java | 5 +- .../shard/GlobalCheckpointListeners.java | 4 +- .../IllegalIndexShardStateException.java | 4 +- .../index/shard/IndexEventListener.java | 4 +- .../opensearch/index/shard/IndexShard.java | 7 ++- .../shard/IndexShardClosedException.java | 4 +- .../IndexShardNotRecoveringException.java | 4 +- .../shard/IndexShardNotStartedException.java | 4 +- .../shard/IndexShardRecoveringException.java | 4 +- .../shard/IndexShardRelocatedException.java | 4 +- .../shard/IndexShardStartedException.java | 4 +- .../index/shard/IndexShardState.java | 5 +- .../shard/IndexingOperationListener.java | 4 +- .../index/shard/PrimaryReplicaSyncer.java | 7 ++- .../index/shard/ReplicationGroup.java | 4 +- .../index/shard/SearchOperationListener.java | 4 +- .../org/opensearch/index/shard/ShardPath.java | 4 +- .../index/similarity/SimilarityProvider.java | 4 +- .../index/similarity/SimilarityService.java | 4 +- .../snapshots/IndexShardSnapshotStatus.java | 11 +++- .../RemoteStoreShardShallowCopySnapshot.java | 4 +- .../stats/IndexingPressurePerShardStats.java | 4 +- .../index/stats/IndexingPressureStats.java | 4 +- .../stats/ShardIndexingPressureStats.java | 4 +- .../store/RemoteSegmentStoreDirectory.java | 7 ++- .../store/RemoteStoreFileDownloader.java | 6 +- .../org/opensearch/index/store/Store.java | 10 +++- .../index/store/StoreFileMetadata.java | 4 +- .../index/store/lockmanager/LockInfo.java | 6 +- .../lockmanager/RemoteStoreLockManager.java | 6 +- .../RemoteStoreLockManagerFactory.java | 4 +- .../RemoteStoreMetadataLockManager.java | 4 +- .../remote/filecache/CachedIndexInput.java | 4 +- .../store/remote/filecache/FileCache.java | 4 +- .../metadata/RemoteSegmentMetadata.java | 4 +- .../store/remote/utils/cache/CacheUsage.java | 3 + .../remote/utils/cache/stats/CacheStats.java | 4 +- .../index/translog/BaseTranslogReader.java | 2 +- .../BufferedChecksumStreamOutput.java | 4 +- .../index/translog/ChannelFactory.java | 5 +- .../opensearch/index/translog/Checkpoint.java | 6 +- .../index/translog/TragicExceptionHolder.java | 4 +- .../opensearch/index/translog/Translog.java | 21 ++++--- .../index/translog/TranslogConfig.java | 4 +- .../translog/TranslogDeletionPolicy.java | 4 +- .../TranslogDeletionPolicyFactory.java | 2 + .../index/translog/TranslogException.java | 4 +- .../index/translog/TranslogFactory.java | 5 +- .../index/translog/TranslogManager.java | 5 +- .../index/translog/TranslogReader.java | 4 +- .../translog/TranslogRecoveryRunner.java | 5 +- .../index/translog/TranslogWriter.java | 6 +- .../index/warmer/ShardIndexWarmerService.java | 4 +- .../opensearch/indices/IndicesQueryCache.java | 4 +- .../opensearch/indices/IndicesService.java | 7 ++- .../opensearch/indices/NodeIndicesStats.java | 4 +- .../cluster/IndicesClusterStateService.java | 4 +- .../cache/IndicesFieldDataCache.java | 7 ++- .../indices/mapper/MapperRegistry.java | 4 +- .../recovery/PeerRecoveryTargetService.java | 4 +- .../indices/recovery/RecoveryListener.java | 2 + .../indices/recovery/RecoveryTarget.java | 4 +- .../recovery/StartRecoveryRequest.java | 4 +- .../checkpoint/PublishCheckpointAction.java | 5 +- .../checkpoint/ReplicationCheckpoint.java | 4 +- ...SegmentReplicationCheckpointPublisher.java | 7 ++- .../common/ReplicationFailedException.java | 4 +- .../replication/common/ReplicationState.java | 5 +- .../node/AdaptiveSelectionStats.java | 4 +- .../main/java/org/opensearch/node/Node.java | 19 +++---- .../node/ResponseCollectorService.java | 7 ++- .../opensearch/plugins/IndexStorePlugin.java | 8 +++ .../org/opensearch/repositories/IndexId.java | 4 +- .../IndexMetaDataGenerations.java | 4 +- .../opensearch/repositories/Repository.java | 7 ++- .../repositories/RepositoryData.java | 4 +- .../repositories/RepositoryInfo.java | 4 +- .../repositories/RepositoryShardId.java | 4 +- .../repositories/RepositoryStats.java | 4 +- .../repositories/RepositoryStatsSnapshot.java | 4 +- .../repositories/ShardGenerations.java | 7 ++- .../opensearch/script/AggregationScript.java | 7 ++- .../org/opensearch/script/FieldScript.java | 7 ++- .../org/opensearch/search/DocValueFormat.java | 4 +- .../org/opensearch/search/MultiValueMode.java | 4 +- .../org/opensearch/search/RescoreDocIds.java | 4 +- .../java/org/opensearch/search/SearchHit.java | 7 ++- .../org/opensearch/search/SearchHits.java | 4 +- .../opensearch/search/SearchPhaseResult.java | 4 +- .../opensearch/search/SearchShardTarget.java | 4 +- .../opensearch/search/SearchSortValues.java | 4 +- .../search/SearchSortValuesAndFormats.java | 4 +- .../search/aggregations/Aggregation.java | 4 +- .../aggregations/AggregationBuilder.java | 7 ++- .../aggregations/AggregationProcessor.java | 4 ++ .../search/aggregations/Aggregations.java | 4 +- .../search/aggregations/Aggregator.java | 8 ++- .../aggregations/AggregatorFactories.java | 7 ++- .../aggregations/AggregatorFactory.java | 4 +- .../BucketCollectorProcessor.java | 4 ++ .../aggregations/CardinalityUpperBound.java | 4 +- .../aggregations/InternalAggregation.java | 10 +++- .../aggregations/InternalAggregations.java | 4 +- .../MultiBucketConsumerService.java | 4 +- .../PipelineAggregationBuilder.java | 4 +- .../SearchContextAggregations.java | 5 +- .../bucket/LocalBucketCountThresholds.java | 4 +- .../bucket/terms/TermsAggregator.java | 4 +- .../pipeline/PipelineAggregator.java | 7 ++- .../aggregations/support/AggregationInfo.java | 4 +- .../aggregations/support/AggregationPath.java | 4 +- .../support/AggregationUsageService.java | 7 ++- .../aggregations/support/FieldContext.java | 4 +- .../aggregations/support/ValueType.java | 4 +- .../aggregations/support/ValuesSource.java | 4 +- .../support/ValuesSourceConfig.java | 4 +- .../support/ValuesSourceRegistry.java | 10 +++- .../support/ValuesSourceType.java | 4 +- .../search/builder/SearchSourceBuilder.java | 10 +++- .../opensearch/search/dfs/AggregatedDfs.java | 4 +- .../search/dfs/DfsSearchResult.java | 4 +- .../opensearch/search/fetch/FetchPhase.java | 4 +- .../search/fetch/FetchSearchResult.java | 4 +- .../search/fetch/StoredFieldsContext.java | 4 +- .../fetch/subphase/FetchDocValuesContext.java | 4 +- .../fetch/subphase/FetchFieldsContext.java | 5 +- .../search/fetch/subphase/FieldAndFormat.java | 4 +- .../fetch/subphase/InnerHitsContext.java | 7 ++- .../fetch/subphase/ScriptFieldsContext.java | 7 ++- .../subphase/highlight/HighlightField.java | 4 +- .../search/internal/ContextIndexSearcher.java | 4 +- .../internal/InternalSearchResponse.java | 4 +- .../search/internal/ReaderContext.java | 4 +- .../search/internal/ScrollContext.java | 4 +- .../search/internal/SearchContext.java | 4 +- .../search/internal/ShardSearchContextId.java | 4 +- .../search/internal/ShardSearchRequest.java | 4 +- .../opensearch/search/lookup/DocLookup.java | 4 +- .../search/lookup/LeafDocLookup.java | 4 +- .../search/lookup/LeafFieldsLookup.java | 4 +- .../search/lookup/LeafSearchLookup.java | 4 +- .../search/lookup/SearchLookup.java | 4 +- .../search/lookup/SourceLookup.java | 4 +- .../opensearch/search/profile/Profilers.java | 4 +- .../AggregationProfileBreakdown.java | 4 +- .../aggregation/AggregationProfiler.java | 4 +- .../query/InternalProfileComponent.java | 5 +- .../search/profile/query/QueryProfiler.java | 4 +- .../search/query/QueryCollectorContext.java | 4 +- .../opensearch/search/query/QueryPhase.java | 4 +- .../query/QueryPhaseExecutionException.java | 4 +- .../search/query/QueryPhaseSearcher.java | 4 +- .../search/query/QuerySearchResult.java | 4 +- .../search/query/ReduceableSearchResult.java | 5 +- .../search/rescore/RescoreContext.java | 4 +- .../opensearch/search/rescore/Rescorer.java | 4 +- .../search/rescore/RescorerBuilder.java | 4 +- .../opensearch/search/sort/BucketedSort.java | 16 ++++-- .../search/sort/NestedSortBuilder.java | 4 +- .../search/sort/SortAndFormats.java | 4 +- .../opensearch/search/sort/SortBuilder.java | 4 +- .../org/opensearch/search/sort/SortValue.java | 4 +- .../InternalSnapshotsInfoService.java | 5 +- .../snapshots/SnapshotShardSizeInfo.java | 4 +- .../org/opensearch/transport/Transport.java | 4 ++ .../transport/TransportException.java | 4 +- .../transport/TransportRequest.java | 4 +- .../transport/TransportRequestOptions.java | 10 +++- .../index/IndexServiceTestUtils.java | 20 +++++++ 354 files changed, 1400 insertions(+), 503 deletions(-) create mode 100644 server/src/test/java/org/opensearch/index/IndexServiceTestUtils.java diff --git a/libs/common/src/main/java/org/opensearch/common/Explicit.java b/libs/common/src/main/java/org/opensearch/common/Explicit.java index e060baf6f187e..da44c6fd4dcef 100644 --- a/libs/common/src/main/java/org/opensearch/common/Explicit.java +++ b/libs/common/src/main/java/org/opensearch/common/Explicit.java @@ -32,6 +32,8 @@ package org.opensearch.common; +import org.opensearch.common.annotation.PublicApi; + import java.util.Objects; /** @@ -43,8 +45,9 @@ * field mapping settings it is preferable to preserve an explicit * choice rather than a choice made only made implicitly by defaults. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class Explicit { private final T value; diff --git a/libs/common/src/main/java/org/opensearch/common/lease/Releasable.java b/libs/common/src/main/java/org/opensearch/common/lease/Releasable.java index 30bea6185febc..dfc4fefb9ee55 100644 --- a/libs/common/src/main/java/org/opensearch/common/lease/Releasable.java +++ b/libs/common/src/main/java/org/opensearch/common/lease/Releasable.java @@ -32,13 +32,16 @@ package org.opensearch.common.lease; +import org.opensearch.common.annotation.PublicApi; + import java.io.Closeable; /** * Specialization of {@link AutoCloseable} for calls that might not throw a checked exception. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface Releasable extends Closeable { @Override diff --git a/libs/core/src/main/java/org/opensearch/core/action/ActionResponse.java b/libs/core/src/main/java/org/opensearch/core/action/ActionResponse.java index 041d8b1bffb4a..7525bfb243aae 100644 --- a/libs/core/src/main/java/org/opensearch/core/action/ActionResponse.java +++ b/libs/core/src/main/java/org/opensearch/core/action/ActionResponse.java @@ -32,6 +32,7 @@ package org.opensearch.core.action; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.transport.TransportResponse; @@ -42,6 +43,7 @@ * * @opensearch.api */ +@PublicApi(since = "1.0.0") public abstract class ActionResponse extends TransportResponse { public ActionResponse() {} diff --git a/libs/core/src/main/java/org/opensearch/core/common/io/stream/Writeable.java b/libs/core/src/main/java/org/opensearch/core/common/io/stream/Writeable.java index af9df51655414..960f4bec5eeb5 100644 --- a/libs/core/src/main/java/org/opensearch/core/common/io/stream/Writeable.java +++ b/libs/core/src/main/java/org/opensearch/core/common/io/stream/Writeable.java @@ -32,6 +32,8 @@ package org.opensearch.core.common.io.stream; +import org.opensearch.common.annotation.PublicApi; + import java.io.IOException; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -41,8 +43,9 @@ * across the wire" using OpenSearch's internal protocol. If the implementer also implements equals and hashCode then a copy made by * serializing and deserializing must be equal and have the same hashCode. It isn't required that such a copy be entirely unchanged. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "2.8.0") public interface Writeable { /** * A WriteableRegistry registers {@link Writer} methods for writing data types over a @@ -135,8 +138,11 @@ public static Class getCustomClassFromInstance(final Object value) { * out.writeMapOfLists(someMap, StreamOutput::writeString, StreamOutput::writeString); * } * + * + * @opensearch.api */ @FunctionalInterface + @PublicApi(since = "2.8.0") interface Writer { /** @@ -161,8 +167,11 @@ interface Writer { * this.someMap = in.readMapOfLists(StreamInput::readString, StreamInput::readString); * } * + * + * @opensearch.api */ @FunctionalInterface + @PublicApi(since = "2.8.0") interface Reader { /** diff --git a/libs/core/src/main/java/org/opensearch/core/common/util/ByteArray.java b/libs/core/src/main/java/org/opensearch/core/common/util/ByteArray.java index e50f24417f8bc..f4d81c4ca4363 100644 --- a/libs/core/src/main/java/org/opensearch/core/common/util/ByteArray.java +++ b/libs/core/src/main/java/org/opensearch/core/common/util/ByteArray.java @@ -33,14 +33,16 @@ package org.opensearch.core.common.util; import org.apache.lucene.util.BytesRef; +import org.opensearch.common.annotation.PublicApi; import java.nio.ByteBuffer; /** * Abstraction of an array of byte values. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface ByteArray extends BigArray { /** diff --git a/libs/core/src/main/java/org/opensearch/core/indices/breaker/AllCircuitBreakerStats.java b/libs/core/src/main/java/org/opensearch/core/indices/breaker/AllCircuitBreakerStats.java index 3ce8b4953b9d6..992655efec8f0 100644 --- a/libs/core/src/main/java/org/opensearch/core/indices/breaker/AllCircuitBreakerStats.java +++ b/libs/core/src/main/java/org/opensearch/core/indices/breaker/AllCircuitBreakerStats.java @@ -32,6 +32,7 @@ package org.opensearch.core.indices.breaker; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; @@ -43,8 +44,9 @@ /** * Stats class encapsulating all of the different circuit breaker stats * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class AllCircuitBreakerStats implements Writeable, ToXContentFragment { /** An array of all the circuit breaker stats */ diff --git a/libs/core/src/main/java/org/opensearch/core/indices/breaker/CircuitBreakerService.java b/libs/core/src/main/java/org/opensearch/core/indices/breaker/CircuitBreakerService.java index ee9c94f432a36..dedeb0803271f 100644 --- a/libs/core/src/main/java/org/opensearch/core/indices/breaker/CircuitBreakerService.java +++ b/libs/core/src/main/java/org/opensearch/core/indices/breaker/CircuitBreakerService.java @@ -32,8 +32,7 @@ package org.opensearch.core.indices.breaker; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.lifecycle.AbstractLifecycleComponent; import org.opensearch.core.common.breaker.CircuitBreaker; @@ -41,11 +40,10 @@ * Interface for Circuit Breaker services, which provide breakers to classes * that load field data. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public abstract class CircuitBreakerService extends AbstractLifecycleComponent { - private static final Logger logger = LogManager.getLogger(CircuitBreakerService.class); - protected CircuitBreakerService() {} /** diff --git a/libs/core/src/main/java/org/opensearch/core/indices/breaker/CircuitBreakerStats.java b/libs/core/src/main/java/org/opensearch/core/indices/breaker/CircuitBreakerStats.java index 9207d3ea77227..ee71cf8d2ac0e 100644 --- a/libs/core/src/main/java/org/opensearch/core/indices/breaker/CircuitBreakerStats.java +++ b/libs/core/src/main/java/org/opensearch/core/indices/breaker/CircuitBreakerStats.java @@ -32,6 +32,7 @@ package org.opensearch.core.indices.breaker; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; @@ -45,8 +46,9 @@ /** * Class encapsulating stats about the {@link org.opensearch.core.common.breaker.CircuitBreaker} * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class CircuitBreakerStats implements Writeable, ToXContentObject { /** The name of the circuit breaker */ diff --git a/libs/core/src/main/java/org/opensearch/core/xcontent/ObjectParser.java b/libs/core/src/main/java/org/opensearch/core/xcontent/ObjectParser.java index 365b36c755dd2..04d0bce27c04f 100644 --- a/libs/core/src/main/java/org/opensearch/core/xcontent/ObjectParser.java +++ b/libs/core/src/main/java/org/opensearch/core/xcontent/ObjectParser.java @@ -32,6 +32,7 @@ package org.opensearch.core.xcontent; import org.opensearch.common.Nullable; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.ParseField; import java.io.IOException; @@ -83,8 +84,9 @@ * It's highly recommended to use the high level declare methods like {@link #declareString(BiConsumer, ParseField)} instead of * {@link #declareField} which can be used to implement exceptional parsing operations not covered by the high level methods. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public final class ObjectParser extends AbstractObjectParser implements BiFunction, diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBackpressureAndResiliencyIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBackpressureAndResiliencyIT.java index f19c9db7874db..6b6a96dc42a84 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBackpressureAndResiliencyIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBackpressureAndResiliencyIT.java @@ -14,7 +14,6 @@ import org.opensearch.action.admin.indices.flush.FlushResponse; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; -import org.opensearch.common.util.concurrent.AbstractAsyncTask; import org.opensearch.common.util.concurrent.UncategorizedExecutionException; import org.opensearch.core.common.bytes.BytesArray; import org.opensearch.core.common.bytes.BytesReference; @@ -22,6 +21,7 @@ import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; import org.opensearch.core.xcontent.MediaTypeRegistry; import org.opensearch.index.IndexService; +import org.opensearch.index.IndexServiceTestUtils; import org.opensearch.index.remote.RemoteSegmentTransferTracker; import org.opensearch.index.shard.IndexShard; import org.opensearch.indices.IndicesService; @@ -175,7 +175,7 @@ public void testAsyncTrimTaskSucceeds() { logger.info("Increasing the frequency of async trim task to ensure it runs in background while indexing"); IndexService indexService = internalCluster().getInstance(IndicesService.class, dataNodeName).iterator().next(); - ((AbstractAsyncTask) indexService.getTrimTranslogTask()).setInterval(TimeValue.timeValueMillis(100)); + IndexServiceTestUtils.setTrimTranslogTaskInterval(indexService, TimeValue.timeValueMillis(100)); logger.info("--> Indexing data"); indexData(randomIntBetween(2, 5), true); diff --git a/server/src/main/java/org/opensearch/action/AliasesRequest.java b/server/src/main/java/org/opensearch/action/AliasesRequest.java index 3632ba2d7304f..8fe6f49c08af0 100644 --- a/server/src/main/java/org/opensearch/action/AliasesRequest.java +++ b/server/src/main/java/org/opensearch/action/AliasesRequest.java @@ -32,14 +32,17 @@ package org.opensearch.action; +import org.opensearch.common.annotation.PublicApi; + /** * Needs to be implemented by all {@link org.opensearch.action.ActionRequest} subclasses that relate to * one or more indices and one or more aliases. Meant to be used for aliases management requests (e.g. add/remove alias, * get aliases) that hold aliases and indices in separate fields. * Allows to retrieve which indices and aliases the action relates to. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface AliasesRequest extends IndicesRequest.Replaceable { /** diff --git a/server/src/main/java/org/opensearch/action/OriginalIndices.java b/server/src/main/java/org/opensearch/action/OriginalIndices.java index 1e24c64bc60fc..1c26bf9d416f5 100644 --- a/server/src/main/java/org/opensearch/action/OriginalIndices.java +++ b/server/src/main/java/org/opensearch/action/OriginalIndices.java @@ -33,6 +33,7 @@ package org.opensearch.action; import org.opensearch.action.support.IndicesOptions; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; @@ -42,8 +43,9 @@ /** * Used to keep track of original indices within internal (e.g. shard level) requests * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public final class OriginalIndices implements IndicesRequest { // constant to use when original indices are not applicable and will not be serialized across the wire diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/allocation/ClusterAllocationExplanation.java b/server/src/main/java/org/opensearch/action/admin/cluster/allocation/ClusterAllocationExplanation.java index f4705a21f5014..3c8f07613561d 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/allocation/ClusterAllocationExplanation.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/allocation/ClusterAllocationExplanation.java @@ -40,6 +40,7 @@ import org.opensearch.cluster.routing.allocation.AllocationDecision; import org.opensearch.cluster.routing.allocation.ShardAllocationDecision; import org.opensearch.common.Nullable; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; @@ -58,8 +59,9 @@ * or if it is not unassigned, then which nodes it could possibly be relocated to. * It is an immutable class. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public final class ClusterAllocationExplanation implements ToXContentObject, Writeable { private final ShardRouting shardRouting; diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/SnapshotStatus.java b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/SnapshotStatus.java index 0b101070db478..e0f380b3ebbe6 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/SnapshotStatus.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/status/SnapshotStatus.java @@ -35,6 +35,7 @@ import org.opensearch.cluster.SnapshotsInProgress; import org.opensearch.cluster.SnapshotsInProgress.State; import org.opensearch.common.Nullable; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.ParseField; import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; @@ -68,8 +69,9 @@ /** * Status of a snapshot * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class SnapshotStatus implements ToXContentObject, Writeable { private final Snapshot snapshot; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/rollover/RolloverInfo.java b/server/src/main/java/org/opensearch/action/admin/indices/rollover/RolloverInfo.java index 475e44e1820d5..75c68350e2204 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/rollover/RolloverInfo.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/rollover/RolloverInfo.java @@ -34,6 +34,7 @@ import org.opensearch.cluster.AbstractDiffable; import org.opensearch.cluster.Diff; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.ParseField; import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; @@ -52,8 +53,9 @@ /** * Class for holding Rollover related information within an index * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class RolloverInfo extends AbstractDiffable implements Writeable, ToXContentFragment { public static final ParseField CONDITION_FIELD = new ParseField("met_conditions"); diff --git a/server/src/main/java/org/opensearch/action/search/ParsedScrollId.java b/server/src/main/java/org/opensearch/action/search/ParsedScrollId.java index 70be9a0b19e08..b723b97b5c413 100644 --- a/server/src/main/java/org/opensearch/action/search/ParsedScrollId.java +++ b/server/src/main/java/org/opensearch/action/search/ParsedScrollId.java @@ -32,13 +32,16 @@ package org.opensearch.action.search; +import org.opensearch.common.annotation.PublicApi; + import java.util.Arrays; /** * Search scroll id that has been parsed * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class ParsedScrollId { public static final String QUERY_THEN_FETCH_TYPE = "queryThenFetch"; diff --git a/server/src/main/java/org/opensearch/action/search/SearchContextIdForNode.java b/server/src/main/java/org/opensearch/action/search/SearchContextIdForNode.java index 59c0e54bb6cbc..7c702c16030b7 100644 --- a/server/src/main/java/org/opensearch/action/search/SearchContextIdForNode.java +++ b/server/src/main/java/org/opensearch/action/search/SearchContextIdForNode.java @@ -33,6 +33,7 @@ package org.opensearch.action.search; import org.opensearch.common.Nullable; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; @@ -43,8 +44,9 @@ /** * Id for a search context per node. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public final class SearchContextIdForNode implements Writeable { private final String node; private final ShardSearchContextId searchContextId; diff --git a/server/src/main/java/org/opensearch/action/search/SearchPhase.java b/server/src/main/java/org/opensearch/action/search/SearchPhase.java index 1c7b3c1f1563c..0890e9f5de8d4 100644 --- a/server/src/main/java/org/opensearch/action/search/SearchPhase.java +++ b/server/src/main/java/org/opensearch/action/search/SearchPhase.java @@ -32,6 +32,7 @@ package org.opensearch.action.search; import org.opensearch.common.CheckedRunnable; +import org.opensearch.common.annotation.PublicApi; import java.io.IOException; import java.util.Locale; @@ -40,8 +41,9 @@ /** * Base class for all individual search phases like collecting distributed frequencies, fetching documents, querying shards. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public abstract class SearchPhase implements CheckedRunnable { private final String name; private long startTimeInNanos; diff --git a/server/src/main/java/org/opensearch/action/search/SearchPhaseContext.java b/server/src/main/java/org/opensearch/action/search/SearchPhaseContext.java index 45d39a6f85ea2..0fa8569413eaf 100644 --- a/server/src/main/java/org/opensearch/action/search/SearchPhaseContext.java +++ b/server/src/main/java/org/opensearch/action/search/SearchPhaseContext.java @@ -34,6 +34,7 @@ import org.apache.logging.log4j.Logger; import org.opensearch.action.OriginalIndices; import org.opensearch.common.Nullable; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.lease.Releasable; import org.opensearch.common.util.concurrent.AtomicArray; import org.opensearch.search.SearchPhaseResult; @@ -48,8 +49,9 @@ /** * This class provide contextual state and access to resources across multiple search phases. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface SearchPhaseContext extends Executor { // TODO maybe we can make this concrete later - for now we just implement this in the base class for all initial phases diff --git a/server/src/main/java/org/opensearch/action/search/SearchRequestOperationsListener.java b/server/src/main/java/org/opensearch/action/search/SearchRequestOperationsListener.java index 89d725b56bded..ecf48b4a9f633 100644 --- a/server/src/main/java/org/opensearch/action/search/SearchRequestOperationsListener.java +++ b/server/src/main/java/org/opensearch/action/search/SearchRequestOperationsListener.java @@ -10,14 +10,16 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; +import org.opensearch.common.annotation.PublicApi; import java.util.List; /** * A listener for search, fetch and context events at the coordinator node level * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface SearchRequestOperationsListener { void onPhaseStart(SearchPhaseContext context); diff --git a/server/src/main/java/org/opensearch/action/search/SearchShardIterator.java b/server/src/main/java/org/opensearch/action/search/SearchShardIterator.java index fbd85a3fc0b8f..33d9bf70021a0 100644 --- a/server/src/main/java/org/opensearch/action/search/SearchShardIterator.java +++ b/server/src/main/java/org/opensearch/action/search/SearchShardIterator.java @@ -36,6 +36,7 @@ import org.opensearch.cluster.routing.PlainShardIterator; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.common.Nullable; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.Countable; import org.opensearch.common.util.PlainIterator; @@ -54,8 +55,9 @@ * the cluster alias. * @see OriginalIndices * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public final class SearchShardIterator implements Comparable, Countable { private final OriginalIndices originalIndices; diff --git a/server/src/main/java/org/opensearch/action/search/SearchShardTask.java b/server/src/main/java/org/opensearch/action/search/SearchShardTask.java index 54faaf363cb70..dfecf4f462c4d 100644 --- a/server/src/main/java/org/opensearch/action/search/SearchShardTask.java +++ b/server/src/main/java/org/opensearch/action/search/SearchShardTask.java @@ -33,6 +33,7 @@ package org.opensearch.action.search; import org.opensearch.common.MemoizedSupplier; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.tasks.TaskId; import org.opensearch.search.fetch.ShardFetchSearchRequest; import org.opensearch.search.internal.ShardSearchRequest; @@ -46,8 +47,9 @@ * Task storing information about a currently running search shard request. * See {@link ShardSearchRequest}, {@link ShardFetchSearchRequest}, ... * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class SearchShardTask extends CancellableTask implements SearchBackpressureTask { // generating metadata in a lazy way since source can be quite big private final MemoizedSupplier metadataSupplier; 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 4d3f9b71a919c..ef490844db692 100644 --- a/server/src/main/java/org/opensearch/action/search/ShardSearchFailure.java +++ b/server/src/main/java/org/opensearch/action/search/ShardSearchFailure.java @@ -37,6 +37,7 @@ import org.opensearch.action.OriginalIndices; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.Nullable; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.action.ShardOperationFailedException; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; @@ -56,8 +57,9 @@ /** * Represents a failure to search on a specific shard. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class ShardSearchFailure extends ShardOperationFailedException { private static final String REASON_FIELD = "reason"; diff --git a/server/src/main/java/org/opensearch/action/support/ActiveShardCount.java b/server/src/main/java/org/opensearch/action/support/ActiveShardCount.java index 15275ba48fc6e..e91342a7ce4b8 100644 --- a/server/src/main/java/org/opensearch/action/support/ActiveShardCount.java +++ b/server/src/main/java/org/opensearch/action/support/ActiveShardCount.java @@ -36,6 +36,7 @@ import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.routing.IndexRoutingTable; import org.opensearch.cluster.routing.IndexShardRoutingTable; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; @@ -48,8 +49,9 @@ * A class whose instances represent a value for counting the number * of active shard copies for a given shard in an index. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public final class ActiveShardCount implements Writeable { private static final int ACTIVE_SHARD_COUNT_DEFAULT = -2; diff --git a/server/src/main/java/org/opensearch/action/support/RetryableAction.java b/server/src/main/java/org/opensearch/action/support/RetryableAction.java index fc2ee277b538a..d555a876f20fb 100644 --- a/server/src/main/java/org/opensearch/action/support/RetryableAction.java +++ b/server/src/main/java/org/opensearch/action/support/RetryableAction.java @@ -36,6 +36,7 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.action.ActionRunnable; import org.opensearch.action.bulk.BackoffPolicy; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.unit.TimeValue; import org.opensearch.core.action.ActionListener; import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; @@ -52,8 +53,9 @@ * default. The action will be retried with exponentially increasing delay periods until the timeout period * has been reached. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public abstract class RetryableAction { private final Logger logger; diff --git a/server/src/main/java/org/opensearch/action/support/replication/PendingReplicationActions.java b/server/src/main/java/org/opensearch/action/support/replication/PendingReplicationActions.java index c52bc6c2c799d..bb6bfc6fd4773 100644 --- a/server/src/main/java/org/opensearch/action/support/replication/PendingReplicationActions.java +++ b/server/src/main/java/org/opensearch/action/support/replication/PendingReplicationActions.java @@ -33,6 +33,7 @@ package org.opensearch.action.support.replication; import org.opensearch.action.support.RetryableAction; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.lease.Releasable; import org.opensearch.common.util.concurrent.ConcurrentCollections; import org.opensearch.core.index.shard.ShardId; @@ -51,8 +52,9 @@ /** * Pending Replication Actions * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class PendingReplicationActions implements Consumer, Releasable { private final Map>> onGoingReplicationActions = ConcurrentCollections.newConcurrentMap(); diff --git a/server/src/main/java/org/opensearch/action/support/replication/ReplicationMode.java b/server/src/main/java/org/opensearch/action/support/replication/ReplicationMode.java index f9b85cc4bd7aa..dd3a38c1398ab 100644 --- a/server/src/main/java/org/opensearch/action/support/replication/ReplicationMode.java +++ b/server/src/main/java/org/opensearch/action/support/replication/ReplicationMode.java @@ -8,11 +8,14 @@ package org.opensearch.action.support.replication; +import org.opensearch.common.annotation.PublicApi; + /** * The type of replication used for inter-node replication. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "2.4.0") public enum ReplicationMode { /** * In this mode, a {@code TransportReplicationAction} is fanned out to underlying concerned shard and is replicated logically. 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 825405cb68549..67480976c500f 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 @@ -36,6 +36,7 @@ import org.opensearch.OpenSearchException; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.Nullable; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.action.ActionResponse; import org.opensearch.core.action.ShardOperationFailedException; import org.opensearch.core.common.io.stream.StreamInput; @@ -57,8 +58,9 @@ /** * Base class for write action responses. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class ReplicationResponse extends ActionResponse { public static final ReplicationResponse.ShardInfo.Failure[] EMPTY = new ReplicationResponse.ShardInfo.Failure[0]; @@ -88,8 +90,9 @@ public void setShardInfo(ShardInfo shardInfo) { /** * Holds shard information * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") public static class ShardInfo implements Writeable, ToXContentObject { private static final String TOTAL = "total"; @@ -235,8 +238,9 @@ public String toString() { /** * Holds failure information * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") public static class Failure extends ShardOperationFailedException implements ToXContentObject { private static final String _INDEX = "_index"; diff --git a/server/src/main/java/org/opensearch/cluster/ClusterState.java b/server/src/main/java/org/opensearch/cluster/ClusterState.java index 1b87a60c2ccf5..9e63f961d241d 100644 --- a/server/src/main/java/org/opensearch/cluster/ClusterState.java +++ b/server/src/main/java/org/opensearch/cluster/ClusterState.java @@ -49,6 +49,7 @@ import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.UUIDs; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.settings.Settings; import org.opensearch.core.common.Strings; @@ -97,8 +98,9 @@ * throws the {@link IncompatibleClusterStateVersionException}, which causes the publishing mechanism to send * a full version of the cluster state to the node on which this exception was thrown. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class ClusterState implements ToXContentFragment, Diffable { public static final ClusterState EMPTY_STATE = builder(ClusterName.CLUSTER_NAME_SETTING.getDefault(Settings.EMPTY)).build(); @@ -139,8 +141,9 @@ static boolean shouldSerializ /** * Custom cluster state. * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") public interface Custom extends NamedDiffable, ToXContentFragment, FeatureAware { /** @@ -596,8 +599,9 @@ public static Builder builder(ClusterState state) { /** * Builder for cluster state. * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") public static class Builder { private final ClusterName clusterName; diff --git a/server/src/main/java/org/opensearch/cluster/ClusterStateApplier.java b/server/src/main/java/org/opensearch/cluster/ClusterStateApplier.java index 140e6426bb801..d7702729ae884 100644 --- a/server/src/main/java/org/opensearch/cluster/ClusterStateApplier.java +++ b/server/src/main/java/org/opensearch/cluster/ClusterStateApplier.java @@ -33,13 +33,15 @@ package org.opensearch.cluster; import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.annotation.PublicApi; /** * A component that is in charge of applying an incoming cluster state to the node internal data structures. * The single apply method is called before the cluster state becomes visible via {@link ClusterService#state()}. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface ClusterStateApplier { /** diff --git a/server/src/main/java/org/opensearch/cluster/ClusterStateListener.java b/server/src/main/java/org/opensearch/cluster/ClusterStateListener.java index 01a8e51a3d13e..57fcfcad099ed 100644 --- a/server/src/main/java/org/opensearch/cluster/ClusterStateListener.java +++ b/server/src/main/java/org/opensearch/cluster/ClusterStateListener.java @@ -32,11 +32,14 @@ package org.opensearch.cluster; +import org.opensearch.common.annotation.PublicApi; + /** * A listener to be notified when a cluster state changes. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface ClusterStateListener { /** diff --git a/server/src/main/java/org/opensearch/cluster/ClusterStateTaskConfig.java b/server/src/main/java/org/opensearch/cluster/ClusterStateTaskConfig.java index 9a4b708548a7d..149d93a158007 100644 --- a/server/src/main/java/org/opensearch/cluster/ClusterStateTaskConfig.java +++ b/server/src/main/java/org/opensearch/cluster/ClusterStateTaskConfig.java @@ -33,13 +33,15 @@ import org.opensearch.common.Nullable; import org.opensearch.common.Priority; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.unit.TimeValue; /** * Cluster state update task configuration for timeout and priority * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface ClusterStateTaskConfig { /** * The timeout for this cluster state update task configuration. If diff --git a/server/src/main/java/org/opensearch/cluster/ClusterStateTaskExecutor.java b/server/src/main/java/org/opensearch/cluster/ClusterStateTaskExecutor.java index bf8494cc36857..115816798959e 100644 --- a/server/src/main/java/org/opensearch/cluster/ClusterStateTaskExecutor.java +++ b/server/src/main/java/org/opensearch/cluster/ClusterStateTaskExecutor.java @@ -33,6 +33,7 @@ import org.opensearch.cluster.service.ClusterManagerTaskThrottler; import org.opensearch.common.Nullable; +import org.opensearch.common.annotation.PublicApi; import java.util.IdentityHashMap; import java.util.List; @@ -41,8 +42,9 @@ /** * Interface that updates the cluster state based on the task * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface ClusterStateTaskExecutor { /** * Update the cluster state based on the current state and the given tasks. Return the *same instance* if no state @@ -103,8 +105,9 @@ default ClusterManagerTaskThrottler.ThrottlingKey getClusterManagerThrottlingKey * Represents the result of a batched execution of cluster state update tasks * @param the type of the cluster state update task * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") class ClusterTasksResult { @Nullable public final ClusterState resultingState; @@ -127,8 +130,9 @@ public static Builder builder() { /** * Builder for cluster state task. * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") public static class Builder { private final Map executionResults = new IdentityHashMap<>(); diff --git a/server/src/main/java/org/opensearch/cluster/ClusterStateTaskListener.java b/server/src/main/java/org/opensearch/cluster/ClusterStateTaskListener.java index d6c4abfad7b8d..0cb24bd3f3eab 100644 --- a/server/src/main/java/org/opensearch/cluster/ClusterStateTaskListener.java +++ b/server/src/main/java/org/opensearch/cluster/ClusterStateTaskListener.java @@ -32,14 +32,16 @@ package org.opensearch.cluster; import org.opensearch.cluster.service.ClusterManagerService; +import org.opensearch.common.annotation.PublicApi; import java.util.List; /** * Interface to implement a cluster state change listener * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface ClusterStateTaskListener { /** diff --git a/server/src/main/java/org/opensearch/cluster/ClusterStateUpdateTask.java b/server/src/main/java/org/opensearch/cluster/ClusterStateUpdateTask.java index 9225914a931b2..ae6626dd4785d 100644 --- a/server/src/main/java/org/opensearch/cluster/ClusterStateUpdateTask.java +++ b/server/src/main/java/org/opensearch/cluster/ClusterStateUpdateTask.java @@ -34,6 +34,7 @@ import org.opensearch.common.Nullable; import org.opensearch.common.Priority; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.unit.TimeValue; import java.util.List; @@ -41,8 +42,9 @@ /** * A task that can update the cluster state. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public abstract class ClusterStateUpdateTask implements ClusterStateTaskConfig, diff --git a/server/src/main/java/org/opensearch/cluster/LocalNodeClusterManagerListener.java b/server/src/main/java/org/opensearch/cluster/LocalNodeClusterManagerListener.java index c07dcc5daaee6..0b2ed7ef66d43 100644 --- a/server/src/main/java/org/opensearch/cluster/LocalNodeClusterManagerListener.java +++ b/server/src/main/java/org/opensearch/cluster/LocalNodeClusterManagerListener.java @@ -31,12 +31,15 @@ package org.opensearch.cluster; +import org.opensearch.common.annotation.PublicApi; + /** * Enables listening to cluster-manager changes events of the local node (when the local node becomes the cluster-manager, and when the local * node cease being a cluster-manager). * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface LocalNodeClusterManagerListener extends ClusterStateListener { /** diff --git a/server/src/main/java/org/opensearch/cluster/LocalNodeMasterListener.java b/server/src/main/java/org/opensearch/cluster/LocalNodeMasterListener.java index 31c0b294b8004..2487aaf0d7c51 100644 --- a/server/src/main/java/org/opensearch/cluster/LocalNodeMasterListener.java +++ b/server/src/main/java/org/opensearch/cluster/LocalNodeMasterListener.java @@ -31,14 +31,17 @@ package org.opensearch.cluster; +import org.opensearch.common.annotation.DeprecatedApi; + /** * Enables listening to cluster-manager changes events of the local node (when the local node becomes the cluster-manager, and when the local * node cease being a cluster-manager). * - * @opensearch.internal + * @opensearch.api * @deprecated As of 2.2, because supporting inclusive language, replaced by {@link LocalNodeClusterManagerListener} */ @Deprecated +@DeprecatedApi(since = "2.2.0") public interface LocalNodeMasterListener extends LocalNodeClusterManagerListener { /** diff --git a/server/src/main/java/org/opensearch/cluster/NamedDiff.java b/server/src/main/java/org/opensearch/cluster/NamedDiff.java index ce971aa723394..e994cfb224386 100644 --- a/server/src/main/java/org/opensearch/cluster/NamedDiff.java +++ b/server/src/main/java/org/opensearch/cluster/NamedDiff.java @@ -33,13 +33,15 @@ package org.opensearch.cluster; import org.opensearch.Version; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.io.stream.NamedWriteable; /** * Diff that also support NamedWriteable interface * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface NamedDiff> extends Diff, NamedWriteable { /** * The minimal version of the recipient this custom object can be sent to diff --git a/server/src/main/java/org/opensearch/cluster/NodeConnectionsService.java b/server/src/main/java/org/opensearch/cluster/NodeConnectionsService.java index 5b7bb19a935b1..1c12c260b3929 100644 --- a/server/src/main/java/org/opensearch/cluster/NodeConnectionsService.java +++ b/server/src/main/java/org/opensearch/cluster/NodeConnectionsService.java @@ -43,6 +43,7 @@ import org.opensearch.cluster.node.DiscoveryNodes; import org.opensearch.cluster.service.ClusterApplier; import org.opensearch.common.Nullable; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.inject.Inject; import org.opensearch.common.lifecycle.AbstractLifecycleComponent; import org.opensearch.common.settings.Setting; @@ -81,8 +82,9 @@ * This component does not block on disconnections at all, because a disconnection might need to wait for an ongoing (background) connection * attempt to complete first. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class NodeConnectionsService extends AbstractLifecycleComponent { private static final Logger logger = LogManager.getLogger(NodeConnectionsService.class); diff --git a/server/src/main/java/org/opensearch/cluster/RestoreInProgress.java b/server/src/main/java/org/opensearch/cluster/RestoreInProgress.java index 3e0c78099e6b4..769f97373f7b7 100644 --- a/server/src/main/java/org/opensearch/cluster/RestoreInProgress.java +++ b/server/src/main/java/org/opensearch/cluster/RestoreInProgress.java @@ -34,6 +34,7 @@ import org.opensearch.Version; import org.opensearch.cluster.ClusterState.Custom; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; @@ -54,8 +55,9 @@ /** * Meta data about restore processes that are currently executing * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class RestoreInProgress extends AbstractNamedDiffable implements Custom, Iterable { /** @@ -139,8 +141,9 @@ public RestoreInProgress build() { /** * Restore metadata * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") public static class Entry { private final String uuid; private final State state; @@ -238,8 +241,9 @@ public int hashCode() { /** * Represents status of a restored shard * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") public static class ShardRestoreStatus implements Writeable { private State state; private String nodeId; @@ -363,8 +367,9 @@ public int hashCode() { /** * Shard restore process state * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") public enum State { /** * Initializing state diff --git a/server/src/main/java/org/opensearch/cluster/SnapshotsInProgress.java b/server/src/main/java/org/opensearch/cluster/SnapshotsInProgress.java index 952591f346536..3de23d2490c63 100644 --- a/server/src/main/java/org/opensearch/cluster/SnapshotsInProgress.java +++ b/server/src/main/java/org/opensearch/cluster/SnapshotsInProgress.java @@ -35,6 +35,7 @@ import org.opensearch.Version; import org.opensearch.cluster.ClusterState.Custom; import org.opensearch.common.Nullable; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.unit.TimeValue; import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; @@ -928,8 +929,9 @@ public String toString() { /** * State of the snapshots. * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") public enum State { INIT((byte) 0, false), STARTED((byte) 1, false), diff --git a/server/src/main/java/org/opensearch/cluster/TimeoutClusterStateListener.java b/server/src/main/java/org/opensearch/cluster/TimeoutClusterStateListener.java index eb31fa2b7e69d..f0fa5af64d157 100644 --- a/server/src/main/java/org/opensearch/cluster/TimeoutClusterStateListener.java +++ b/server/src/main/java/org/opensearch/cluster/TimeoutClusterStateListener.java @@ -32,13 +32,15 @@ package org.opensearch.cluster; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.unit.TimeValue; /** * An exception to cluster state listener that allows for timeouts and for post added notifications. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface TimeoutClusterStateListener extends ClusterStateListener { void postAdded(); diff --git a/server/src/main/java/org/opensearch/cluster/ack/ClusterStateUpdateResponse.java b/server/src/main/java/org/opensearch/cluster/ack/ClusterStateUpdateResponse.java index db26496c6f263..837bf8af449ad 100644 --- a/server/src/main/java/org/opensearch/cluster/ack/ClusterStateUpdateResponse.java +++ b/server/src/main/java/org/opensearch/cluster/ack/ClusterStateUpdateResponse.java @@ -32,11 +32,14 @@ package org.opensearch.cluster.ack; +import org.opensearch.common.annotation.PublicApi; + /** * Base response returned after a cluster state update * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class ClusterStateUpdateResponse { private final boolean acknowledged; diff --git a/server/src/main/java/org/opensearch/cluster/block/ClusterBlockException.java b/server/src/main/java/org/opensearch/cluster/block/ClusterBlockException.java index 4673f075e8439..ea6c0c69c523d 100644 --- a/server/src/main/java/org/opensearch/cluster/block/ClusterBlockException.java +++ b/server/src/main/java/org/opensearch/cluster/block/ClusterBlockException.java @@ -33,6 +33,7 @@ package org.opensearch.cluster.block; import org.opensearch.OpenSearchException; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.rest.RestStatus; @@ -49,8 +50,9 @@ /** * Internal exception on obtaining a cluster block * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class ClusterBlockException extends OpenSearchException { private final Set blocks; diff --git a/server/src/main/java/org/opensearch/cluster/block/ClusterBlocks.java b/server/src/main/java/org/opensearch/cluster/block/ClusterBlocks.java index bcb4f7f71268e..304136166d515 100644 --- a/server/src/main/java/org/opensearch/cluster/block/ClusterBlocks.java +++ b/server/src/main/java/org/opensearch/cluster/block/ClusterBlocks.java @@ -37,6 +37,7 @@ import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.MetadataIndexStateService; import org.opensearch.common.Nullable; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.util.set.Sets; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; @@ -59,8 +60,9 @@ /** * Represents current cluster level blocks to block dirty operations done against the cluster. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class ClusterBlocks extends AbstractDiffable { public static final ClusterBlocks EMPTY_CLUSTER_BLOCK = new ClusterBlocks(emptySet(), Map.of()); @@ -355,8 +357,9 @@ public static Builder builder() { /** * Builder for cluster blocks. * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") public static class Builder { private final Set global = new HashSet<>(); diff --git a/server/src/main/java/org/opensearch/cluster/block/IndexCreateBlockException.java b/server/src/main/java/org/opensearch/cluster/block/IndexCreateBlockException.java index 729d76c72e99e..3e9fa3e23a12f 100644 --- a/server/src/main/java/org/opensearch/cluster/block/IndexCreateBlockException.java +++ b/server/src/main/java/org/opensearch/cluster/block/IndexCreateBlockException.java @@ -8,6 +8,7 @@ package org.opensearch.cluster.block; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.io.stream.StreamInput; import java.io.IOException; @@ -16,8 +17,9 @@ /** * Internal exception on obtaining an index create block * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class IndexCreateBlockException extends ClusterBlockException { public IndexCreateBlockException(Set globalLevelBlocks) { diff --git a/server/src/main/java/org/opensearch/cluster/coordination/ClusterStatePublisher.java b/server/src/main/java/org/opensearch/cluster/coordination/ClusterStatePublisher.java index 3a506397bcac8..0ef8262a216ee 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/ClusterStatePublisher.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/ClusterStatePublisher.java @@ -34,14 +34,16 @@ import org.opensearch.cluster.ClusterChangedEvent; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.common.Nullable; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.unit.TimeValue; import org.opensearch.core.action.ActionListener; /** * Publishes the cluster state * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface ClusterStatePublisher { /** * Publish all the changes to the cluster from the cluster-manager (can be called just by the cluster-manager). The publish @@ -59,8 +61,9 @@ public interface ClusterStatePublisher { /** * An acknowledgement listener. * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") interface AckListener { /** * Should be called when the cluster coordination layer has committed the cluster state (i.e. even if this publication fails, diff --git a/server/src/main/java/org/opensearch/cluster/coordination/CoordinationMetadata.java b/server/src/main/java/org/opensearch/cluster/coordination/CoordinationMetadata.java index 869c86ae75b90..53398d6f3f98f 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/CoordinationMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/CoordinationMetadata.java @@ -32,6 +32,7 @@ package org.opensearch.cluster.coordination; import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.util.set.Sets; import org.opensearch.core.ParseField; import org.opensearch.core.common.io.stream.StreamInput; @@ -55,8 +56,9 @@ /** * Metadata for cluster coordination * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class CoordinationMetadata implements Writeable, ToXContentFragment { public static final CoordinationMetadata EMPTY_METADATA = builder().build(); @@ -214,8 +216,9 @@ public String toString() { /** * Builder for coordination metadata. * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") public static class Builder { private long term = 0; private VotingConfiguration lastCommittedConfiguration = VotingConfiguration.EMPTY_CONFIG; @@ -266,8 +269,9 @@ public CoordinationMetadata build() { /** * Excluded nodes from voting config. * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") public static class VotingConfigExclusion implements Writeable, ToXContentFragment { public static final String MISSING_VALUE_MARKER = "_absent_"; private final String nodeId; @@ -362,8 +366,9 @@ public String toString() { /** * A collection of persistent node ids, denoting the voting configuration for cluster state changes. * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") public static class VotingConfiguration implements Writeable, ToXContentFragment { public static final VotingConfiguration EMPTY_CONFIG = new VotingConfiguration(Collections.emptySet()); diff --git a/server/src/main/java/org/opensearch/cluster/coordination/PersistedStateStats.java b/server/src/main/java/org/opensearch/cluster/coordination/PersistedStateStats.java index 4d466c4b3ad73..0b7ed4fee5775 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/PersistedStateStats.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/PersistedStateStats.java @@ -8,6 +8,7 @@ package org.opensearch.cluster.coordination; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; import org.opensearch.core.common.io.stream.Writeable; @@ -22,8 +23,9 @@ /** * Persisted cluster state related stats. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "2.12.0") public class PersistedStateStats implements Writeable, ToXContentObject { private final String statsName; private AtomicLong totalTimeInMillis = new AtomicLong(0); diff --git a/server/src/main/java/org/opensearch/cluster/decommission/DecommissionAttributeMetadata.java b/server/src/main/java/org/opensearch/cluster/decommission/DecommissionAttributeMetadata.java index 3df807a4f94d3..254d70b0422d7 100644 --- a/server/src/main/java/org/opensearch/cluster/decommission/DecommissionAttributeMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/decommission/DecommissionAttributeMetadata.java @@ -14,6 +14,7 @@ import org.opensearch.cluster.NamedDiff; import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.metadata.Metadata.Custom; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; @@ -30,8 +31,9 @@ /** * Contains metadata about decommission attribute * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "2.4.0") public class DecommissionAttributeMetadata extends AbstractNamedDiffable implements Custom { public static final String TYPE = "decommissionedAttribute"; diff --git a/server/src/main/java/org/opensearch/cluster/metadata/ComponentTemplate.java b/server/src/main/java/org/opensearch/cluster/metadata/ComponentTemplate.java index cab1b75afe3e1..abc3712ee07e3 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/ComponentTemplate.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/ComponentTemplate.java @@ -35,6 +35,7 @@ import org.opensearch.cluster.AbstractDiffable; import org.opensearch.cluster.Diff; import org.opensearch.common.Nullable; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.ParseField; import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; @@ -55,8 +56,9 @@ * contains a field "foo", it's expected to contain all the necessary settings/mappings/etc for the * "foo" field. These component templates make up the individual pieces composing an index template. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class ComponentTemplate extends AbstractDiffable implements ToXContentObject { private static final ParseField TEMPLATE = new ParseField("template"); private static final ParseField VERSION = new ParseField("version"); diff --git a/server/src/main/java/org/opensearch/cluster/metadata/ComposableIndexTemplate.java b/server/src/main/java/org/opensearch/cluster/metadata/ComposableIndexTemplate.java index 5f1291f6c6d82..e7f1b97f28842 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/ComposableIndexTemplate.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/ComposableIndexTemplate.java @@ -36,6 +36,7 @@ import org.opensearch.cluster.Diff; import org.opensearch.cluster.metadata.DataStream.TimestampField; import org.opensearch.common.Nullable; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.ParseField; import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; @@ -63,8 +64,9 @@ * ids corresponding to component templates that should be composed in order when creating a new * index. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class ComposableIndexTemplate extends AbstractDiffable implements ToXContentObject { private static final ParseField INDEX_PATTERNS = new ParseField("index_patterns"); private static final ParseField TEMPLATE = new ParseField("template"); @@ -284,8 +286,9 @@ public String toString() { /** * Template for data stream. * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") public static class DataStreamTemplate implements Writeable, ToXContentObject { private static final ParseField TIMESTAMP_FIELD_FIELD = new ParseField("timestamp_field"); diff --git a/server/src/main/java/org/opensearch/cluster/metadata/DiffableStringMap.java b/server/src/main/java/org/opensearch/cluster/metadata/DiffableStringMap.java index 8209c7bb56ad8..a8102182576ff 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/DiffableStringMap.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/DiffableStringMap.java @@ -34,6 +34,7 @@ import org.opensearch.cluster.Diff; import org.opensearch.cluster.Diffable; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; @@ -50,8 +51,9 @@ * This is a {@code Map} that implements AbstractDiffable so it * can be used for cluster state purposes * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class DiffableStringMap extends AbstractMap implements Diffable { public static final DiffableStringMap EMPTY = new DiffableStringMap(Collections.emptyMap()); 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 0c316373e484f..0f3ee894a7f63 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/IndexAbstraction.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/IndexAbstraction.java @@ -33,6 +33,7 @@ import org.opensearch.common.Nullable; import org.opensearch.common.SetOnce; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.collect.Tuple; import org.opensearch.core.common.Strings; @@ -52,8 +53,9 @@ * An index abstraction has a unique name and encapsulates all the {@link IndexMetadata} instances it is pointing to. * Also depending on type it may refer to a single or many concrete indices and may or may not have a write index. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public interface IndexAbstraction { /** @@ -102,7 +104,10 @@ public interface IndexAbstraction { /** * An index abstraction type. + * + * @opensearch.api */ + @PublicApi(since = "1.0.0") enum Type { /** @@ -335,8 +340,9 @@ private boolean isNonEmpty(List idxMetas) { /** * A data stream. * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") class DataStream implements IndexAbstraction { private final org.opensearch.cluster.metadata.DataStream dataStream; diff --git a/server/src/main/java/org/opensearch/cluster/metadata/IndexGraveyard.java b/server/src/main/java/org/opensearch/cluster/metadata/IndexGraveyard.java index 85a203e5e059a..b4f8b6b188531 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/IndexGraveyard.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/IndexGraveyard.java @@ -35,6 +35,7 @@ import org.opensearch.Version; import org.opensearch.cluster.Diff; import org.opensearch.cluster.NamedDiff; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; import org.opensearch.common.time.DateFormatter; @@ -68,8 +69,9 @@ * tombstones remain in the cluster state for a fixed period of time, after which * they are purged. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public final class IndexGraveyard implements Metadata.Custom { /** @@ -191,8 +193,9 @@ public static IndexGraveyard.Builder builder(final IndexGraveyard graveyard) { /** * A class to build an IndexGraveyard. * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") public static final class Builder { private List tombstones; private int numPurged = -1; @@ -367,8 +370,9 @@ public String getWriteableName() { /** * An individual tombstone entry for representing a deleted index. * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") public static final class Tombstone implements ToXContentObject, Writeable { private static final String INDEX_KEY = "index"; 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 2e1421c278879..03784df509ed6 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java @@ -97,8 +97,9 @@ /** * Index metadata information * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class IndexMetadata implements Diffable, ToXContentFragment { public static final ClusterBlock INDEX_READ_ONLY_BLOCK = new ClusterBlock( @@ -160,8 +161,9 @@ public class IndexMetadata implements Diffable, ToXContentFragmen /** * The state of the index. * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") public enum State { OPEN((byte) 0), CLOSE((byte) 1); @@ -1211,8 +1213,9 @@ public static Builder builder(IndexMetadata indexMetadata) { /** * Builder of index metadata. * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") public static class Builder { private String index; 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 70c1d059a1b9e..1871ed24973c2 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/Metadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/Metadata.java @@ -49,6 +49,7 @@ import org.opensearch.cluster.decommission.DecommissionAttributeMetadata; import org.opensearch.common.Nullable; import org.opensearch.common.UUIDs; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.regex.Regex; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Setting.Property; @@ -98,8 +99,9 @@ /** * Metadata information * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class Metadata implements Iterable, Diffable, ToXContentFragment { private static final Logger logger = LogManager.getLogger(Metadata.class); @@ -127,8 +129,9 @@ public boolean isSegmentReplicationEnabled(String indexName) { /** * Context of the XContent. * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") public enum XContentContext { /* Custom metadata should be returns as part of API call */ API, @@ -166,8 +169,9 @@ public enum XContentContext { /** * Custom metadata. * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") public interface Custom extends NamedDiffable, ToXContentFragment, ClusterState.FeatureAware { EnumSet context(); @@ -1129,8 +1133,9 @@ public static Builder builder(Metadata metadata) { /** * Builder of metadata. * - * @opensearch.internal + * @opensearch.api */ + @PublicApi(since = "1.0.0") public static class Builder { private String clusterUUID; diff --git a/server/src/main/java/org/opensearch/cluster/metadata/Template.java b/server/src/main/java/org/opensearch/cluster/metadata/Template.java index 8e367c71ed166..bd110c6af8975 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/Template.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/Template.java @@ -34,6 +34,7 @@ import org.opensearch.cluster.AbstractDiffable; import org.opensearch.common.Nullable; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.compress.CompressedXContent; import org.opensearch.common.settings.Settings; import org.opensearch.common.xcontent.XContentHelper; @@ -58,8 +59,9 @@ * it is entirely independent from an index. It's a building block forming part of a regular index * template and a {@link ComponentTemplate}. * - * @opensearch.internal + * @opensearch.api */ +@PublicApi(since = "1.0.0") public class Template extends AbstractDiffable