From 584617e519226b93cda6eb014a5baadf968e34c2 Mon Sep 17 00:00:00 2001 From: Sagar <99425694+sgup432@users.noreply.github.com> Date: Sat, 10 Jun 2023 00:38:28 -0700 Subject: [PATCH 001/109] Task cancellation monitoring service (#7642) * Task cancellation monitoring service Signed-off-by: Sagar Upadhyaya --- CHANGELOG.md | 1 + .../admin/cluster/node/stats/NodeStats.java | 24 +- .../cluster/node/stats/NodesStatsRequest.java | 3 +- .../node/stats/TransportNodesStatsAction.java | 3 +- .../stats/TransportClusterStatsAction.java | 1 + .../common/settings/ClusterSettings.java | 7 +- .../main/java/org/opensearch/node/Node.java | 17 +- .../java/org/opensearch/node/NodeService.java | 16 +- .../SearchShardTaskCancellationStats.java | 75 ++++ .../TaskCancellationMonitoringService.java | 179 +++++++++ .../TaskCancellationMonitoringSettings.java | 93 +++++ .../tasks/TaskCancellationStats.java | 64 +++ .../org/opensearch/tasks/TaskManager.java | 34 ++ .../cluster/node/stats/NodeStatsTests.java | 1 + .../node/tasks/CancellableTasksTests.java | 31 ++ .../opensearch/cluster/DiskUsageTests.java | 6 + ...SearchShardTaskCancellationStatsTests.java | 28 ++ ...askCancellationMonitoringServiceTests.java | 371 ++++++++++++++++++ ...skCancellationMonitoringSettingsTests.java | 39 ++ .../tasks/TaskCancellationStatsTests.java | 28 ++ .../MockInternalClusterInfoService.java | 3 +- .../opensearch/test/InternalTestCluster.java | 1 + 22 files changed, 1016 insertions(+), 9 deletions(-) create mode 100644 server/src/main/java/org/opensearch/tasks/SearchShardTaskCancellationStats.java create mode 100644 server/src/main/java/org/opensearch/tasks/TaskCancellationMonitoringService.java create mode 100644 server/src/main/java/org/opensearch/tasks/TaskCancellationMonitoringSettings.java create mode 100644 server/src/main/java/org/opensearch/tasks/TaskCancellationStats.java create mode 100644 server/src/test/java/org/opensearch/tasks/SearchShardTaskCancellationStatsTests.java create mode 100644 server/src/test/java/org/opensearch/tasks/TaskCancellationMonitoringServiceTests.java create mode 100644 server/src/test/java/org/opensearch/tasks/TaskCancellationMonitoringSettingsTests.java create mode 100644 server/src/test/java/org/opensearch/tasks/TaskCancellationStatsTests.java diff --git a/CHANGELOG.md b/CHANGELOG.md index a07c8c87a60b2..c5a0bc00fb7c3 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -91,6 +91,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), ## [Unreleased 2.x] ### Added +- Add task cancellation monitoring service ([#7642](https://github.com/opensearch-project/OpenSearch/pull/7642)) - Add TokenManager Interface ([#7452](https://github.com/opensearch-project/OpenSearch/pull/7452)) - Add Remote store as a segment replication source ([#7653](https://github.com/opensearch-project/OpenSearch/pull/7653)) diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodeStats.java b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodeStats.java index f92963af1681a..d03011774bb83 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodeStats.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodeStats.java @@ -59,6 +59,7 @@ import org.opensearch.script.ScriptCacheStats; import org.opensearch.script.ScriptStats; import org.opensearch.search.backpressure.stats.SearchBackpressureStats; +import org.opensearch.tasks.TaskCancellationStats; import org.opensearch.threadpool.ThreadPoolStats; import org.opensearch.transport.TransportStats; @@ -134,6 +135,9 @@ public class NodeStats extends BaseNodeResponse implements ToXContentFragment { @Nullable private FileCacheStats fileCacheStats; + @Nullable + private TaskCancellationStats taskCancellationStats; + public NodeStats(StreamInput in) throws IOException { super(in); timestamp = in.readVLong(); @@ -180,6 +184,11 @@ public NodeStats(StreamInput in) throws IOException { } else { fileCacheStats = null; } + if (in.getVersion().onOrAfter(Version.V_3_0_0)) { + taskCancellationStats = in.readOptionalWriteable(TaskCancellationStats::new); + } else { + taskCancellationStats = null; + } } public NodeStats( @@ -204,7 +213,8 @@ public NodeStats( @Nullable SearchBackpressureStats searchBackpressureStats, @Nullable ClusterManagerThrottlingStats clusterManagerThrottlingStats, @Nullable WeightedRoutingStats weightedRoutingStats, - @Nullable FileCacheStats fileCacheStats + @Nullable FileCacheStats fileCacheStats, + @Nullable TaskCancellationStats taskCancellationStats ) { super(node); this.timestamp = timestamp; @@ -228,6 +238,7 @@ public NodeStats( this.clusterManagerThrottlingStats = clusterManagerThrottlingStats; this.weightedRoutingStats = weightedRoutingStats; this.fileCacheStats = fileCacheStats; + this.taskCancellationStats = taskCancellationStats; } public long getTimestamp() { @@ -355,6 +366,11 @@ public FileCacheStats getFileCacheStats() { return fileCacheStats; } + @Nullable + public TaskCancellationStats getTaskCancellationStats() { + return taskCancellationStats; + } + @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); @@ -392,6 +408,9 @@ public void writeTo(StreamOutput out) throws IOException { if (out.getVersion().onOrAfter(Version.V_2_7_0)) { out.writeOptionalWriteable(fileCacheStats); } + if (out.getVersion().onOrAfter(Version.V_3_0_0)) { + out.writeOptionalWriteable(taskCancellationStats); + } } @Override @@ -476,6 +495,9 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (getFileCacheStats() != null) { getFileCacheStats().toXContent(builder, params); } + if (getTaskCancellationStats() != null) { + getTaskCancellationStats().toXContent(builder, params); + } return builder; } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodesStatsRequest.java b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodesStatsRequest.java index a9c58ac803590..68f391b91507c 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodesStatsRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodesStatsRequest.java @@ -210,7 +210,8 @@ public enum Metric { SEARCH_BACKPRESSURE("search_backpressure"), CLUSTER_MANAGER_THROTTLING("cluster_manager_throttling"), WEIGHTED_ROUTING_STATS("weighted_routing"), - FILE_CACHE_STATS("file_cache"); + FILE_CACHE_STATS("file_cache"), + TASK_CANCELLATION("task_cancellation"); private String metricName; diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/TransportNodesStatsAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/TransportNodesStatsAction.java index 02b5ceef2c7e4..6aadf546d30f7 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/TransportNodesStatsAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/TransportNodesStatsAction.java @@ -122,7 +122,8 @@ protected NodeStats nodeOperation(NodeStatsRequest nodeStatsRequest) { NodesStatsRequest.Metric.SEARCH_BACKPRESSURE.containedIn(metrics), NodesStatsRequest.Metric.CLUSTER_MANAGER_THROTTLING.containedIn(metrics), NodesStatsRequest.Metric.WEIGHTED_ROUTING_STATS.containedIn(metrics), - NodesStatsRequest.Metric.FILE_CACHE_STATS.containedIn(metrics) + NodesStatsRequest.Metric.FILE_CACHE_STATS.containedIn(metrics), + NodesStatsRequest.Metric.TASK_CANCELLATION.containedIn(metrics) ); } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/stats/TransportClusterStatsAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/stats/TransportClusterStatsAction.java index 26332f762bdf2..726f8a0de19ae 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/stats/TransportClusterStatsAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/stats/TransportClusterStatsAction.java @@ -166,6 +166,7 @@ protected ClusterStatsNodeResponse nodeOperation(ClusterStatsNodeRequest nodeReq false, false, false, + false, false ); List shardsStats = new ArrayList<>(); diff --git a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java index c12eb87ddbcb5..6dfa705b12896 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -48,6 +48,7 @@ import org.opensearch.search.backpressure.settings.SearchBackpressureSettings; import org.opensearch.search.backpressure.settings.SearchShardTaskSettings; import org.opensearch.search.backpressure.settings.SearchTaskSettings; +import org.opensearch.tasks.TaskCancellationMonitoringSettings; import org.opensearch.tasks.TaskManager; import org.opensearch.tasks.TaskResourceTrackingService; import org.opensearch.tasks.consumer.TopNSearchTasksLogger; @@ -649,7 +650,11 @@ public void apply(Settings value, Settings current, Settings previous) { RemoteRefreshSegmentPressureSettings.MIN_CONSECUTIVE_FAILURES_LIMIT, RemoteRefreshSegmentPressureSettings.UPLOAD_BYTES_MOVING_AVERAGE_WINDOW_SIZE, RemoteRefreshSegmentPressureSettings.UPLOAD_BYTES_PER_SEC_MOVING_AVERAGE_WINDOW_SIZE, - RemoteRefreshSegmentPressureSettings.UPLOAD_TIME_MOVING_AVERAGE_WINDOW_SIZE + RemoteRefreshSegmentPressureSettings.UPLOAD_TIME_MOVING_AVERAGE_WINDOW_SIZE, + + // Related to monitoring of task cancellation + TaskCancellationMonitoringSettings.IS_ENABLED_SETTING, + TaskCancellationMonitoringSettings.DURATION_MILLIS_SETTING ) ) ); diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index dd205ad87812b..eb1fc2008df06 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -59,6 +59,8 @@ import org.opensearch.search.backpressure.SearchBackpressureService; import org.opensearch.search.backpressure.settings.SearchBackpressureSettings; import org.opensearch.search.pipeline.SearchPipelineService; +import org.opensearch.tasks.TaskCancellationMonitoringService; +import org.opensearch.tasks.TaskCancellationMonitoringSettings; import org.opensearch.tasks.TaskResourceTrackingService; import org.opensearch.tasks.consumer.TopNSearchTasksLogger; import org.opensearch.threadpool.RunnableTaskExecutionListener; @@ -972,6 +974,15 @@ protected Node( client, FeatureFlags.isEnabled(SEARCH_PIPELINE) ); + final TaskCancellationMonitoringSettings taskCancellationMonitoringSettings = new TaskCancellationMonitoringSettings( + settings, + clusterService.getClusterSettings() + ); + final TaskCancellationMonitoringService taskCancellationMonitoringService = new TaskCancellationMonitoringService( + threadPool, + transportService.getTaskManager(), + taskCancellationMonitoringSettings + ); this.nodeService = new NodeService( settings, threadPool, @@ -992,7 +1003,8 @@ protected Node( searchModule.getValuesSourceRegistry().getUsageService(), searchBackpressureService, searchPipelineService, - fileCache + fileCache, + taskCancellationMonitoringService ); final SearchService searchService = newSearchService( @@ -1222,6 +1234,7 @@ public Node start() throws NodeValidationException { injector.getInstance(FsHealthService.class).start(); nodeService.getMonitorService().start(); nodeService.getSearchBackpressureService().start(); + nodeService.getTaskCancellationMonitoringService().start(); final ClusterService clusterService = injector.getInstance(ClusterService.class); @@ -1380,6 +1393,7 @@ private Node stop() { injector.getInstance(GatewayService.class).stop(); injector.getInstance(SearchService.class).stop(); injector.getInstance(TransportService.class).stop(); + nodeService.getTaskCancellationMonitoringService().stop(); pluginLifecycleComponents.forEach(LifecycleComponent::stop); // we should stop this last since it waits for resources to get released @@ -1443,6 +1457,7 @@ public synchronized void close() throws IOException { toClose.add(injector.getInstance(SearchService.class)); toClose.add(() -> stopWatch.stop().start("transport")); toClose.add(injector.getInstance(TransportService.class)); + toClose.add(nodeService.getTaskCancellationMonitoringService()); for (LifecycleComponent plugin : pluginLifecycleComponents) { toClose.add(() -> stopWatch.stop().start("plugin(" + plugin.getClass().getName() + ")")); diff --git a/server/src/main/java/org/opensearch/node/NodeService.java b/server/src/main/java/org/opensearch/node/NodeService.java index 0eab742a8da7d..9382746081c18 100644 --- a/server/src/main/java/org/opensearch/node/NodeService.java +++ b/server/src/main/java/org/opensearch/node/NodeService.java @@ -57,6 +57,7 @@ import org.opensearch.search.aggregations.support.AggregationUsageService; import org.opensearch.search.backpressure.SearchBackpressureService; import org.opensearch.search.pipeline.SearchPipelineService; +import org.opensearch.tasks.TaskCancellationMonitoringService; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportService; @@ -90,6 +91,7 @@ public class NodeService implements Closeable { private final ClusterService clusterService; private final Discovery discovery; private final FileCache fileCache; + private final TaskCancellationMonitoringService taskCancellationMonitoringService; NodeService( Settings settings, @@ -111,7 +113,8 @@ public class NodeService implements Closeable { AggregationUsageService aggregationUsageService, SearchBackpressureService searchBackpressureService, SearchPipelineService searchPipelineService, - FileCache fileCache + FileCache fileCache, + TaskCancellationMonitoringService taskCancellationMonitoringService ) { this.settings = settings; this.threadPool = threadPool; @@ -133,6 +136,7 @@ public class NodeService implements Closeable { this.searchPipelineService = searchPipelineService; this.clusterService = clusterService; this.fileCache = fileCache; + this.taskCancellationMonitoringService = taskCancellationMonitoringService; clusterService.addStateApplier(ingestService); clusterService.addStateApplier(searchPipelineService); } @@ -211,7 +215,8 @@ public NodeStats stats( boolean searchBackpressure, boolean clusterManagerThrottling, boolean weightedRoutingStats, - boolean fileCacheStats + boolean fileCacheStats, + boolean taskCancellation ) { // for indices stats we want to include previous allocated shards stats as well (it will // only be applied to the sensible ones to use, like refresh/merge/flush/indexing stats) @@ -237,7 +242,8 @@ public NodeStats stats( searchBackpressure ? this.searchBackpressureService.nodeStats() : null, clusterManagerThrottling ? this.clusterService.getClusterManagerService().getThrottlingStats() : null, weightedRoutingStats ? WeightedRoutingStats.getInstance() : null, - fileCacheStats && fileCache != null ? fileCache.fileCacheStats() : null + fileCacheStats && fileCache != null ? fileCache.fileCacheStats() : null, + taskCancellation ? this.taskCancellationMonitoringService.stats() : null ); } @@ -253,6 +259,10 @@ public SearchBackpressureService getSearchBackpressureService() { return searchBackpressureService; } + public TaskCancellationMonitoringService getTaskCancellationMonitoringService() { + return taskCancellationMonitoringService; + } + @Override public void close() throws IOException { IOUtils.close(indicesService); diff --git a/server/src/main/java/org/opensearch/tasks/SearchShardTaskCancellationStats.java b/server/src/main/java/org/opensearch/tasks/SearchShardTaskCancellationStats.java new file mode 100644 index 0000000000000..d78a4480700da --- /dev/null +++ b/server/src/main/java/org/opensearch/tasks/SearchShardTaskCancellationStats.java @@ -0,0 +1,75 @@ +/* + * 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.tasks; + +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.xcontent.ToXContentObject; +import org.opensearch.core.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.Objects; + +/** + * Holds monitoring service stats specific to search shard task. + */ +public class SearchShardTaskCancellationStats implements ToXContentObject, Writeable { + + private final long currentLongRunningCancelledTaskCount; + private final long totalLongRunningCancelledTaskCount; + + public SearchShardTaskCancellationStats(long currentTaskCount, long totalTaskCount) { + this.currentLongRunningCancelledTaskCount = currentTaskCount; + this.totalLongRunningCancelledTaskCount = totalTaskCount; + } + + public SearchShardTaskCancellationStats(StreamInput in) throws IOException { + this.currentLongRunningCancelledTaskCount = in.readVLong(); + this.totalLongRunningCancelledTaskCount = in.readVLong(); + } + + // package private for testing + protected long getCurrentLongRunningCancelledTaskCount() { + return this.currentLongRunningCancelledTaskCount; + } + + // package private for testing + protected long getTotalLongRunningCancelledTaskCount() { + return this.totalLongRunningCancelledTaskCount; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field("current_count_post_cancel", currentLongRunningCancelledTaskCount); + builder.field("total_count_post_cancel", totalLongRunningCancelledTaskCount); + return builder.endObject(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVLong(currentLongRunningCancelledTaskCount); + out.writeVLong(totalLongRunningCancelledTaskCount); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + SearchShardTaskCancellationStats that = (SearchShardTaskCancellationStats) o; + return currentLongRunningCancelledTaskCount == that.currentLongRunningCancelledTaskCount + && totalLongRunningCancelledTaskCount == that.totalLongRunningCancelledTaskCount; + } + + @Override + public int hashCode() { + return Objects.hash(currentLongRunningCancelledTaskCount, totalLongRunningCancelledTaskCount); + } +} diff --git a/server/src/main/java/org/opensearch/tasks/TaskCancellationMonitoringService.java b/server/src/main/java/org/opensearch/tasks/TaskCancellationMonitoringService.java new file mode 100644 index 0000000000000..5b512af56e195 --- /dev/null +++ b/server/src/main/java/org/opensearch/tasks/TaskCancellationMonitoringService.java @@ -0,0 +1,179 @@ +/* + * 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.tasks; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.action.search.SearchShardTask; +import org.opensearch.common.component.AbstractLifecycleComponent; +import org.opensearch.common.metrics.CounterMetric; +import org.opensearch.threadpool.Scheduler; +import org.opensearch.threadpool.ThreadPool; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +/** + * This monitoring service is responsible to track long-running(defined by a threshold) cancelled tasks as part of + * node stats. + */ +public class TaskCancellationMonitoringService extends AbstractLifecycleComponent implements TaskManager.TaskEventListeners { + + private static final Logger logger = LogManager.getLogger(TaskCancellationMonitoringService.class); + private final static List> TASKS_TO_TRACK = Arrays.asList(SearchShardTask.class); + + private volatile Scheduler.Cancellable scheduledFuture; + private final ThreadPool threadPool; + private final TaskManager taskManager; + /** + * This is to keep track of currently running cancelled tasks. This is needed to accurately calculate cumulative + * sum(from genesis) of cancelled tasks which have been running beyond a threshold and avoid double count + * problem. + * For example: + * A task M was cancelled at some point of time and continues to run for long. This Monitoring service sees this + * M for the first time and adds it as part of stats. In next iteration of monitoring service, it might see + * this M(if still running) again, but using below map we will not double count this task as part of our cumulative + * metric. + */ + private final Map cancelledTaskTracker; + /** + * This map holds statistics for each cancellable task type. + */ + private final Map, TaskCancellationStatsHolder> cancellationStatsHolder; + private final TaskCancellationMonitoringSettings taskCancellationMonitoringSettings; + + public TaskCancellationMonitoringService( + ThreadPool threadPool, + TaskManager taskManager, + TaskCancellationMonitoringSettings taskCancellationMonitoringSettings + ) { + this.threadPool = threadPool; + this.taskManager = taskManager; + this.taskCancellationMonitoringSettings = taskCancellationMonitoringSettings; + this.cancelledTaskTracker = new ConcurrentHashMap<>(); + cancellationStatsHolder = TASKS_TO_TRACK.stream() + .collect(Collectors.toConcurrentMap(task -> task, task -> new TaskCancellationStatsHolder())); + taskManager.addTaskEventListeners(this); + } + + void doRun() { + if (!taskCancellationMonitoringSettings.isEnabled() || this.cancelledTaskTracker.isEmpty()) { + return; + } + Map, List> taskCancellationListByType = getCurrentRunningTasksPostCancellation(); + taskCancellationListByType.forEach((key, value) -> { + long uniqueTasksRunningCount = value.stream().filter(task -> { + if (this.cancelledTaskTracker.containsKey(task.getId()) && !this.cancelledTaskTracker.get(task.getId())) { + // Mark it as seen by the stats logic. + this.cancelledTaskTracker.put(task.getId(), true); + return true; + } else { + return false; + } + }).count(); + cancellationStatsHolder.get(key).totalLongRunningCancelledTaskCount.inc(uniqueTasksRunningCount); + }); + } + + @Override + protected void doStart() { + scheduledFuture = threadPool.scheduleWithFixedDelay(() -> { + try { + doRun(); + } catch (Exception e) { + logger.debug("Exception occurred in Task monitoring service", e); + } + }, taskCancellationMonitoringSettings.getInterval(), ThreadPool.Names.GENERIC); + } + + @Override + protected void doStop() { + if (scheduledFuture != null) { + scheduledFuture.cancel(); + } + } + + @Override + protected void doClose() throws IOException { + + } + + // For testing + protected Map getCancelledTaskTracker() { + return this.cancelledTaskTracker; + } + + /** + * Invoked when a task is completed. This helps us to disable monitoring service when there are no cancelled tasks + * running to avoid wasteful work. + * @param task task which got completed. + */ + @Override + public void onTaskCompleted(Task task) { + if (!TASKS_TO_TRACK.contains(task.getClass())) { + return; + } + this.cancelledTaskTracker.entrySet().removeIf(entry -> entry.getKey() == task.getId()); + } + + /** + * Invoked when a task is cancelled. This is to keep track of tasks being cancelled. More importantly also helps + * us to enable this monitoring service only when needed. + * @param task task which got cancelled. + */ + @Override + public void onTaskCancelled(CancellableTask task) { + if (!TASKS_TO_TRACK.contains(task.getClass())) { + return; + } + // Add task to tracker and mark it as not seen(false) yet by the stats logic. + this.cancelledTaskTracker.putIfAbsent(task.getId(), false); + } + + public TaskCancellationStats stats() { + Map, List> currentRunningCancelledTasks = + getCurrentRunningTasksPostCancellation(); + return new TaskCancellationStats( + new SearchShardTaskCancellationStats( + Optional.of(currentRunningCancelledTasks).map(mapper -> mapper.get(SearchShardTask.class)).map(List::size).orElse(0), + cancellationStatsHolder.get(SearchShardTask.class).totalLongRunningCancelledTaskCount.count() + ) + ); + } + + private Map, List> getCurrentRunningTasksPostCancellation() { + long currentTimeInNanos = System.nanoTime(); + + return taskManager.getCancellableTasks() + .values() + .stream() + .filter(task -> TASKS_TO_TRACK.contains(task.getClass())) + .filter(CancellableTask::isCancelled) + .filter(task -> { + long runningTimeSinceCancellationSeconds = TimeUnit.NANOSECONDS.toSeconds( + currentTimeInNanos - task.getCancellationStartTimeNanos() + ); + return runningTimeSinceCancellationSeconds >= taskCancellationMonitoringSettings.getDuration().getSeconds(); + }) + .collect(Collectors.groupingBy(CancellableTask::getClass, Collectors.toList())); + } + + /** + * Holds stats related to monitoring service + */ + public static class TaskCancellationStatsHolder { + CounterMetric totalLongRunningCancelledTaskCount = new CounterMetric(); + } +} diff --git a/server/src/main/java/org/opensearch/tasks/TaskCancellationMonitoringSettings.java b/server/src/main/java/org/opensearch/tasks/TaskCancellationMonitoringSettings.java new file mode 100644 index 0000000000000..d4ec99873d584 --- /dev/null +++ b/server/src/main/java/org/opensearch/tasks/TaskCancellationMonitoringSettings.java @@ -0,0 +1,93 @@ +/* + * 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.tasks; + +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Setting; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; + +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * Settings related to task cancellation monitoring service. + */ +public class TaskCancellationMonitoringSettings { + + public static final long INTERVAL_MILLIS_SETTING_DEFAULT_VALUE = 5000; + public static final long DURATION_MILLIS_SETTING_DEFAULT_VALUE = 10000; + public static final boolean IS_ENABLED_SETTING_DEFAULT_VALUE = true; + + /** + * Defines the interval(in millis) at which task cancellation service monitors and gather stats. + */ + public static final Setting INTERVAL_MILLIS_SETTING = Setting.longSetting( + "task_cancellation.interval_millis", + INTERVAL_MILLIS_SETTING_DEFAULT_VALUE, + 1, + Setting.Property.NodeScope + ); + + /** + * Setting which defines the duration threshold(in millis) of current running cancelled tasks above which they + * are tracked as part of stats. + */ + public static final Setting DURATION_MILLIS_SETTING = Setting.longSetting( + "task_cancellation.duration_millis", + DURATION_MILLIS_SETTING_DEFAULT_VALUE, + Setting.Property.Dynamic, + Setting.Property.NodeScope + ); + + /** + * Setting to enable/disable monitoring service. + */ + public static final Setting IS_ENABLED_SETTING = Setting.boolSetting( + "task_cancellation.enabled", + IS_ENABLED_SETTING_DEFAULT_VALUE, + Setting.Property.Dynamic, + Setting.Property.NodeScope + ); + + private final TimeValue interval; + private TimeValue duration; + private final AtomicBoolean isEnabled; + private final Settings settings; + private final ClusterSettings clusterSettings; + + public TaskCancellationMonitoringSettings(Settings settings, ClusterSettings clusterSettings) { + this.settings = settings; + this.clusterSettings = clusterSettings; + this.interval = new TimeValue(INTERVAL_MILLIS_SETTING.get(settings)); + this.duration = new TimeValue(DURATION_MILLIS_SETTING.get(settings)); + this.isEnabled = new AtomicBoolean(IS_ENABLED_SETTING.get(settings)); + clusterSettings.addSettingsUpdateConsumer(IS_ENABLED_SETTING, this::setIsEnabled); + clusterSettings.addSettingsUpdateConsumer(DURATION_MILLIS_SETTING, this::setDurationMillis); + } + + public TimeValue getInterval() { + return this.interval; + } + + public TimeValue getDuration() { + return this.duration; + } + + public void setDurationMillis(long durationMillis) { + this.duration = new TimeValue(durationMillis); + } + + public boolean isEnabled() { + return isEnabled.get(); + } + + public void setIsEnabled(boolean isEnabled) { + this.isEnabled.set(isEnabled); + } +} diff --git a/server/src/main/java/org/opensearch/tasks/TaskCancellationStats.java b/server/src/main/java/org/opensearch/tasks/TaskCancellationStats.java new file mode 100644 index 0000000000000..2ccb3738b1235 --- /dev/null +++ b/server/src/main/java/org/opensearch/tasks/TaskCancellationStats.java @@ -0,0 +1,64 @@ +/* + * 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.tasks; + +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.xcontent.ToXContentFragment; +import org.opensearch.core.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.Objects; + +/** + * Holds stats related to task cancellation. + */ +public class TaskCancellationStats implements ToXContentFragment, Writeable { + + private final SearchShardTaskCancellationStats searchShardTaskCancellationStats; + + public TaskCancellationStats(SearchShardTaskCancellationStats searchShardTaskCancellationStats) { + this.searchShardTaskCancellationStats = searchShardTaskCancellationStats; + } + + public TaskCancellationStats(StreamInput in) throws IOException { + searchShardTaskCancellationStats = new SearchShardTaskCancellationStats(in); + } + + // package private for testing + protected SearchShardTaskCancellationStats getSearchShardTaskCancellationStats() { + return this.searchShardTaskCancellationStats; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject("task_cancellation"); + builder.field("search_shard_task", searchShardTaskCancellationStats); + return builder.endObject(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + searchShardTaskCancellationStats.writeTo(out); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + TaskCancellationStats that = (TaskCancellationStats) o; + return Objects.equals(searchShardTaskCancellationStats, that.searchShardTaskCancellationStats); + } + + @Override + public int hashCode() { + return Objects.hash(searchShardTaskCancellationStats); + } +} diff --git a/server/src/main/java/org/opensearch/tasks/TaskManager.java b/server/src/main/java/org/opensearch/tasks/TaskManager.java index 1d0e19e7a557b..6aeba47766842 100644 --- a/server/src/main/java/org/opensearch/tasks/TaskManager.java +++ b/server/src/main/java/org/opensearch/tasks/TaskManager.java @@ -132,6 +132,7 @@ public class TaskManager implements ClusterStateApplier { private volatile boolean taskResourceConsumersEnabled; private final Set> taskResourceConsumer; + private final List taskEventListeners = new ArrayList<>(); public static TaskManager createTaskManagerWithClusterSettings( Settings settings, @@ -152,6 +153,19 @@ public TaskManager(Settings settings, ThreadPool threadPool, Set taskHea taskResourceConsumer = new HashSet<>(); } + /** + * Listener that gets invoked during an event such as task cancellation/completion. + */ + public interface TaskEventListeners { + default void onTaskCancelled(CancellableTask task) {} + + default void onTaskCompleted(Task task) {} + } + + public void addTaskEventListeners(TaskEventListeners taskEventListeners) { + this.taskEventListeners.add(taskEventListeners); + } + public void registerTaskResourceConsumer(Consumer consumer) { taskResourceConsumer.add(consumer); } @@ -261,6 +275,17 @@ private void registerCancellableTask(Task task) { */ public void cancel(CancellableTask task, String reason, Runnable listener) { CancellableTaskHolder holder = cancellableTasks.get(task.getId()); + List exceptions = new ArrayList<>(); + for (TaskEventListeners taskEventListener : taskEventListeners) { + try { + taskEventListener.onTaskCancelled(task); + } catch (Exception e) { + exceptions.add(e); + } + } + // Throwing exception in case any of the cancellation listener results into exception. + // Should we just swallow such exceptions? + ExceptionsHelper.maybeThrowRuntimeAndSuppress(exceptions); if (holder != null) { logger.trace("cancelling task with id {}", task.getId()); holder.cancel(reason, listener); @@ -274,6 +299,15 @@ public void cancel(CancellableTask task, String reason, Runnable listener) { */ public Task unregister(Task task) { logger.trace("unregister task for id: {}", task.getId()); + List exceptions = new ArrayList<>(); + for (TaskEventListeners taskEventListener : taskEventListeners) { + try { + taskEventListener.onTaskCompleted(task); + } catch (Exception e) { + exceptions.add(e); + } + } + ExceptionsHelper.maybeThrowRuntimeAndSuppress(exceptions); // Decrement the task's self-thread as part of unregistration. task.decrementResourceTrackingThreads(); diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java b/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java index 473ab3d26a05c..d99b93b780140 100644 --- a/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java +++ b/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java @@ -749,6 +749,7 @@ public static NodeStats createNodeStats() { null, clusterManagerThrottlingStats, weightedRoutingStats, + null, null ); } diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/node/tasks/CancellableTasksTests.java b/server/src/test/java/org/opensearch/action/admin/cluster/node/tasks/CancellableTasksTests.java index ffd3a66ad1d48..e7026e9bc34cb 100644 --- a/server/src/test/java/org/opensearch/action/admin/cluster/node/tasks/CancellableTasksTests.java +++ b/server/src/test/java/org/opensearch/action/admin/cluster/node/tasks/CancellableTasksTests.java @@ -33,6 +33,7 @@ import com.carrotsearch.randomizedtesting.RandomizedContext; import com.carrotsearch.randomizedtesting.generators.RandomNumbers; +import org.opensearch.OpenSearchException; import org.opensearch.action.ActionListener; import org.opensearch.action.admin.cluster.node.tasks.cancel.CancelTasksAction; import org.opensearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest; @@ -563,6 +564,19 @@ public void testNonExistingTaskCancellation() throws Exception { public void testCancelConcurrently() throws Exception { setupTestNodes(Settings.EMPTY); final TaskManager taskManager = testNodes[0].transportService.getTaskManager(); + AtomicBoolean onTaskCancelled = new AtomicBoolean(); + AtomicBoolean onTaskCompleted = new AtomicBoolean(); + taskManager.addTaskEventListeners(new TaskManager.TaskEventListeners() { + @Override + public void onTaskCancelled(CancellableTask task) { + onTaskCancelled.set(true); + } + + @Override + public void onTaskCompleted(Task task) { + onTaskCompleted.set(true); + } + }); int numTasks = randomIntBetween(1, 10); List tasks = new ArrayList<>(numTasks); for (int i = 0; i < numTasks; i++) { @@ -577,11 +591,13 @@ public void testCancelConcurrently() throws Exception { threads[i] = new Thread(() -> { phaser.arriveAndAwaitAdvance(); taskManager.cancel(cancellingTask, "test", () -> assertTrue(notified.compareAndSet(idx, 0, 1))); + assertTrue(onTaskCancelled.get()); }); threads[i].start(); } phaser.arriveAndAwaitAdvance(); taskManager.unregister(cancellingTask); + assertTrue(onTaskCompleted.get()); for (int i = 0; i < threads.length; i++) { threads[i].join(); assertThat(notified.get(i), equalTo(1)); @@ -591,6 +607,21 @@ public void testCancelConcurrently() throws Exception { assertTrue(called.get()); } + public void testCancelWithCancellationListenerThrowingException() { + setupTestNodes(Settings.EMPTY); + final TaskManager taskManager = testNodes[0].transportService.getTaskManager(); + taskManager.addTaskEventListeners(new TaskManager.TaskEventListeners() { + @Override + public void onTaskCancelled(CancellableTask task) { + throw new OpenSearchException("Exception"); + } + }); + CancellableTask cancellableTask = (CancellableTask) taskManager.register("type-0", "action-0", new CancellableNodeRequest()); + AtomicBoolean taskCompleted = new AtomicBoolean(); + assertThrows(OpenSearchException.class, () -> taskManager.cancel(cancellableTask, "test", () -> taskCompleted.set(true))); + assertFalse(taskCompleted.get()); + } + private static void debugDelay(String name) { // Introduce an additional pseudo random repeatable race conditions String delayName = RandomizedContext.current().getRunnerSeedAsString() + ":" + name; diff --git a/server/src/test/java/org/opensearch/cluster/DiskUsageTests.java b/server/src/test/java/org/opensearch/cluster/DiskUsageTests.java index 1fad9ad5086d8..73349d45bd5c7 100644 --- a/server/src/test/java/org/opensearch/cluster/DiskUsageTests.java +++ b/server/src/test/java/org/opensearch/cluster/DiskUsageTests.java @@ -188,6 +188,7 @@ public void testFillDiskUsage() { null, null, null, + null, null ), new NodeStats( @@ -212,6 +213,7 @@ public void testFillDiskUsage() { null, null, null, + null, null ), new NodeStats( @@ -236,6 +238,7 @@ public void testFillDiskUsage() { null, null, null, + null, null ) ); @@ -291,6 +294,7 @@ public void testFillDiskUsageSomeInvalidValues() { null, null, null, + null, null ), new NodeStats( @@ -315,6 +319,7 @@ public void testFillDiskUsageSomeInvalidValues() { null, null, null, + null, null ), new NodeStats( @@ -339,6 +344,7 @@ public void testFillDiskUsageSomeInvalidValues() { null, null, null, + null, null ) ); diff --git a/server/src/test/java/org/opensearch/tasks/SearchShardTaskCancellationStatsTests.java b/server/src/test/java/org/opensearch/tasks/SearchShardTaskCancellationStatsTests.java new file mode 100644 index 0000000000000..4bab365536a49 --- /dev/null +++ b/server/src/test/java/org/opensearch/tasks/SearchShardTaskCancellationStatsTests.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.tasks; + +import org.opensearch.common.io.stream.Writeable; +import org.opensearch.test.AbstractWireSerializingTestCase; + +public class SearchShardTaskCancellationStatsTests extends AbstractWireSerializingTestCase { + @Override + protected Writeable.Reader instanceReader() { + return SearchShardTaskCancellationStats::new; + } + + @Override + protected SearchShardTaskCancellationStats createTestInstance() { + return randomInstance(); + } + + public static SearchShardTaskCancellationStats randomInstance() { + return new SearchShardTaskCancellationStats(randomNonNegativeLong(), randomNonNegativeLong()); + } +} diff --git a/server/src/test/java/org/opensearch/tasks/TaskCancellationMonitoringServiceTests.java b/server/src/test/java/org/opensearch/tasks/TaskCancellationMonitoringServiceTests.java new file mode 100644 index 0000000000000..e068e6ee6e319 --- /dev/null +++ b/server/src/test/java/org/opensearch/tasks/TaskCancellationMonitoringServiceTests.java @@ -0,0 +1,371 @@ +/* + * 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.tasks; + +import org.junit.After; +import org.junit.Before; +import org.opensearch.Version; +import org.opensearch.action.search.SearchShardTask; +import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.test.transport.MockTransportService; +import org.opensearch.threadpool.Scheduler; +import org.opensearch.threadpool.TestThreadPool; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.TransportRequest; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Phaser; +import java.util.concurrent.TimeUnit; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.opensearch.tasks.TaskCancellationMonitoringSettings.DURATION_MILLIS_SETTING; + +public class TaskCancellationMonitoringServiceTests extends OpenSearchTestCase { + + MockTransportService transportService; + TaskManager taskManager; + ThreadPool threadPool; + + @Before + public void setup() { + threadPool = new TestThreadPool(getClass().getName()); + transportService = MockTransportService.createNewService(Settings.EMPTY, Version.CURRENT, threadPool); + transportService.start(); + transportService.acceptIncomingRequests(); + taskManager = transportService.getTaskManager(); + taskManager.setTaskCancellationService(new TaskCancellationService(transportService)); + } + + @After + public void cleanup() { + transportService.close(); + ThreadPool.terminate(threadPool, 5, TimeUnit.SECONDS); + } + + public void testWithNoCurrentRunningCancelledTasks() { + TaskCancellationMonitoringSettings settings = new TaskCancellationMonitoringSettings( + Settings.EMPTY, + new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) + ); + TaskManager mockTaskManager = mock(TaskManager.class); + TaskCancellationMonitoringService taskCancellationMonitoringService = new TaskCancellationMonitoringService( + threadPool, + mockTaskManager, + settings + ); + + taskCancellationMonitoringService.doRun(); + // Task manager should not be invoked. + verify(mockTaskManager, times(0)).getTasks(); + } + + public void testWithNonZeroCancelledSearchShardTasksRunning() throws InterruptedException { + Settings settings = Settings.builder() + .put(DURATION_MILLIS_SETTING.getKey(), 0) // Setting to zero for testing + .build(); + TaskCancellationMonitoringSettings taskCancellationMonitoringSettings = new TaskCancellationMonitoringSettings( + settings, + new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) + ); + + TaskCancellationMonitoringService taskCancellationMonitoringService = new TaskCancellationMonitoringService( + threadPool, + taskManager, + taskCancellationMonitoringSettings + ); + int numTasks = randomIntBetween(5, 50); + List tasks = createTasks(numTasks); + + int cancelFromIdx = randomIntBetween(0, numTasks - 1); + int cancelTillIdx = randomIntBetween(cancelFromIdx, numTasks - 1); + + int numberOfTasksCancelled = cancelTillIdx - cancelFromIdx + 1; + CountDownLatch countDownLatch = cancelTasksConcurrently(tasks, cancelFromIdx, cancelTillIdx); + + countDownLatch.await(); // Wait for all threads execution. + taskCancellationMonitoringService.doRun(); // 1st run to verify whether we are able to track running cancelled + // tasks. + TaskCancellationStats stats = taskCancellationMonitoringService.stats(); + assertEquals(numberOfTasksCancelled, stats.getSearchShardTaskCancellationStats().getCurrentLongRunningCancelledTaskCount()); + assertEquals(numberOfTasksCancelled, stats.getSearchShardTaskCancellationStats().getTotalLongRunningCancelledTaskCount()); + + taskCancellationMonitoringService.doRun(); // 2nd run. Verify same. + stats = taskCancellationMonitoringService.stats(); + assertEquals(numberOfTasksCancelled, stats.getSearchShardTaskCancellationStats().getCurrentLongRunningCancelledTaskCount()); + assertEquals(numberOfTasksCancelled, stats.getSearchShardTaskCancellationStats().getTotalLongRunningCancelledTaskCount()); + completeTasksConcurrently(tasks, 0, tasks.size() - 1).await(); + taskCancellationMonitoringService.doRun(); // 3rd run to verify current count is 0 and total remains the same. + stats = taskCancellationMonitoringService.stats(); + assertTrue(taskCancellationMonitoringService.getCancelledTaskTracker().isEmpty()); + assertEquals(0, stats.getSearchShardTaskCancellationStats().getCurrentLongRunningCancelledTaskCount()); + assertEquals(numberOfTasksCancelled, stats.getSearchShardTaskCancellationStats().getTotalLongRunningCancelledTaskCount()); + } + + public void testShouldRunGetsDisabledAfterTaskCompletion() throws InterruptedException { + Settings settings = Settings.builder() + .put(DURATION_MILLIS_SETTING.getKey(), 0) // Setting to zero for testing + .build(); + TaskCancellationMonitoringSettings taskCancellationMonitoringSettings = new TaskCancellationMonitoringSettings( + settings, + new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) + ); + TaskCancellationMonitoringService taskCancellationMonitoringService = new TaskCancellationMonitoringService( + threadPool, + taskManager, + taskCancellationMonitoringSettings + ); + assertTrue(taskCancellationMonitoringService.getCancelledTaskTracker().isEmpty()); + assertEquals(0, taskCancellationMonitoringService.getCancelledTaskTracker().size()); + + // Start few tasks. + int numTasks = randomIntBetween(5, 50); + List tasks = createTasks(numTasks); + + taskCancellationMonitoringService.doRun(); + TaskCancellationStats stats = taskCancellationMonitoringService.stats(); + // verify no cancelled tasks currently being recorded + assertEquals(0, stats.getSearchShardTaskCancellationStats().getCurrentLongRunningCancelledTaskCount()); + assertEquals(0, stats.getSearchShardTaskCancellationStats().getTotalLongRunningCancelledTaskCount()); + cancelTasksConcurrently(tasks, 0, tasks.size() - 1).await(); + taskCancellationMonitoringService.doRun(); + stats = taskCancellationMonitoringService.stats(); + assertFalse(taskCancellationMonitoringService.getCancelledTaskTracker().isEmpty()); + assertEquals(numTasks, stats.getSearchShardTaskCancellationStats().getCurrentLongRunningCancelledTaskCount()); + assertEquals(numTasks, stats.getSearchShardTaskCancellationStats().getTotalLongRunningCancelledTaskCount()); + + completeTasksConcurrently(tasks, 0, tasks.size() - 1).await(); + stats = taskCancellationMonitoringService.stats(); + assertTrue(taskCancellationMonitoringService.getCancelledTaskTracker().isEmpty()); + assertEquals(0, stats.getSearchShardTaskCancellationStats().getCurrentLongRunningCancelledTaskCount()); + assertEquals(numTasks, stats.getSearchShardTaskCancellationStats().getTotalLongRunningCancelledTaskCount()); + } + + public void testWithVaryingCancelledTasksDuration() throws InterruptedException { + long cancelledTaskDurationThresholdMilis = 2000; + Settings settings = Settings.builder() + .put(DURATION_MILLIS_SETTING.getKey(), cancelledTaskDurationThresholdMilis) // Setting to one for testing + .build(); + TaskCancellationMonitoringSettings taskCancellationMonitoringSettings = new TaskCancellationMonitoringSettings( + settings, + new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) + ); + + TaskCancellationMonitoringService taskCancellationMonitoringService = new TaskCancellationMonitoringService( + threadPool, + taskManager, + taskCancellationMonitoringSettings + ); + + int numTasks = randomIntBetween(5, 50); + List tasks = createTasks(numTasks); + + int numTasksToBeCancelledInFirstIteration = randomIntBetween(1, numTasks - 1); + CountDownLatch countDownLatch = cancelTasksConcurrently(tasks, 0, numTasksToBeCancelledInFirstIteration - 1); + countDownLatch.await(); // Wait for all tasks to be cancelled in first iteration + + Thread.sleep(cancelledTaskDurationThresholdMilis); // Sleep, so we later verify whether above tasks are being + // captured as part of stats. + + taskCancellationMonitoringService.doRun(); + TaskCancellationStats stats = taskCancellationMonitoringService.stats(); + // Verify only tasks that were cancelled as part of first iteration is being captured as part of stats as + // they have been running longer as per threshold. + assertEquals( + numTasksToBeCancelledInFirstIteration, + stats.getSearchShardTaskCancellationStats().getCurrentLongRunningCancelledTaskCount() + ); + assertEquals( + numTasksToBeCancelledInFirstIteration, + stats.getSearchShardTaskCancellationStats().getTotalLongRunningCancelledTaskCount() + ); + + countDownLatch = cancelTasksConcurrently(tasks, numTasksToBeCancelledInFirstIteration, numTasks - 1); + countDownLatch.await(); // Wait for rest of tasks to be cancelled. + + Thread.sleep(cancelledTaskDurationThresholdMilis); // Sleep again, so we now verify whether all tasks are + // being captured as part of stats. + taskCancellationMonitoringService.doRun(); + stats = taskCancellationMonitoringService.stats(); + assertEquals(numTasks, stats.getSearchShardTaskCancellationStats().getCurrentLongRunningCancelledTaskCount()); + assertEquals(numTasks, stats.getSearchShardTaskCancellationStats().getTotalLongRunningCancelledTaskCount()); + + completeTasksConcurrently(tasks, 0, tasks.size() - 1).await(); + taskCancellationMonitoringService.doRun(); + stats = taskCancellationMonitoringService.stats(); + // Verify no current running tasks + assertEquals(0, stats.getSearchShardTaskCancellationStats().getCurrentLongRunningCancelledTaskCount()); + assertEquals(numTasks, stats.getSearchShardTaskCancellationStats().getTotalLongRunningCancelledTaskCount()); + } + + public void testTasksAreGettingEvictedCorrectlyAfterCompletion() throws InterruptedException { + Settings settings = Settings.builder() + .put(DURATION_MILLIS_SETTING.getKey(), 0) // Setting to one for testing + .build(); + TaskCancellationMonitoringSettings taskCancellationMonitoringSettings = new TaskCancellationMonitoringSettings( + settings, + new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) + ); + + TaskCancellationMonitoringService taskCancellationMonitoringService = new TaskCancellationMonitoringService( + threadPool, + taskManager, + taskCancellationMonitoringSettings + ); + + // Start few tasks. + int numTasks = randomIntBetween(5, 50); + List tasks = createTasks(numTasks); + assertTrue(taskCancellationMonitoringService.getCancelledTaskTracker().isEmpty()); + int numTasksToBeCancelledInFirstIteration = randomIntBetween(2, numTasks - 1); + CountDownLatch countDownLatch = cancelTasksConcurrently(tasks, 0, numTasksToBeCancelledInFirstIteration - 1); + countDownLatch.await(); // Wait for all tasks to be cancelled in first iteration + + assertEquals(numTasksToBeCancelledInFirstIteration, taskCancellationMonitoringService.getCancelledTaskTracker().size()); + // Verify desired task ids are present. + for (int itr = 0; itr < numTasksToBeCancelledInFirstIteration; itr++) { + assertTrue(taskCancellationMonitoringService.getCancelledTaskTracker().containsKey(tasks.get(itr).getId())); + } + // Cancel rest of the tasks + cancelTasksConcurrently(tasks, numTasksToBeCancelledInFirstIteration, numTasks - 1).await(); + for (int itr = 0; itr < tasks.size(); itr++) { + assertTrue(taskCancellationMonitoringService.getCancelledTaskTracker().containsKey(tasks.get(itr).getId())); + } + // Complete one task to start with. + completeTasksConcurrently(tasks, 0, 0).await(); + assertFalse(taskCancellationMonitoringService.getCancelledTaskTracker().containsKey(tasks.get(0).getId())); + // Verify rest of the tasks are still present in tracker + for (int itr = 1; itr < tasks.size(); itr++) { + assertTrue(taskCancellationMonitoringService.getCancelledTaskTracker().containsKey(tasks.get(itr).getId())); + } + // Complete first iteration tasks + completeTasksConcurrently(tasks, 1, numTasksToBeCancelledInFirstIteration - 1).await(); + // Verify desired tasks were evicted from tracker map + for (int itr = 0; itr < numTasksToBeCancelledInFirstIteration; itr++) { + assertFalse(taskCancellationMonitoringService.getCancelledTaskTracker().containsKey(tasks.get(0).getId())); + } + // Verify rest of the tasks are still present in tracker + for (int itr = numTasksToBeCancelledInFirstIteration; itr < tasks.size(); itr++) { + assertTrue(taskCancellationMonitoringService.getCancelledTaskTracker().containsKey(tasks.get(itr).getId())); + } + // Complete all of them finally + completeTasksConcurrently(tasks, numTasksToBeCancelledInFirstIteration, tasks.size() - 1).await(); + assertTrue(taskCancellationMonitoringService.getCancelledTaskTracker().isEmpty()); + for (int itr = 0; itr < tasks.size(); itr++) { + assertFalse(taskCancellationMonitoringService.getCancelledTaskTracker().containsKey(tasks.get(itr).getId())); + } + } + + public void testDoStartAndStop() { + TaskCancellationMonitoringSettings settings = new TaskCancellationMonitoringSettings( + Settings.EMPTY, + new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) + ); + ThreadPool mockThreadPool = mock(ThreadPool.class); + Scheduler.Cancellable scheduleFuture = mock(Scheduler.Cancellable.class); + when(scheduleFuture.cancel()).thenReturn(true); + when(mockThreadPool.scheduleWithFixedDelay(any(), any(), any())).thenReturn(scheduleFuture); + TaskCancellationMonitoringService taskCancellationMonitoringService = new TaskCancellationMonitoringService( + mockThreadPool, + taskManager, + settings + ); + + taskCancellationMonitoringService.doStart(); + taskCancellationMonitoringService.doStop(); + verify(scheduleFuture, times(1)).cancel(); + } + + private List createTasks(int numTasks) { + List tasks = new ArrayList<>(numTasks); + for (int i = 0; i < numTasks; i++) { + tasks.add((SearchShardTask) taskManager.register("type-" + i, "action-" + i, new MockQuerySearchRequest())); + } + return tasks; + } + + // Caller can this with the list of tasks specifically mentioning which ones to cancel. And can call CountDownLatch + // .await() to wait for all tasks be cancelled. + private CountDownLatch cancelTasksConcurrently(List tasks, int cancelFromIdx, int cancelTillIdx) { + assert cancelFromIdx >= 0; + assert cancelTillIdx <= tasks.size() - 1; + assert cancelTillIdx >= cancelFromIdx; + int totalTasksToBeCancelled = cancelTillIdx - cancelFromIdx + 1; + Thread[] threads = new Thread[totalTasksToBeCancelled]; + Phaser phaser = new Phaser(totalTasksToBeCancelled + 1); // Used to concurrently cancel tasks by multiple threads. + CountDownLatch countDownLatch = new CountDownLatch(totalTasksToBeCancelled); // To wait for all threads to finish. + for (int i = 0; i < totalTasksToBeCancelled; i++) { + int idx = i + cancelFromIdx; + threads[i] = new Thread(() -> { + phaser.arriveAndAwaitAdvance(); + taskManager.cancel(tasks.get(idx), "test", () -> {}); + countDownLatch.countDown(); + }); + threads[i].start(); + } + phaser.arriveAndAwaitAdvance(); + return countDownLatch; + } + + private CountDownLatch completeTasksConcurrently(List tasks, int completeFromIdx, int completeTillIdx) { + assert completeFromIdx >= 0; + assert completeTillIdx <= tasks.size() - 1; + assert completeTillIdx >= completeFromIdx; + int totalTasksToBeCompleted = completeTillIdx - completeFromIdx + 1; + Thread[] threads = new Thread[totalTasksToBeCompleted]; + Phaser phaser = new Phaser(totalTasksToBeCompleted + 1); + CountDownLatch countDownLatch = new CountDownLatch(totalTasksToBeCompleted); + for (int i = 0; i < totalTasksToBeCompleted; i++) { + int idx = i + completeFromIdx; + threads[i] = new Thread(() -> { + phaser.arriveAndAwaitAdvance(); + taskManager.unregister(tasks.get(idx)); + countDownLatch.countDown(); + }); + threads[i].start(); + } + phaser.arriveAndAwaitAdvance(); + return countDownLatch; + } + + public static class MockQuerySearchRequest extends TransportRequest { + protected String requestName; + + public MockQuerySearchRequest() { + super(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(requestName); + } + + @Override + public String getDescription() { + return "MockQuerySearchRequest[" + requestName + "]"; + } + + @Override + public Task createTask(long id, String type, String action, TaskId parentTaskId, Map headers) { + return new SearchShardTask(id, type, action, getDescription(), parentTaskId, headers); + } + } + +} diff --git a/server/src/test/java/org/opensearch/tasks/TaskCancellationMonitoringSettingsTests.java b/server/src/test/java/org/opensearch/tasks/TaskCancellationMonitoringSettingsTests.java new file mode 100644 index 0000000000000..245f410b2609a --- /dev/null +++ b/server/src/test/java/org/opensearch/tasks/TaskCancellationMonitoringSettingsTests.java @@ -0,0 +1,39 @@ +/* + * 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.tasks; + +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.test.OpenSearchTestCase; + +public class TaskCancellationMonitoringSettingsTests extends OpenSearchTestCase { + + public void testDefaults() { + ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + TaskCancellationMonitoringSettings settings = new TaskCancellationMonitoringSettings(Settings.EMPTY, clusterSettings); + assertEquals(TaskCancellationMonitoringSettings.DURATION_MILLIS_SETTING_DEFAULT_VALUE, settings.getDuration().millis()); + assertEquals(TaskCancellationMonitoringSettings.INTERVAL_MILLIS_SETTING_DEFAULT_VALUE, settings.getInterval().millis()); + assertEquals(TaskCancellationMonitoringSettings.IS_ENABLED_SETTING_DEFAULT_VALUE, settings.isEnabled()); + } + + public void testUpdate() { + ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); + TaskCancellationMonitoringSettings settings = new TaskCancellationMonitoringSettings(Settings.EMPTY, clusterSettings); + + Settings newSettings = Settings.builder() + .put(TaskCancellationMonitoringSettings.DURATION_MILLIS_SETTING.getKey(), 20000) + .put(TaskCancellationMonitoringSettings.INTERVAL_MILLIS_SETTING.getKey(), 2000) + .put(TaskCancellationMonitoringSettings.IS_ENABLED_SETTING.getKey(), false) + .build(); + clusterSettings.applySettings(newSettings); + assertEquals(20000, settings.getDuration().millis()); + assertFalse(settings.isEnabled()); + assertNotEquals(2000, settings.getInterval().millis()); + } +} diff --git a/server/src/test/java/org/opensearch/tasks/TaskCancellationStatsTests.java b/server/src/test/java/org/opensearch/tasks/TaskCancellationStatsTests.java new file mode 100644 index 0000000000000..a81110b59e98a --- /dev/null +++ b/server/src/test/java/org/opensearch/tasks/TaskCancellationStatsTests.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.tasks; + +import org.opensearch.common.io.stream.Writeable; +import org.opensearch.test.AbstractWireSerializingTestCase; + +public class TaskCancellationStatsTests extends AbstractWireSerializingTestCase { + @Override + protected Writeable.Reader instanceReader() { + return TaskCancellationStats::new; + } + + @Override + protected TaskCancellationStats createTestInstance() { + return randomInstance(); + } + + public static TaskCancellationStats randomInstance() { + return new TaskCancellationStats(SearchShardTaskCancellationStatsTests.randomInstance()); + } +} diff --git a/test/framework/src/main/java/org/opensearch/cluster/MockInternalClusterInfoService.java b/test/framework/src/main/java/org/opensearch/cluster/MockInternalClusterInfoService.java index 007e717149a62..cf5f6613c3ea1 100644 --- a/test/framework/src/main/java/org/opensearch/cluster/MockInternalClusterInfoService.java +++ b/test/framework/src/main/java/org/opensearch/cluster/MockInternalClusterInfoService.java @@ -118,7 +118,8 @@ List adjustNodesStats(List nodesStats) { nodeStats.getSearchBackpressureStats(), nodeStats.getClusterManagerThrottlingStats(), nodeStats.getWeightedRoutingStats(), - nodeStats.getFileCacheStats() + nodeStats.getFileCacheStats(), + nodeStats.getTaskCancellationStats() ); }).collect(Collectors.toList()); } 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 3faf13c373720..a3612167f16c3 100644 --- a/test/framework/src/main/java/org/opensearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/opensearch/test/InternalTestCluster.java @@ -2684,6 +2684,7 @@ public void ensureEstimatedStats() { false, false, false, + false, false ); assertThat( From 9b2b3c94728bc7f5f3771b440245a55341940a38 Mon Sep 17 00:00:00 2001 From: Nikhil-Kumar <81920482+nkumar04@users.noreply.github.com> Date: Mon, 12 Jun 2023 11:49:33 +0530 Subject: [PATCH 002/109] Align range and default value for deletes_pct_allowed in merge policy (#7730) Adjust the range and default value for deletes_pct_allowed in merge policy for more control and optimal disk usage (#7360) Signed-off-by: Nikhil Kumar Co-authored-by: Nikhil Kumar --- CHANGELOG.md | 1 + .../main/java/org/opensearch/index/MergePolicyConfig.java | 8 ++++---- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index c5a0bc00fb7c3..e3418e4112f63 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -61,6 +61,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Add ZSTD compression for snapshotting ([#2996](https://github.com/opensearch-project/OpenSearch/pull/2996)) - Change `com.amazonaws.sdk.ec2MetadataServiceEndpointOverride` to `aws.ec2MetadataServiceEndpoint` ([7372](https://github.com/opensearch-project/OpenSearch/pull/7372/)) - Change `com.amazonaws.sdk.stsEndpointOverride` to `aws.stsEndpointOverride` ([7372](https://github.com/opensearch-project/OpenSearch/pull/7372/)) +- Align range and default value for deletes_pct_allowed in merge policy ([#7730](https://github.com/opensearch-project/OpenSearch/pull/7730)) ### Deprecated diff --git a/server/src/main/java/org/opensearch/index/MergePolicyConfig.java b/server/src/main/java/org/opensearch/index/MergePolicyConfig.java index 9ff19a94bbabd..d0416aaf54a40 100644 --- a/server/src/main/java/org/opensearch/index/MergePolicyConfig.java +++ b/server/src/main/java/org/opensearch/index/MergePolicyConfig.java @@ -94,8 +94,8 @@ * * Controls the maximum percentage of deleted documents that is tolerated in * the index. Lower values make the index more space efficient at the - * expense of increased CPU and I/O activity. Values must be between 20 and - * 50. Default value is 33. + * expense of increased CPU and I/O activity. Values must be between 5 and + * 50. Default value is 20. * * *

@@ -136,7 +136,7 @@ public final class MergePolicyConfig { public static final ByteSizeValue DEFAULT_MAX_MERGED_SEGMENT = new ByteSizeValue(5, ByteSizeUnit.GB); public static final double DEFAULT_SEGMENTS_PER_TIER = 10.0d; public static final double DEFAULT_RECLAIM_DELETES_WEIGHT = 2.0d; - public static final double DEFAULT_DELETES_PCT_ALLOWED = 33.0d; + public static final double DEFAULT_DELETES_PCT_ALLOWED = 20.0d; public static final Setting INDEX_COMPOUND_FORMAT_SETTING = new Setting<>( "index.compound_format", Double.toString(TieredMergePolicy.DEFAULT_NO_CFS_RATIO), @@ -189,7 +189,7 @@ public final class MergePolicyConfig { public static final Setting INDEX_MERGE_POLICY_DELETES_PCT_ALLOWED_SETTING = Setting.doubleSetting( "index.merge.policy.deletes_pct_allowed", DEFAULT_DELETES_PCT_ALLOWED, - 20.0d, + 5.0d, 50.0d, Property.Dynamic, Property.IndexScope From 5c3225692dcea1eddbb3e76ae19f47de5ea23a96 Mon Sep 17 00:00:00 2001 From: Chaitanya Gohel <104654647+gashutos@users.noreply.github.com> Date: Mon, 12 Jun 2023 22:29:32 +0530 Subject: [PATCH 003/109] Adding back [Time series based workload desc order optimization through reverse segment read (#7244)] with fixes (#7967) * Revert "Revert "Time series based workload desc order optimization through reverse segment read (#7244)" (#7892)" This reverts commit bb265369d67431a7aa15efca8734326857db7e32. Signed-off-by: gashutos * Enable time series optimization only if it is not IndexSorted index, also ASC order reverse should only consider in @timestamp field Signed-off-by: gashutos * Modifying CHANGELOG Signed-off-by: gashutos * Adding integ test for scroll API where sort by _doc is getting early termination Signed-off-by: gashutos --------- Signed-off-by: gashutos --- CHANGELOG.md | 2 + .../test/scroll/10_basic_timeseries.yml | 161 ++++++++++++++++++ .../cluster/metadata/DataStream.java | 23 +++ .../org/opensearch/index/IndexSettings.java | 12 ++ .../opensearch/index/engine/EngineConfig.java | 19 +++ .../index/engine/EngineConfigFactory.java | 6 +- .../index/engine/InternalEngine.java | 3 + .../index/mapper/MappingLookup.java | 10 ++ .../opensearch/index/shard/IndexShard.java | 17 +- .../search/internal/ContextIndexSearcher.java | 34 +++- .../engine/EngineConfigFactoryTests.java | 6 +- .../test/OpenSearchIntegTestCase.java | 1 + 12 files changed, 288 insertions(+), 6 deletions(-) create mode 100644 rest-api-spec/src/main/resources/rest-api-spec/test/scroll/10_basic_timeseries.yml diff --git a/CHANGELOG.md b/CHANGELOG.md index e3418e4112f63..379966b44e9a5 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -95,6 +95,8 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Add task cancellation monitoring service ([#7642](https://github.com/opensearch-project/OpenSearch/pull/7642)) - Add TokenManager Interface ([#7452](https://github.com/opensearch-project/OpenSearch/pull/7452)) - Add Remote store as a segment replication source ([#7653](https://github.com/opensearch-project/OpenSearch/pull/7653)) +- Add descending order search optimization through reverse segment read. ([#7967](https://github.com/opensearch-project/OpenSearch/pull/7967)) + ### Dependencies - Bump `jackson` from 2.15.1 to 2.15.2 ([#7897](https://github.com/opensearch-project/OpenSearch/pull/7897)) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/scroll/10_basic_timeseries.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/scroll/10_basic_timeseries.yml new file mode 100644 index 0000000000000..1995bee89a0a4 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/scroll/10_basic_timeseries.yml @@ -0,0 +1,161 @@ +--- +"Basic scroll on time series workload for reversed leaf sorter": + - do: + indices.create: + index: test_scroll_time_series + body: + mappings: + properties: + name: + type: keyword + '@timestamp': + type: date + + - do: + bulk: + refresh: true + index: test_scroll_time_series + body: + - '{"index": {}}' + - '{"name": "1", "@timestamp": "2010-03-12T01:07:00"}' + - '{"index": {}}' + - '{"name": "2", "@timestamp": "2010-03-12T01:07:01"}' + - '{"index": {}}' + - '{"name": "3", "@timestamp": "2010-03-12T01:07:02"}' + - '{"index": {}}' + - '{"name": "4", "@timestamp": "2010-03-12T01:07:03"}' + - '{"index": {}}' + - '{"name": "5", "@timestamp": "2010-03-12T01:07:04"}' + - '{"index": {}}' + - '{"name": "6", "@timestamp": "2010-03-12T01:07:05"}' + - '{"index": {}}' + - '{"name": "7", "@timestamp": "2010-03-12T01:07:06"}' + - '{"index": {}}' + - '{"name": "8", "@timestamp": "2010-03-12T01:07:07"}' + - '{"index": {}}' + - '{"name": "9", "@timestamp": "2010-03-12T01:07:08"}' + - '{"index": {}}' + - '{"name": "10", "@timestamp": "2010-03-12T01:07:09"}' + - do: + indices.refresh: {} + - do: + bulk: + refresh: true + index: test_scroll_time_series + body: + - '{"index": {}}' + - '{"name": "11", "@timestamp": "2010-03-12T01:07:10"}' + - '{"index": {}}' + - '{"name": "12", "@timestamp": "2010-03-12T01:07:11"}' + - '{"index": {}}' + - '{"name": "13", "@timestamp": "2010-03-12T01:07:12"}' + - '{"index": {}}' + - '{"name": "14", "@timestamp": "2010-03-12T01:07:13"}' + - '{"index": {}}' + - '{"name": "15", "@timestamp": "2010-03-12T01:07:14"}' + - '{"index": {}}' + - '{"name": "16", "@timestamp": "2010-03-12T01:07:15"}' + - '{"index": {}}' + - '{"name": "17", "@timestamp": "2010-03-12T01:07:16"}' + - '{"index": {}}' + - '{"name": "18", "@timestamp": "2010-03-12T01:07:17"}' + - '{"index": {}}' + - '{"name": "19", "@timestamp": "2010-03-12T01:07:18"}' + - '{"index": {}}' + - '{"name": "20", "@timestamp": "2010-03-12T01:07:19"}' + - do: + indices.refresh: { } + - do: + bulk: + refresh: true + index: test_scroll_time_series + body: + - '{"index": {}}' + - '{"name": "21", "@timestamp": "2010-03-12T01:07:20"}' + - '{"index": {}}' + - '{"name": "22", "@timestamp": "2010-03-12T01:07:21"}' + - '{"index": {}}' + - '{"name": "23", "@timestamp": "2010-03-12T01:07:22"}' + - '{"index": {}}' + - '{"name": "24", "@timestamp": "2010-03-12T01:07:23"}' + - '{"index": {}}' + - '{"name": "25", "@timestamp": "2010-03-12T01:07:24"}' + - '{"index": {}}' + - '{"name": "26", "@timestamp": "2010-03-12T01:07:25"}' + - '{"index": {}}' + - '{"name": "27", "@timestamp": "2010-03-12T01:07:26"}' + - '{"index": {}}' + - '{"name": "28", "@timestamp": "2010-03-12T01:07:27"}' + - '{"index": {}}' + - '{"name": "29", "@timestamp": "2010-03-12T01:07:28"}' + - '{"index": {}}' + - '{"name": "30", "@timestamp": "2010-03-12T01:07:29"}' + - do: + indices.refresh: { } + + - do: + search: + rest_total_hits_as_int: true + index: test_scroll_time_series + size: 5 + scroll: 1m + sort: _doc + body: + query: + match_all: {} + + - set: {_scroll_id: scroll_id} + - match: {hits.total: 30 } + - length: {hits.hits: 5 } + + - do: + scroll: + rest_total_hits_as_int: true + body: { "scroll_id": "$scroll_id", "scroll": "1m"} + + - match: {hits.total: 30 } + - length: {hits.hits: 5 } + + - do: + scroll: + rest_total_hits_as_int: true + body: { "scroll_id": "$scroll_id", "scroll": "1m" } + + - match: { hits.total: 30 } + - length: { hits.hits: 5 } + + - do: + scroll: + rest_total_hits_as_int: true + body: { "scroll_id": "$scroll_id", "scroll": "1m" } + + - match: { hits.total: 30 } + - length: { hits.hits: 5 } + + - do: + scroll: + rest_total_hits_as_int: true + body: { "scroll_id": "$scroll_id", "scroll": "1m" } + + - match: { hits.total: 30 } + - length: { hits.hits: 5 } + + - do: + scroll: + rest_total_hits_as_int: true + body: { "scroll_id": "$scroll_id", "scroll": "1m" } + + - match: { hits.total: 30 } + - length: { hits.hits: 5 } + + - do: + scroll: + rest_total_hits_as_int: true + body: { "scroll_id": "$scroll_id", "scroll": "1m" } + + - match: { hits.total: 30 } + - length: { hits.hits: 0 } + + - do: + clear_scroll: + scroll_id: $scroll_id diff --git a/server/src/main/java/org/opensearch/cluster/metadata/DataStream.java b/server/src/main/java/org/opensearch/cluster/metadata/DataStream.java index 825aaee1ad1f8..f4be1cfff489c 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/DataStream.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/DataStream.java @@ -31,6 +31,10 @@ package org.opensearch.cluster.metadata; +import org.apache.lucene.document.LongPoint; +import org.apache.lucene.index.LeafReader; +import org.apache.lucene.index.PointValues; +import org.opensearch.OpenSearchException; import org.opensearch.cluster.AbstractDiffable; import org.opensearch.cluster.Diff; import org.opensearch.core.ParseField; @@ -46,6 +50,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collections; +import java.util.Comparator; import java.util.List; import java.util.Locale; import java.util.Map; @@ -59,6 +64,24 @@ public final class DataStream extends AbstractDiffable implements ToXContentObject { public static final String BACKING_INDEX_PREFIX = ".ds-"; + public static final String TIMESERIES_FIELDNAME = "@timestamp"; + public static final Comparator TIMESERIES_LEAF_SORTER = Comparator.comparingLong((LeafReader r) -> { + try { + PointValues points = r.getPointValues(TIMESERIES_FIELDNAME); + if (points != null) { + // could be a multipoint (probably not) but get the maximum time value anyway + byte[] sortValue = points.getMaxPackedValue(); + // decode the first dimension because this should not be a multi dimension field + // it's a bug in the date field if it is + return LongPoint.decodeDimension(sortValue, 0); + } else { + // segment does not have a timestamp field, just return the minimum value + return Long.MIN_VALUE; + } + } catch (IOException e) { + throw new OpenSearchException("Not a timeseries Index! Field [{}] not found!", TIMESERIES_FIELDNAME); + } + }).reversed(); private final String name; private final TimestampField timeStampField; diff --git a/server/src/main/java/org/opensearch/index/IndexSettings.java b/server/src/main/java/org/opensearch/index/IndexSettings.java index e125facb76059..de7dc102939ce 100644 --- a/server/src/main/java/org/opensearch/index/IndexSettings.java +++ b/server/src/main/java/org/opensearch/index/IndexSettings.java @@ -665,6 +665,7 @@ private void setRetentionLeaseMillis(final TimeValue retentionLease) { private volatile long mappingTotalFieldsLimit; private volatile long mappingDepthLimit; private volatile long mappingFieldNameLengthLimit; + private volatile boolean searchSegmentOrderReversed; /** * The maximum number of refresh listeners allows on this shard. @@ -905,6 +906,10 @@ public IndexSettings(final IndexMetadata indexMetadata, final Settings nodeSetti scopedSettings.addSettingsUpdateConsumer(DEFAULT_SEARCH_PIPELINE, this::setDefaultSearchPipeline); } + private void setSearchSegmentOrderReversed(boolean reversed) { + this.searchSegmentOrderReversed = reversed; + } + private void setSearchIdleAfter(TimeValue searchIdleAfter) { this.searchIdleAfter = searchIdleAfter; } @@ -1084,6 +1089,13 @@ public Settings getNodeSettings() { return nodeSettings; } + /** + * Returns true if index level setting for leaf reverse order search optimization is enabled + */ + public boolean getSearchSegmentOrderReversed() { + return this.searchSegmentOrderReversed; + } + /** * Updates the settings and index metadata and notifies all registered settings consumers with the new settings iff at least one * setting has changed. diff --git a/server/src/main/java/org/opensearch/index/engine/EngineConfig.java b/server/src/main/java/org/opensearch/index/engine/EngineConfig.java index fe003405fd3f8..338a541af387a 100644 --- a/server/src/main/java/org/opensearch/index/engine/EngineConfig.java +++ b/server/src/main/java/org/opensearch/index/engine/EngineConfig.java @@ -33,6 +33,7 @@ import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.codecs.Codec; +import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.MergePolicy; import org.apache.lucene.search.QueryCache; import org.apache.lucene.search.QueryCachingPolicy; @@ -59,6 +60,7 @@ import org.opensearch.indices.breaker.CircuitBreakerService; import org.opensearch.threadpool.ThreadPool; +import java.util.Comparator; import java.util.List; import java.util.Objects; import java.util.function.BooleanSupplier; @@ -102,6 +104,7 @@ public final class EngineConfig { private final Supplier retentionLeasesSupplier; private final boolean isReadOnlyReplica; private final BooleanSupplier primaryModeSupplier; + private final Comparator leafSorter; /** * A supplier of the outstanding retention leases. This is used during merged operations to determine which operations that have been @@ -204,6 +207,7 @@ private EngineConfig(Builder builder) { this.isReadOnlyReplica = builder.isReadOnlyReplica; this.primaryModeSupplier = builder.primaryModeSupplier; this.translogFactory = builder.translogFactory; + this.leafSorter = builder.leafSorter; } /** @@ -451,6 +455,15 @@ public TranslogDeletionPolicyFactory getCustomTranslogDeletionPolicyFactory() { return translogDeletionPolicyFactory; } + /** + * Returns subReaderSorter for org.apache.lucene.index.BaseCompositeReader. + * This gets used in lucene IndexReader and decides order of segment read. + * @return comparator + */ + public Comparator getLeafSorter() { + return this.leafSorter; + } + /** * Builder for EngineConfig class * @@ -483,6 +496,7 @@ public static class Builder { private boolean isReadOnlyReplica; private BooleanSupplier primaryModeSupplier; private TranslogFactory translogFactory = new InternalTranslogFactory(); + Comparator leafSorter; public Builder shardId(ShardId shardId) { this.shardId = shardId; @@ -614,6 +628,11 @@ public Builder translogFactory(TranslogFactory translogFactory) { return this; } + public Builder leafSorter(Comparator leafSorter) { + this.leafSorter = leafSorter; + return this; + } + public EngineConfig build() { return new EngineConfig(this); } diff --git a/server/src/main/java/org/opensearch/index/engine/EngineConfigFactory.java b/server/src/main/java/org/opensearch/index/engine/EngineConfigFactory.java index f5a5d50e11220..76b13ee244a2c 100644 --- a/server/src/main/java/org/opensearch/index/engine/EngineConfigFactory.java +++ b/server/src/main/java/org/opensearch/index/engine/EngineConfigFactory.java @@ -10,6 +10,7 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.analysis.Analyzer; +import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.MergePolicy; import org.apache.lucene.search.QueryCache; import org.apache.lucene.search.QueryCachingPolicy; @@ -36,6 +37,7 @@ import java.util.Collection; import java.util.Collections; +import java.util.Comparator; import java.util.List; import java.util.Optional; import java.util.function.BooleanSupplier; @@ -151,7 +153,8 @@ public EngineConfig newEngineConfig( EngineConfig.TombstoneDocSupplier tombstoneDocSupplier, boolean isReadOnlyReplica, BooleanSupplier primaryModeSupplier, - TranslogFactory translogFactory + TranslogFactory translogFactory, + Comparator leafSorter ) { CodecService codecServiceToUse = codecService; if (codecService == null && this.codecServiceFactory != null) { @@ -184,6 +187,7 @@ public EngineConfig newEngineConfig( .readOnlyReplica(isReadOnlyReplica) .primaryModeSupplier(primaryModeSupplier) .translogFactory(translogFactory) + .leafSorter(leafSorter) .build(); } 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 63afc6585a99d..b96bade177be2 100644 --- a/server/src/main/java/org/opensearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/opensearch/index/engine/InternalEngine.java @@ -2322,6 +2322,9 @@ private IndexWriterConfig getIndexWriterConfig() { if (config().getIndexSort() != null) { iwc.setIndexSort(config().getIndexSort()); } + if (config().getLeafSorter() != null) { + iwc.setLeafSorter(config().getLeafSorter()); // The default segment search order + } return iwc; } diff --git a/server/src/main/java/org/opensearch/index/mapper/MappingLookup.java b/server/src/main/java/org/opensearch/index/mapper/MappingLookup.java index 5bccb4f6e827e..024f4b71584bf 100644 --- a/server/src/main/java/org/opensearch/index/mapper/MappingLookup.java +++ b/server/src/main/java/org/opensearch/index/mapper/MappingLookup.java @@ -33,6 +33,7 @@ package org.opensearch.index.mapper; import org.apache.lucene.analysis.Analyzer; +import org.opensearch.cluster.metadata.DataStream; import org.opensearch.index.IndexSettings; import org.opensearch.index.analysis.FieldNameAnalyzer; @@ -261,6 +262,15 @@ public String getNestedScope(String path) { return null; } + /** + * If this index contains @timestamp field with Date type, it will return true + * @return true or false based on above condition + */ + public boolean containsTimeStampField() { + MappedFieldType timeSeriesFieldType = this.fieldTypeLookup.get(DataStream.TIMESERIES_FIELDNAME); + return timeSeriesFieldType != null && timeSeriesFieldType instanceof DateFieldMapper.DateFieldType; // has to be Date field type + } + private static String parentObject(String field) { int lastDot = field.lastIndexOf('.'); if (lastDot == -1) { diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index fc236d7d97a04..ed38c561c7e29 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -58,6 +58,7 @@ import org.apache.lucene.store.IndexInput; import org.apache.lucene.util.ThreadInterruptedException; import org.opensearch.common.lucene.store.ByteArrayIndexInput; +import org.opensearch.cluster.metadata.DataStream; import org.opensearch.core.Assertions; import org.opensearch.ExceptionsHelper; import org.opensearch.OpenSearchException; @@ -333,6 +334,7 @@ Runnable getGlobalCheckpointSyncer() { private volatile boolean useRetentionLeasesInPeerRecovery; private final Store remoteStore; private final BiFunction translogFactorySupplier; + private final boolean isTimeSeriesIndex; private final RemoteRefreshSegmentPressureService remoteRefreshSegmentPressureService; public IndexShard( @@ -451,6 +453,9 @@ public boolean shouldCache(Query query) { this.checkpointPublisher = checkpointPublisher; this.remoteStore = remoteStore; this.translogFactorySupplier = translogFactorySupplier; + this.isTimeSeriesIndex = (mapperService == null || mapperService.documentMapper() == null) + ? false + : mapperService.documentMapper().mappers().containsTimeStampField(); this.remoteRefreshSegmentPressureService = remoteRefreshSegmentPressureService; } @@ -3627,7 +3632,9 @@ private EngineConfig newEngineConfig(LongSupplier globalCheckpointSupplier) thro tombstoneDocSupplier(), isReadOnlyReplica, replicationTracker::isPrimaryMode, - translogFactorySupplier.apply(indexSettings, shardRouting) + translogFactorySupplier.apply(indexSettings, shardRouting), + isTimeSeriesDescSortOptimizationEnabled() ? DataStream.TIMESERIES_LEAF_SORTER : null // DESC @timestamp default order for + // timeseries ); } @@ -3639,6 +3646,14 @@ public boolean isRemoteTranslogEnabled() { return indexSettings() != null && indexSettings().isRemoteTranslogStoreEnabled(); } + /** + * @return true if segment reverse search optimization is enabled for time series based workload. + */ + public boolean isTimeSeriesDescSortOptimizationEnabled() { + // Do not change segment order in case of index sort. + return isTimeSeriesIndex && getIndexSort() == null; + } + /** * @return True if settings indicate this shard is backed by a remote snapshot, false otherwise. */ diff --git a/server/src/main/java/org/opensearch/search/internal/ContextIndexSearcher.java b/server/src/main/java/org/opensearch/search/internal/ContextIndexSearcher.java index cf21712fc912a..f02d28c2c375b 100644 --- a/server/src/main/java/org/opensearch/search/internal/ContextIndexSearcher.java +++ b/server/src/main/java/org/opensearch/search/internal/ContextIndexSearcher.java @@ -62,6 +62,7 @@ import org.apache.lucene.util.Bits; import org.apache.lucene.util.CombinedBitSet; import org.apache.lucene.util.SparseFixedBitSet; +import org.opensearch.cluster.metadata.DataStream; import org.opensearch.common.lucene.search.TopDocsAndMaxScore; import org.opensearch.core.common.lease.Releasable; import org.opensearch.search.DocValueFormat; @@ -282,8 +283,17 @@ public void search( @Override protected void search(List leaves, Weight weight, Collector collector) throws IOException { - for (LeafReaderContext ctx : leaves) { // search each subreader - searchLeaf(ctx, weight, collector); + if (shouldReverseLeafReaderContexts()) { + // reverse the segment search order if this flag is true. + // Certain queries can benefit if we reverse the segment read order, + // for example time series based queries if searched for desc sort order. + for (int i = leaves.size() - 1; i >= 0; i--) { + searchLeaf(leaves.get(i), weight, collector); + } + } else { + for (int i = 0; i < leaves.size(); i++) { + searchLeaf(leaves.get(i), weight, collector); + } } } @@ -496,4 +506,24 @@ private boolean canMatchSearchAfter(LeafReaderContext ctx) throws IOException { } return true; } + + private boolean shouldReverseLeafReaderContexts() { + // Time series based workload by default traverses segments in desc order i.e. latest to the oldest order. + // This is actually beneficial for search queries to start search on latest segments first for time series workload. + // That can slow down ASC order queries on timestamp workload. So to avoid that slowdown, we will reverse leaf + // reader order here. + if (searchContext != null && searchContext.indexShard().isTimeSeriesDescSortOptimizationEnabled()) { + // Only reverse order for asc order sort queries + if (searchContext.sort() != null + && searchContext.sort().sort != null + && searchContext.sort().sort.getSort() != null + && searchContext.sort().sort.getSort().length > 0 + && searchContext.sort().sort.getSort()[0].getReverse() == false + && searchContext.sort().sort.getSort()[0].getField() != null + && searchContext.sort().sort.getSort()[0].getField().equals(DataStream.TIMESERIES_FIELDNAME)) { + return true; + } + } + return false; + } } diff --git a/server/src/test/java/org/opensearch/index/engine/EngineConfigFactoryTests.java b/server/src/test/java/org/opensearch/index/engine/EngineConfigFactoryTests.java index 2db3cd24da80d..f8bedc76ea994 100644 --- a/server/src/test/java/org/opensearch/index/engine/EngineConfigFactoryTests.java +++ b/server/src/test/java/org/opensearch/index/engine/EngineConfigFactoryTests.java @@ -69,7 +69,8 @@ public void testCreateEngineConfigFromFactory() { null, false, () -> Boolean.TRUE, - new InternalTranslogFactory() + new InternalTranslogFactory(), + null ); assertNotNull(config.getCodec()); @@ -148,7 +149,8 @@ public void testCreateCodecServiceFromFactory() { null, false, () -> Boolean.TRUE, - new InternalTranslogFactory() + new InternalTranslogFactory(), + null ); assertNotNull(config.getCodec()); } 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 06f74a55be13d..4474aae1f0631 100644 --- a/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java +++ b/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java @@ -761,6 +761,7 @@ public Settings indexSettings() { ).getStringRep() ); } + return builder.build(); } From 21fa9f8fa56af450708d1f68df21733f2a2632fe Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 12 Jun 2023 11:59:34 -0700 Subject: [PATCH 004/109] Bump com.google.guava:guava from 32.0.0-jre to 32.0.1-jre in /distribution/tools/keystore-cli (#8009) * Bump com.google.guava:guava in /distribution/tools/keystore-cli Bumps [com.google.guava:guava](https://github.com/google/guava) from 32.0.0-jre to 32.0.1-jre. - [Release notes](https://github.com/google/guava/releases) - [Commits](https://github.com/google/guava/commits) --- updated-dependencies: - dependency-name: com.google.guava:guava dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] * Update changelog Signed-off-by: dependabot[bot] --------- Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: dependabot[bot] --- CHANGELOG.md | 3 ++- distribution/tools/keystore-cli/build.gradle | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 379966b44e9a5..a17b7202c3e70 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -102,6 +102,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `jackson` from 2.15.1 to 2.15.2 ([#7897](https://github.com/opensearch-project/OpenSearch/pull/7897)) - Bump `netty` from 4.1.91.Final to 4.1.93.Final ([#7901](https://github.com/opensearch-project/OpenSearch/pull/7901)) - Bump `com.netflix.nebula:gradle-extra-configurations-plugin` from 9.0.0 to 10.0.0 in /buildSrc ([#7068](https://github.com/opensearch-project/OpenSearch/pull/7068)) +- Bump `com.google.guava:guava` from 32.0.0-jre to 32.0.1-jre (#8009) ### Changed - Replace jboss-annotations-api_1.2_spec with jakarta.annotation-api ([#7836](https://github.com/opensearch-project/OpenSearch/pull/7836)) @@ -120,4 +121,4 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), ### Security [Unreleased 3.0]: https://github.com/opensearch-project/OpenSearch/compare/2.x...HEAD -[Unreleased 2.x]: https://github.com/opensearch-project/OpenSearch/compare/2.8...2.x +[Unreleased 2.x]: https://github.com/opensearch-project/OpenSearch/compare/2.8...2.x \ No newline at end of file diff --git a/distribution/tools/keystore-cli/build.gradle b/distribution/tools/keystore-cli/build.gradle index 1d9b3ce3dd8bc..3005aef5b6e67 100644 --- a/distribution/tools/keystore-cli/build.gradle +++ b/distribution/tools/keystore-cli/build.gradle @@ -35,5 +35,5 @@ dependencies { compileOnly project(":libs:opensearch-cli") testImplementation project(":test:framework") testImplementation 'com.google.jimfs:jimfs:1.2' - testRuntimeOnly 'com.google.guava:guava:32.0.0-jre' + testRuntimeOnly 'com.google.guava:guava:32.0.1-jre' } From 16f5371aee57fc64b3534d6802756890a4f2ce12 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 12 Jun 2023 15:58:00 -0400 Subject: [PATCH 005/109] Bump commons-io:commons-io from 2.12.0 to 2.13.0 in /plugins/repository-hdfs (#8014) * Bump commons-io:commons-io in /plugins/repository-hdfs Bumps commons-io:commons-io from 2.12.0 to 2.13.0. --- updated-dependencies: - dependency-name: commons-io:commons-io dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] * Updating SHAs Signed-off-by: dependabot[bot] * Update changelog Signed-off-by: dependabot[bot] --------- Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: dependabot[bot] --- CHANGELOG.md | 1 + plugins/repository-hdfs/build.gradle | 2 +- plugins/repository-hdfs/licenses/commons-io-2.12.0.jar.sha1 | 1 - plugins/repository-hdfs/licenses/commons-io-2.13.0.jar.sha1 | 1 + 4 files changed, 3 insertions(+), 2 deletions(-) delete mode 100644 plugins/repository-hdfs/licenses/commons-io-2.12.0.jar.sha1 create mode 100644 plugins/repository-hdfs/licenses/commons-io-2.13.0.jar.sha1 diff --git a/CHANGELOG.md b/CHANGELOG.md index a17b7202c3e70..d567aeaa3194e 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -103,6 +103,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `netty` from 4.1.91.Final to 4.1.93.Final ([#7901](https://github.com/opensearch-project/OpenSearch/pull/7901)) - Bump `com.netflix.nebula:gradle-extra-configurations-plugin` from 9.0.0 to 10.0.0 in /buildSrc ([#7068](https://github.com/opensearch-project/OpenSearch/pull/7068)) - Bump `com.google.guava:guava` from 32.0.0-jre to 32.0.1-jre (#8009) +- Bump `commons-io:commons-io` from 2.12.0 to 2.13.0 (#8014) ### Changed - Replace jboss-annotations-api_1.2_spec with jakarta.annotation-api ([#7836](https://github.com/opensearch-project/OpenSearch/pull/7836)) diff --git a/plugins/repository-hdfs/build.gradle b/plugins/repository-hdfs/build.gradle index 44ab5f75bfed2..7edc4def0b199 100644 --- a/plugins/repository-hdfs/build.gradle +++ b/plugins/repository-hdfs/build.gradle @@ -75,7 +75,7 @@ dependencies { api 'commons-collections:commons-collections:3.2.2' api 'org.apache.commons:commons-compress:1.23.0' api 'org.apache.commons:commons-configuration2:2.9.0' - api 'commons-io:commons-io:2.12.0' + api 'commons-io:commons-io:2.13.0' api 'org.apache.commons:commons-lang3:3.12.0' implementation 'com.google.re2j:re2j:1.7' api 'javax.servlet:servlet-api:2.5' diff --git a/plugins/repository-hdfs/licenses/commons-io-2.12.0.jar.sha1 b/plugins/repository-hdfs/licenses/commons-io-2.12.0.jar.sha1 deleted file mode 100644 index 5225b130fb817..0000000000000 --- a/plugins/repository-hdfs/licenses/commons-io-2.12.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -e5e3eb2ff05b494287f51476bc715161412c525f \ No newline at end of file diff --git a/plugins/repository-hdfs/licenses/commons-io-2.13.0.jar.sha1 b/plugins/repository-hdfs/licenses/commons-io-2.13.0.jar.sha1 new file mode 100644 index 0000000000000..c165136eb5822 --- /dev/null +++ b/plugins/repository-hdfs/licenses/commons-io-2.13.0.jar.sha1 @@ -0,0 +1 @@ +8bb2bc9b4df17e2411533a0708a69f983bf5e83b \ No newline at end of file From 068404ed1cef5e1d023816d57054aef98a7aa343 Mon Sep 17 00:00:00 2001 From: Jay Deng Date: Mon, 12 Jun 2023 13:00:51 -0700 Subject: [PATCH 006/109] Add dynamic index and cluster setting for concurrent segment search (#7956) * Add dynamic index and cluster setting for concurrent segment search Signed-off-by: Jay Deng * Use feature flagged settings map Signed-off-by: Jay Deng --------- Signed-off-by: Jay Deng --- CHANGELOG.md | 1 + distribution/src/config/opensearch.yml | 7 ++ .../common/settings/ClusterSettings.java | 4 +- .../common/settings/IndexScopedSettings.java | 4 +- .../org/opensearch/index/IndexSettings.java | 15 ++- .../search/DefaultSearchContext.java | 22 ++++ .../org/opensearch/search/SearchModule.java | 15 +-- .../org/opensearch/search/SearchService.java | 7 ++ .../search/internal/SearchContext.java | 7 ++ .../query/ConcurrentQueryPhaseSearcher.java | 14 +-- .../query/QueryPhaseSearcherWrapper.java | 82 ++++++++++++++ .../common/settings/SettingsModuleTests.java | 53 +++++++++ .../opensearch/search/SearchModuleTests.java | 6 +- .../opensearch/search/SearchServiceTests.java | 106 +++++++++++++++++- .../aggregations/AggregationSetupTests.java | 2 + .../search/query/QueryPhaseTests.java | 3 + .../opensearch/test/TestSearchContext.java | 16 +++ 17 files changed, 334 insertions(+), 30 deletions(-) create mode 100644 server/src/main/java/org/opensearch/search/query/QueryPhaseSearcherWrapper.java diff --git a/CHANGELOG.md b/CHANGELOG.md index d567aeaa3194e..d4cf64a600366 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -11,6 +11,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Add events correlation engine plugin ([#6854](https://github.com/opensearch-project/OpenSearch/issues/6854)) - Add support for ignoring missing Javadoc on generated code using annotation ([#7604](https://github.com/opensearch-project/OpenSearch/pull/7604)) - Implement concurrent aggregations support without profile option ([#7514](https://github.com/opensearch-project/OpenSearch/pull/7514)) +- Add dynamic index and cluster setting for concurrent segment search ([#7956](https://github.com/opensearch-project/OpenSearch/pull/7956)) ### Dependencies - Bump `log4j-core` from 2.18.0 to 2.19.0 diff --git a/distribution/src/config/opensearch.yml b/distribution/src/config/opensearch.yml index 107fe345c942b..8c4160db98857 100644 --- a/distribution/src/config/opensearch.yml +++ b/distribution/src/config/opensearch.yml @@ -130,4 +130,11 @@ ${path.logs} # # Gates the search pipeline feature. This feature enables configurable processors # for search requests and search responses, similar to ingest pipelines. +# #opensearch.experimental.feature.search_pipeline.enabled: false +# +# +# Gates the concurrent segment search feature. This feature enables concurrent segment search in a separate +# index searcher threadpool. +# +#opensearch.experimental.feature.concurrent_segment_search.enabled: false diff --git a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java index 6dfa705b12896..fe1d292dbd8f6 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -673,6 +673,8 @@ public void apply(Settings value, Settings current, Settings previous) { IndicesService.CLUSTER_REMOTE_STORE_REPOSITORY_SETTING, IndicesService.CLUSTER_REMOTE_TRANSLOG_STORE_ENABLED_SETTING, IndicesService.CLUSTER_REMOTE_TRANSLOG_REPOSITORY_SETTING - ) + ), + List.of(FeatureFlags.CONCURRENT_SEGMENT_SEARCH), + List.of(SearchService.CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING) ); } diff --git a/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java b/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java index 734830f99e6fb..95c0f3b55222f 100644 --- a/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java @@ -235,7 +235,9 @@ public final class IndexScopedSettings extends AbstractScopedSettings { IndexMetadata.INDEX_REMOTE_TRANSLOG_STORE_ENABLED_SETTING, IndexMetadata.INDEX_REMOTE_TRANSLOG_REPOSITORY_SETTING, IndexMetadata.INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING - ) + ), + FeatureFlags.CONCURRENT_SEGMENT_SEARCH, + List.of(IndexSettings.INDEX_CONCURRENT_SEGMENT_SEARCH_SETTING) ); public static final IndexScopedSettings DEFAULT_SCOPED_SETTINGS = new IndexScopedSettings(Settings.EMPTY, BUILT_IN_INDEX_SETTINGS); diff --git a/server/src/main/java/org/opensearch/index/IndexSettings.java b/server/src/main/java/org/opensearch/index/IndexSettings.java index de7dc102939ce..9c6613495ba80 100644 --- a/server/src/main/java/org/opensearch/index/IndexSettings.java +++ b/server/src/main/java/org/opensearch/index/IndexSettings.java @@ -588,6 +588,13 @@ public final class IndexSettings { Property.IndexScope ); + public static final Setting INDEX_CONCURRENT_SEGMENT_SEARCH_SETTING = Setting.boolSetting( + "index.search.concurrent_segment_search.enabled", + false, + Property.IndexScope, + Property.Dynamic + ); + private final Index index; private final Version version; private final Logger logger; @@ -1602,7 +1609,13 @@ public void setDefaultSearchPipeline(String defaultSearchPipeline) { if (FeatureFlags.isEnabled(SEARCH_PIPELINE)) { this.defaultSearchPipeline = defaultSearchPipeline; } else { - throw new SettingsException("Unsupported setting: " + DEFAULT_SEARCH_PIPELINE.getKey()); + throw new SettingsException( + "Unable to update setting: " + + DEFAULT_SEARCH_PIPELINE.getKey() + + ". This is an experimental feature that is currently disabled, please enable the " + + SEARCH_PIPELINE + + " feature flag first." + ); } } } diff --git a/server/src/main/java/org/opensearch/search/DefaultSearchContext.java b/server/src/main/java/org/opensearch/search/DefaultSearchContext.java index ae369fd87345e..c50d0280aec2d 100644 --- a/server/src/main/java/org/opensearch/search/DefaultSearchContext.java +++ b/server/src/main/java/org/opensearch/search/DefaultSearchContext.java @@ -48,6 +48,7 @@ import org.opensearch.common.lucene.search.Queries; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.BigArrays; +import org.opensearch.common.util.FeatureFlags; import org.opensearch.core.common.lease.Releasables; import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; import org.opensearch.index.IndexService; @@ -104,6 +105,8 @@ import java.util.function.Function; import java.util.function.LongSupplier; +import static org.opensearch.search.SearchService.CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING; + /** * The main search context used during search phase * @@ -869,6 +872,25 @@ public Profilers getProfilers() { return profilers; } + /** + * Returns concurrent segment search status for the search context + */ + @Override + public boolean isConcurrentSegmentSearchEnabled() { + if (FeatureFlags.isEnabled(FeatureFlags.CONCURRENT_SEGMENT_SEARCH) + && (clusterService != null) + && (searcher().getExecutor() != null)) { + return indexService.getIndexSettings() + .getSettings() + .getAsBoolean( + IndexSettings.INDEX_CONCURRENT_SEGMENT_SEARCH_SETTING.getKey(), + clusterService.getClusterSettings().get(CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING) + ); + } else { + return false; + } + } + public void setProfilers(Profilers profilers) { this.profilers = profilers; } diff --git a/server/src/main/java/org/opensearch/search/SearchModule.java b/server/src/main/java/org/opensearch/search/SearchModule.java index a4aa1cbf0d3c2..aeb1d8325b1b8 100644 --- a/server/src/main/java/org/opensearch/search/SearchModule.java +++ b/server/src/main/java/org/opensearch/search/SearchModule.java @@ -257,9 +257,9 @@ import org.opensearch.search.fetch.subphase.highlight.Highlighter; import org.opensearch.search.fetch.subphase.highlight.PlainHighlighter; import org.opensearch.search.fetch.subphase.highlight.UnifiedHighlighter; -import org.opensearch.search.query.ConcurrentQueryPhaseSearcher; import org.opensearch.search.query.QueryPhase; import org.opensearch.search.query.QueryPhaseSearcher; +import org.opensearch.search.query.QueryPhaseSearcherWrapper; import org.opensearch.search.rescore.QueryRescorerBuilder; import org.opensearch.search.rescore.RescorerBuilder; import org.opensearch.search.sort.FieldSortBuilder; @@ -1258,8 +1258,8 @@ private QueryPhaseSearcher registerQueryPhaseSearcher(List plugins } } - if (searcher == null && FeatureFlags.isEnabled(FeatureFlags.CONCURRENT_SEGMENT_SEARCH)) { - searcher = new ConcurrentQueryPhaseSearcher(); + if (searcher == null) { + searcher = new QueryPhaseSearcherWrapper(); } return searcher; } @@ -1290,14 +1290,7 @@ public FetchPhase getFetchPhase() { } public QueryPhase getQueryPhase() { - QueryPhase queryPhase; - if (queryPhaseSearcher == null) { - // use the defaults - queryPhase = new QueryPhase(); - } else { - queryPhase = new QueryPhase(queryPhaseSearcher); - } - return queryPhase; + return new QueryPhase(queryPhaseSearcher); } public @Nullable ExecutorService getIndexSearcherExecutor(ThreadPool pool) { diff --git a/server/src/main/java/org/opensearch/search/SearchService.java b/server/src/main/java/org/opensearch/search/SearchService.java index d72759c506561..23d35cb823342 100644 --- a/server/src/main/java/org/opensearch/search/SearchService.java +++ b/server/src/main/java/org/opensearch/search/SearchService.java @@ -247,6 +247,13 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv Property.NodeScope ); + public static final Setting CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING = Setting.boolSetting( + "search.concurrent_segment_search.enabled", + true, + Property.Dynamic, + Property.NodeScope + ); + public static final int DEFAULT_SIZE = 10; public static final int DEFAULT_FROM = 0; diff --git a/server/src/main/java/org/opensearch/search/internal/SearchContext.java b/server/src/main/java/org/opensearch/search/internal/SearchContext.java index 94ae490ef7d56..79a5b89cfa881 100644 --- a/server/src/main/java/org/opensearch/search/internal/SearchContext.java +++ b/server/src/main/java/org/opensearch/search/internal/SearchContext.java @@ -366,6 +366,13 @@ public final void assignRescoreDocIds(RescoreDocIds rescoreDocIds) { */ public abstract Profilers getProfilers(); + /** + * Returns concurrent segment search status for the search context + */ + public boolean isConcurrentSegmentSearchEnabled() { + return false; + } + /** * Adds a releasable that will be freed when this context is closed. */ diff --git a/server/src/main/java/org/opensearch/search/query/ConcurrentQueryPhaseSearcher.java b/server/src/main/java/org/opensearch/search/query/ConcurrentQueryPhaseSearcher.java index 844e84c081b1c..e3ba0eda4af55 100644 --- a/server/src/main/java/org/opensearch/search/query/ConcurrentQueryPhaseSearcher.java +++ b/server/src/main/java/org/opensearch/search/query/ConcurrentQueryPhaseSearcher.java @@ -48,14 +48,7 @@ protected boolean searchWithCollector( boolean hasFilterCollector, boolean hasTimeout ) throws IOException { - boolean couldUseConcurrentSegmentSearch = allowConcurrentSegmentSearch(searcher); - - if (couldUseConcurrentSegmentSearch) { - LOGGER.debug("Using concurrent search over index segments (experimental)"); - return searchWithCollectorManager(searchContext, searcher, query, collectors, hasFilterCollector, hasTimeout); - } else { - return super.searchWithCollector(searchContext, searcher, query, collectors, hasFilterCollector, hasTimeout); - } + return searchWithCollectorManager(searchContext, searcher, query, collectors, hasFilterCollector, hasTimeout); } private static boolean searchWithCollectorManager( @@ -108,9 +101,4 @@ private static boolean searchWithCollectorManager( public AggregationProcessor aggregationProcessor(SearchContext searchContext) { return aggregationProcessor; } - - private static boolean allowConcurrentSegmentSearch(final ContextIndexSearcher searcher) { - return (searcher.getExecutor() != null); - } - } diff --git a/server/src/main/java/org/opensearch/search/query/QueryPhaseSearcherWrapper.java b/server/src/main/java/org/opensearch/search/query/QueryPhaseSearcherWrapper.java new file mode 100644 index 0000000000000..407603f00461e --- /dev/null +++ b/server/src/main/java/org/opensearch/search/query/QueryPhaseSearcherWrapper.java @@ -0,0 +1,82 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.search.query; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.lucene.search.Query; +import org.opensearch.common.util.FeatureFlags; +import org.opensearch.search.aggregations.AggregationProcessor; +import org.opensearch.search.internal.ContextIndexSearcher; +import org.opensearch.search.internal.SearchContext; +import org.apache.lucene.search.CollectorManager; + +import java.io.IOException; +import java.util.LinkedList; + +/** + * Wrapper class for QueryPhaseSearcher that handles path selection for concurrent vs + * non-concurrent search query phase and aggregation processor. + * + * @opensearch.internal + */ +public class QueryPhaseSearcherWrapper implements QueryPhaseSearcher { + private static final Logger LOGGER = LogManager.getLogger(QueryPhaseSearcherWrapper.class); + private final QueryPhaseSearcher defaultQueryPhaseSearcher; + private final QueryPhaseSearcher concurrentQueryPhaseSearcher; + + public QueryPhaseSearcherWrapper() { + this.defaultQueryPhaseSearcher = new QueryPhase.DefaultQueryPhaseSearcher(); + this.concurrentQueryPhaseSearcher = FeatureFlags.isEnabled(FeatureFlags.CONCURRENT_SEGMENT_SEARCH) + ? new ConcurrentQueryPhaseSearcher() + : null; + } + + /** + * Perform search using {@link CollectorManager} + * + * @param searchContext search context + * @param searcher context index searcher + * @param query query + * @param hasTimeout "true" if timeout was set, "false" otherwise + * @return is rescoring required or not + * @throws java.io.IOException IOException + */ + @Override + public boolean searchWith( + SearchContext searchContext, + ContextIndexSearcher searcher, + Query query, + LinkedList collectors, + boolean hasFilterCollector, + boolean hasTimeout + ) throws IOException { + if (searchContext.isConcurrentSegmentSearchEnabled()) { + LOGGER.info("Using concurrent search over segments (experimental)"); + return concurrentQueryPhaseSearcher.searchWith(searchContext, searcher, query, collectors, hasFilterCollector, hasTimeout); + } else { + return defaultQueryPhaseSearcher.searchWith(searchContext, searcher, query, collectors, hasFilterCollector, hasTimeout); + } + } + + /** + * {@link AggregationProcessor} to use to setup and post process aggregation related collectors during search request + * @param searchContext search context + * @return {@link AggregationProcessor} to use + */ + @Override + public AggregationProcessor aggregationProcessor(SearchContext searchContext) { + if (searchContext.isConcurrentSegmentSearchEnabled()) { + LOGGER.info("Using concurrent search over segments (experimental)"); + return concurrentQueryPhaseSearcher.aggregationProcessor(searchContext); + } else { + return defaultQueryPhaseSearcher.aggregationProcessor(searchContext); + } + } +} diff --git a/server/src/test/java/org/opensearch/common/settings/SettingsModuleTests.java b/server/src/test/java/org/opensearch/common/settings/SettingsModuleTests.java index b15d3518e2f99..4490f6b39996f 100644 --- a/server/src/test/java/org/opensearch/common/settings/SettingsModuleTests.java +++ b/server/src/test/java/org/opensearch/common/settings/SettingsModuleTests.java @@ -36,6 +36,8 @@ import org.opensearch.common.settings.Setting.Property; import org.hamcrest.Matchers; import org.opensearch.common.util.FeatureFlags; +import org.opensearch.index.IndexSettings; +import org.opensearch.search.SearchService; import org.opensearch.test.FeatureFlagSetter; import java.util.Arrays; @@ -282,4 +284,55 @@ public void testDynamicIndexSettingsRegistration() { () -> module.registerDynamicSetting(Setting.floatSetting("index.custom.setting2", 1.0f, Property.IndexScope)) ); } + + public void testConcurrentSegmentSearchClusterSettings() { + // Test that we throw an exception without the feature flag + Settings settings = Settings.builder().put(SearchService.CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING.getKey(), true).build(); + SettingsException ex = expectThrows(SettingsException.class, () -> new SettingsModule(settings)); + assertEquals( + "unknown setting [" + + SearchService.CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING.getKey() + + "] please check that any required plugins are installed, or check the breaking " + + "changes documentation for removed settings", + ex.getMessage() + ); + + // Test that the settings updates correctly with the feature flag + FeatureFlagSetter.set(FeatureFlags.CONCURRENT_SEGMENT_SEARCH); + boolean settingValue = randomBoolean(); + Settings settingsWithFeatureFlag = Settings.builder() + .put(SearchService.CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING.getKey(), settingValue) + .build(); + SettingsModule settingsModule = new SettingsModule(settingsWithFeatureFlag); + assertEquals(settingValue, SearchService.CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING.get(settingsModule.getSettings())); + } + + public void testConcurrentSegmentSearchIndexSettings() { + Settings.Builder target = Settings.builder().put(Settings.EMPTY); + Settings.Builder update = Settings.builder(); + + // Test that we throw an exception without the feature flag + SettingsModule module = new SettingsModule(Settings.EMPTY); + IndexScopedSettings indexScopedSettings = module.getIndexScopedSettings(); + expectThrows( + SettingsException.class, + () -> indexScopedSettings.updateDynamicSettings( + Settings.builder().put(IndexSettings.INDEX_CONCURRENT_SEGMENT_SEARCH_SETTING.getKey(), true).build(), + target, + update, + "node" + ) + ); + + // Test that the settings updates correctly with the feature flag + FeatureFlagSetter.set(FeatureFlags.CONCURRENT_SEGMENT_SEARCH); + SettingsModule moduleWithFeatureFlag = new SettingsModule(Settings.EMPTY); + IndexScopedSettings indexScopedSettingsWithFeatureFlag = moduleWithFeatureFlag.getIndexScopedSettings(); + indexScopedSettingsWithFeatureFlag.updateDynamicSettings( + Settings.builder().put(IndexSettings.INDEX_CONCURRENT_SEGMENT_SEARCH_SETTING.getKey(), true).build(), + target, + update, + "node" + ); + } } diff --git a/server/src/test/java/org/opensearch/search/SearchModuleTests.java b/server/src/test/java/org/opensearch/search/SearchModuleTests.java index 08a19cb89ac68..c0351c9dccbc1 100644 --- a/server/src/test/java/org/opensearch/search/SearchModuleTests.java +++ b/server/src/test/java/org/opensearch/search/SearchModuleTests.java @@ -79,6 +79,7 @@ import org.opensearch.search.query.ConcurrentQueryPhaseSearcher; import org.opensearch.search.query.QueryPhase; import org.opensearch.search.query.QueryPhaseSearcher; +import org.opensearch.search.query.QueryPhaseSearcherWrapper; import org.opensearch.search.rescore.QueryRescorerBuilder; import org.opensearch.search.rescore.RescoreContext; import org.opensearch.search.rescore.RescorerBuilder; @@ -425,7 +426,7 @@ public void testDefaultQueryPhaseSearcher() { SearchModule searchModule = new SearchModule(Settings.EMPTY, Collections.emptyList()); TestSearchContext searchContext = new TestSearchContext(null); QueryPhase queryPhase = searchModule.getQueryPhase(); - assertTrue(queryPhase.getQueryPhaseSearcher() instanceof QueryPhase.DefaultQueryPhaseSearcher); + assertTrue(queryPhase.getQueryPhaseSearcher() instanceof QueryPhaseSearcherWrapper); assertTrue(queryPhase.getQueryPhaseSearcher().aggregationProcessor(searchContext) instanceof DefaultAggregationProcessor); } @@ -434,8 +435,9 @@ public void testConcurrentQueryPhaseSearcher() { FeatureFlags.initializeFeatureFlags(settings); SearchModule searchModule = new SearchModule(settings, Collections.emptyList()); TestSearchContext searchContext = new TestSearchContext(null); + searchContext.setConcurrentSegmentSearchEnabled(true); QueryPhase queryPhase = searchModule.getQueryPhase(); - assertTrue(queryPhase.getQueryPhaseSearcher() instanceof ConcurrentQueryPhaseSearcher); + assertTrue(queryPhase.getQueryPhaseSearcher() instanceof QueryPhaseSearcherWrapper); assertTrue(queryPhase.getQueryPhaseSearcher().aggregationProcessor(searchContext) instanceof ConcurrentAggregationProcessor); FeatureFlags.initializeFeatureFlags(Settings.EMPTY); } diff --git a/server/src/test/java/org/opensearch/search/SearchServiceTests.java b/server/src/test/java/org/opensearch/search/SearchServiceTests.java index 72c74ddb71725..8f8789a3a0323 100644 --- a/server/src/test/java/org/opensearch/search/SearchServiceTests.java +++ b/server/src/test/java/org/opensearch/search/SearchServiceTests.java @@ -61,6 +61,7 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.settings.SettingsException; 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; @@ -227,7 +228,7 @@ public void onQueryPhase(SearchContext context, long tookInNanos) { @Override protected Settings nodeSettings() { - return Settings.builder().put("search.default_search_timeout", "5s").build(); + return Settings.builder().put("search.default_search_timeout", "5s").put(FeatureFlags.CONCURRENT_SEGMENT_SEARCH, true).build(); } public void testClearOnClose() { @@ -1177,6 +1178,109 @@ public void testCreateSearchContext() throws IOException { } } + /** + * Test that the Search Context for concurrent segment search enabled is set correctly based on both + * index and cluster settings. + */ + public void testConcurrentSegmentSearchSearchContext() throws IOException { + Boolean[][] scenarios = { + // cluster setting, index setting, concurrent search enabled? + { null, null, true }, + { null, false, false }, + { null, true, true }, + { true, null, true }, + { true, false, false }, + { true, true, true }, + { false, null, false }, + { false, false, false }, + { false, true, true } }; + + String index = randomAlphaOfLengthBetween(5, 10).toLowerCase(Locale.ROOT); + IndexService indexService = createIndex(index); + final SearchService service = getInstanceFromNode(SearchService.class); + ShardId shardId = new ShardId(indexService.index(), 0); + long nowInMillis = System.currentTimeMillis(); + String clusterAlias = randomBoolean() ? null : randomAlphaOfLengthBetween(3, 10); + SearchRequest searchRequest = new SearchRequest(); + searchRequest.allowPartialSearchResults(randomBoolean()); + ShardSearchRequest request = new ShardSearchRequest( + OriginalIndices.NONE, + searchRequest, + shardId, + indexService.numberOfShards(), + AliasFilter.EMPTY, + 1f, + nowInMillis, + clusterAlias, + Strings.EMPTY_ARRAY + ); + + for (Boolean[] scenario : scenarios) { + Boolean clusterSetting = scenario[0]; + Boolean indexSetting = scenario[1]; + Boolean concurrentSearchEnabled = scenario[2]; + + if (clusterSetting == null) { + client().admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings(Settings.builder().putNull(SearchService.CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING.getKey())) + .get(); + } else { + client().admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings( + Settings.builder().put(SearchService.CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING.getKey(), clusterSetting) + ) + .get(); + } + + if (indexSetting == null) { + client().admin() + .indices() + .prepareUpdateSettings(index) + .setSettings(Settings.builder().putNull(IndexSettings.INDEX_CONCURRENT_SEGMENT_SEARCH_SETTING.getKey())) + .get(); + } else { + client().admin() + .indices() + .prepareUpdateSettings(index) + .setSettings(Settings.builder().put(IndexSettings.INDEX_CONCURRENT_SEGMENT_SEARCH_SETTING.getKey(), indexSetting)) + .get(); + } + + try (DefaultSearchContext searchContext = service.createSearchContext(request, new TimeValue(System.currentTimeMillis()))) { + assertEquals( + clusterSetting, + client().admin() + .cluster() + .prepareState() + .get() + .getState() + .getMetadata() + .transientSettings() + .getAsBoolean(SearchService.CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING.getKey(), null) + ); + assertEquals( + indexSetting == null ? null : indexSetting.toString(), + client().admin() + .indices() + .prepareGetSettings(index) + .get() + .getSetting(index, IndexSettings.INDEX_CONCURRENT_SEGMENT_SEARCH_SETTING.getKey()) + ); + assertEquals(concurrentSearchEnabled, searchContext.isConcurrentSegmentSearchEnabled()); + } + } + // Cleanup + client().admin() + .cluster() + .prepareUpdateSettings() + .setTransientSettings(Settings.builder().putNull(SearchService.CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING.getKey())) + .get(); + } + /** * While we have no NPE in DefaultContext constructor anymore, we still want to guard against it (or other failures) in the future to * avoid leaking searchers. diff --git a/server/src/test/java/org/opensearch/search/aggregations/AggregationSetupTests.java b/server/src/test/java/org/opensearch/search/aggregations/AggregationSetupTests.java index 0095fd097d3f5..73ab8d7dc814c 100644 --- a/server/src/test/java/org/opensearch/search/aggregations/AggregationSetupTests.java +++ b/server/src/test/java/org/opensearch/search/aggregations/AggregationSetupTests.java @@ -13,6 +13,7 @@ import org.opensearch.index.IndexService; import org.opensearch.search.internal.SearchContext; import org.opensearch.test.OpenSearchSingleNodeTestCase; +import org.opensearch.test.TestSearchContext; import java.io.IOException; @@ -36,6 +37,7 @@ public void setUp() throws Exception { client().prepareIndex("idx").setId("1").setSource("f", 5).execute().get(); client().admin().indices().prepareRefresh("idx").get(); context = createSearchContext(index); + ((TestSearchContext) context).setConcurrentSegmentSearchEnabled(true); } protected AggregatorFactories getAggregationFactories(String agg) throws IOException { diff --git a/server/src/test/java/org/opensearch/search/query/QueryPhaseTests.java b/server/src/test/java/org/opensearch/search/query/QueryPhaseTests.java index 7e6d31a51bd4d..dcec4842fc81e 100644 --- a/server/src/test/java/org/opensearch/search/query/QueryPhaseTests.java +++ b/server/src/test/java/org/opensearch/search/query/QueryPhaseTests.java @@ -411,6 +411,9 @@ public void testTerminateAfterEarlyTermination() throws Exception { TestSearchContext context = new TestSearchContext(null, indexShard, newContextSearcher(reader, executor)); context.setTask(new SearchShardTask(123L, "", "", "", null, Collections.emptyMap())); context.parsedQuery(new ParsedQuery(new MatchAllDocsQuery())); + if (this.executor != null) { + context.setConcurrentSegmentSearchEnabled(true); + } context.terminateAfter(numDocs); { diff --git a/test/framework/src/main/java/org/opensearch/test/TestSearchContext.java b/test/framework/src/main/java/org/opensearch/test/TestSearchContext.java index 2b7e1450b9fbc..82e16e6c13005 100644 --- a/test/framework/src/main/java/org/opensearch/test/TestSearchContext.java +++ b/test/framework/src/main/java/org/opensearch/test/TestSearchContext.java @@ -115,6 +115,14 @@ public class TestSearchContext extends SearchContext { private FieldDoc searchAfter; private Profilers profilers; private CollapseContext collapse; + protected boolean concurrentSegmentSearchEnabled; + + /** + * Sets the concurrent segment search enabled field + */ + public void setConcurrentSegmentSearchEnabled(boolean concurrentSegmentSearchEnabled) { + this.concurrentSegmentSearchEnabled = concurrentSegmentSearchEnabled; + } private final Map searchExtBuilders = new HashMap<>(); @@ -605,6 +613,14 @@ public Profilers getProfilers() { return profilers; } + /** + * Returns concurrent segment search status for the search context + */ + @Override + public boolean isConcurrentSegmentSearchEnabled() { + return concurrentSegmentSearchEnabled; + } + @Override public Map, CollectorManager> queryCollectorManagers() { return queryCollectorManagers; From 743eed4c8ecb11c4898d340c0127c84347df5b40 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 12 Jun 2023 13:54:05 -0700 Subject: [PATCH 007/109] Bump commons-io:commons-io from 2.11.0 to 2.13.0 in /buildSrc (#8013) * Bump commons-io:commons-io from 2.11.0 to 2.13.0 in /buildSrc Bumps commons-io:commons-io from 2.11.0 to 2.13.0. --- updated-dependencies: - dependency-name: commons-io:commons-io dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] * Update changelog Signed-off-by: dependabot[bot] --------- Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: dependabot[bot] --- CHANGELOG.md | 2 +- buildSrc/build.gradle | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index d4cf64a600366..cf0c2518efbcd 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -104,7 +104,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `netty` from 4.1.91.Final to 4.1.93.Final ([#7901](https://github.com/opensearch-project/OpenSearch/pull/7901)) - Bump `com.netflix.nebula:gradle-extra-configurations-plugin` from 9.0.0 to 10.0.0 in /buildSrc ([#7068](https://github.com/opensearch-project/OpenSearch/pull/7068)) - Bump `com.google.guava:guava` from 32.0.0-jre to 32.0.1-jre (#8009) -- Bump `commons-io:commons-io` from 2.12.0 to 2.13.0 (#8014) +- Bump `commons-io:commons-io` from 2.12.0 to 2.13.0 (#8014, #8013) ### Changed - Replace jboss-annotations-api_1.2_spec with jakarta.annotation-api ([#7836](https://github.com/opensearch-project/OpenSearch/pull/7836)) diff --git a/buildSrc/build.gradle b/buildSrc/build.gradle index 7e667e81aaa43..56e00ace22dd6 100644 --- a/buildSrc/build.gradle +++ b/buildSrc/build.gradle @@ -109,7 +109,7 @@ dependencies { api 'com.netflix.nebula:nebula-publishing-plugin:20.3.0' api 'com.netflix.nebula:gradle-info-plugin:12.1.3' api 'org.apache.rat:apache-rat:0.15' - api 'commons-io:commons-io:2.11.0' + api 'commons-io:commons-io:2.13.0' api "net.java.dev.jna:jna:5.13.0" api 'gradle.plugin.com.github.johnrengelman:shadow:8.0.0' api 'org.jdom:jdom2:2.0.6.1' From c45073c2e8c8eb850f707fae2adf3ca37c08c896 Mon Sep 17 00:00:00 2001 From: Andrew Ross Date: Mon, 12 Jun 2023 15:31:03 -0700 Subject: [PATCH 008/109] Add @sachinpkale as an OpenSearch maintainer (#8019) I have nominated and maintainers have agreed to invite Sachin Kale (@sachinpkale) to be a co-maintainer. Sachin has kindly accepted. Sachin has led the design and implementation of the remote backed storage feature in OpenSearch. This feature was introduced as experimental in OpenSearch 2.3 and is planned for general availability in 2.9. Some significant issues and PRs authored by Sachin for this effort are as follows: Feature proposal: https://github.com/opensearch-project/OpenSearch/issues/1968 Upload segments to remote store post refresh: https://github.com/opensearch-project/OpenSearch/pull/3460 Add rest endpoint for remote store restore: https://github.com/opensearch-project/OpenSearch/pull/3576 Add RemoteSegmentStoreDirectory to interact with remote segment store: https://github.com/opensearch-project/OpenSearch/pull/4020 In total, Sachin has authored 57 PRs going back to May 2022. He also frequently reviews contributions from others and has reviewed nearly 100 PRs in the same time frame. Signed-off-by: Andrew Ross --- .github/CODEOWNERS | 2 +- MAINTAINERS.md | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS index 7237a21700add..e5d34bfc1e6dd 100644 --- a/.github/CODEOWNERS +++ b/.github/CODEOWNERS @@ -1 +1 @@ -* @reta @anasalkouz @andrross @reta @Bukhtawar @CEHENKLE @dblock @gbbafna @setiah @kartg @kotwanikunal @mch2 @nknize @owaiskazi19 @Rishikesh1159 @ryanbogan @saratvemulapalli @shwetathareja @dreamer-89 @tlfeng @VachaShah @dbwiddis +* @reta @anasalkouz @andrross @reta @Bukhtawar @CEHENKLE @dblock @gbbafna @setiah @kartg @kotwanikunal @mch2 @nknize @owaiskazi19 @Rishikesh1159 @ryanbogan @saratvemulapalli @shwetathareja @dreamer-89 @tlfeng @VachaShah @dbwiddis @sachinpkale diff --git a/MAINTAINERS.md b/MAINTAINERS.md index e05590fb705e7..8027c5daffc69 100644 --- a/MAINTAINERS.md +++ b/MAINTAINERS.md @@ -22,6 +22,7 @@ This document contains a list of maintainers in this repo. See [opensearch-proje | Owais Kazi | [owaiskazi19](https://github.com/owaiskazi19) | Amazon | | Rishikesh Pasham | [Rishikesh1159](https://github.com/Rishikesh1159) | Amazon | | Ryan Bogan | [ryanbogan](https://github.com/ryanbogan) | Amazon | +| Sachin Kale | [sachinpkale](https://github.com/sachinpkale) | Amazon | | Sarat Vemulapalli | [saratvemulapalli](https://github.com/saratvemulapalli) | Amazon | | Shweta Thareja | [shwetathareja](https://github.com/shwetathareja) | Amazon | | Suraj Singh | [dreamer-89](https://github.com/dreamer-89) | Amazon | From 4b4d84e0bc663f8374ee63aea6dab4b8b159bf5b Mon Sep 17 00:00:00 2001 From: Bhumika Saini <108734800+BhumikaSaini-Amazon@users.noreply.github.com> Date: Tue, 13 Jun 2023 10:55:59 +0530 Subject: [PATCH 009/109] Refactor Translog metadata upload/download to write/read header and footer via VersionedCodecStreamWrapper (#7953) Signed-off-by: Bhumika Saini --- .../transfer/TranslogTransferManager.java | 43 ++++++++- .../transfer/TranslogTransferMetadata.java | 52 +---------- .../TranslogTransferMetadataHandler.java | 63 +++++++++++++ .../TranslogTransferManagerTests.java | 4 +- .../TranslogTransferMetadataHandlerTests.java | 93 +++++++++++++++++++ 5 files changed, 201 insertions(+), 54 deletions(-) create mode 100644 server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferMetadataHandler.java create mode 100644 server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferMetadataHandlerTests.java diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java index 352e7dc2cc0e6..489c81f802695 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java @@ -12,9 +12,13 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.OutputStreamIndexOutput; import org.opensearch.action.ActionListener; import org.opensearch.action.LatchedActionListener; import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.common.bytes.BytesReference; +import org.opensearch.common.io.VersionedCodecStreamWrapper; +import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.lucene.store.ByteArrayIndexInput; import org.opensearch.index.shard.ShardId; import org.opensearch.index.translog.Translog; @@ -61,6 +65,12 @@ public class TranslogTransferManager { private final static String METADATA_DIR = "metadata"; private final static String DATA_DIR = "data"; + private static final VersionedCodecStreamWrapper metadataStreamWrapper = new VersionedCodecStreamWrapper<>( + new TranslogTransferMetadataHandler(), + TranslogTransferMetadata.CURRENT_VERSION, + TranslogTransferMetadata.METADATA_CODEC + ); + public TranslogTransferManager( ShardId shardId, TransferService transferService, @@ -174,9 +184,9 @@ private void downloadToFS(String fileName, Path location, String primaryTerm) th public TranslogTransferMetadata readMetadata() throws IOException { return transferService.listAll(remoteMetadataTransferPath).stream().max(METADATA_FILENAME_COMPARATOR).map(filename -> { - try (InputStream inputStream = transferService.downloadBlob(remoteMetadataTransferPath, filename);) { + try (InputStream inputStream = transferService.downloadBlob(remoteMetadataTransferPath, filename)) { IndexInput indexInput = new ByteArrayIndexInput("metadata file", inputStream.readAllBytes()); - return new TranslogTransferMetadata(indexInput); + return metadataStreamWrapper.readStream(indexInput); } catch (IOException e) { logger.error(() -> new ParameterizedMessage("Exception while reading metadata file: {}", filename), e); return null; @@ -197,13 +207,40 @@ private TransferFileSnapshot prepareMetadata(TransferSnapshot transferSnapshot) ); TranslogTransferMetadata translogTransferMetadata = transferSnapshot.getTranslogTransferMetadata(); translogTransferMetadata.setGenerationToPrimaryTermMapper(new HashMap<>(generationPrimaryTermMap)); + return new TransferFileSnapshot( getFileName(translogTransferMetadata.getPrimaryTerm(), translogTransferMetadata.getGeneration()), - translogTransferMetadata.createMetadataBytes(), + getMetadataBytes(translogTransferMetadata), translogTransferMetadata.getPrimaryTerm() ); } + /** + * Get the metadata bytes for a {@link TranslogTransferMetadata} object + * + * @param metadata The object to be parsed + * @return Byte representation for the given metadata + */ + public byte[] getMetadataBytes(TranslogTransferMetadata metadata) throws IOException { + byte[] metadataBytes; + + try (BytesStreamOutput output = new BytesStreamOutput()) { + try ( + OutputStreamIndexOutput indexOutput = new OutputStreamIndexOutput( + "translog transfer metadata " + metadata.getPrimaryTerm(), + getFileName(metadata.getPrimaryTerm(), metadata.getGeneration()), + output, + TranslogTransferMetadata.BUFFER_SIZE + ) + ) { + metadataStreamWrapper.writeStream(indexOutput, metadata); + } + metadataBytes = BytesReference.toBytes(output.bytes()); + } + + return metadataBytes; + } + /** * This method handles deletion of multiple generations for a single primary term. The deletion happens for translog * and metadata files. diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferMetadata.java b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferMetadata.java index 9ce0faaa3352d..7a2fee9a69d5e 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferMetadata.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferMetadata.java @@ -8,18 +8,10 @@ package org.opensearch.index.translog.transfer; -import org.apache.lucene.codecs.CodecUtil; -import org.apache.lucene.store.DataOutput; -import org.apache.lucene.store.IndexInput; -import org.apache.lucene.store.OutputStreamIndexOutput; import org.opensearch.common.SetOnce; -import org.opensearch.common.bytes.BytesReference; -import org.opensearch.common.io.stream.BytesStreamOutput; -import java.io.IOException; import java.util.Arrays; import java.util.Comparator; -import java.util.HashMap; import java.util.Map; import java.util.Objects; @@ -44,11 +36,11 @@ public class TranslogTransferMetadata { public static final String METADATA_SEPARATOR = "__"; - private static final int BUFFER_SIZE = 4096; + static final int BUFFER_SIZE = 4096; - private static final int CURRENT_VERSION = 1; + static final int CURRENT_VERSION = 1; - private static final String METADATA_CODEC = "md"; + static final String METADATA_CODEC = "md"; public static final Comparator METADATA_FILENAME_COMPARATOR = new MetadataFilenameComparator(); @@ -59,15 +51,6 @@ public TranslogTransferMetadata(long primaryTerm, long generation, long minTrans this.count = count; } - public TranslogTransferMetadata(IndexInput indexInput) throws IOException { - CodecUtil.checksumEntireFile(indexInput); - CodecUtil.checkHeader(indexInput, METADATA_CODEC, CURRENT_VERSION, CURRENT_VERSION); - this.primaryTerm = indexInput.readLong(); - this.generation = indexInput.readLong(); - this.minTranslogGeneration = indexInput.readLong(); - this.generationToPrimaryTermMapper.set(indexInput.readMapOfStrings()); - } - public long getPrimaryTerm() { return primaryTerm; } @@ -96,24 +79,6 @@ public static String getFileName(long primaryTerm, long generation) { return String.join(METADATA_SEPARATOR, Arrays.asList(String.valueOf(primaryTerm), String.valueOf(generation))); } - public byte[] createMetadataBytes() throws IOException { - try (BytesStreamOutput output = new BytesStreamOutput()) { - try ( - OutputStreamIndexOutput indexOutput = new OutputStreamIndexOutput( - "translog transfer metadata " + primaryTerm, - getFileName(primaryTerm, generation), - output, - BUFFER_SIZE - ) - ) { - CodecUtil.writeHeader(indexOutput, METADATA_CODEC, CURRENT_VERSION); - write(indexOutput); - CodecUtil.writeFooter(indexOutput); - } - return BytesReference.toBytes(output.bytes()); - } - } - @Override public int hashCode() { return Objects.hash(primaryTerm, generation); @@ -127,17 +92,6 @@ public boolean equals(Object o) { return Objects.equals(this.primaryTerm, other.primaryTerm) && Objects.equals(this.generation, other.generation); } - private void write(DataOutput out) throws IOException { - out.writeLong(primaryTerm); - out.writeLong(generation); - out.writeLong(minTranslogGeneration); - if (generationToPrimaryTermMapper.get() != null) { - out.writeMapOfStrings(generationToPrimaryTermMapper.get()); - } else { - out.writeMapOfStrings(new HashMap<>()); - } - } - private static class MetadataFilenameComparator implements Comparator { @Override public int compare(String first, String second) { diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferMetadataHandler.java b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferMetadataHandler.java new file mode 100644 index 0000000000000..cea7ef8a4e6dd --- /dev/null +++ b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferMetadataHandler.java @@ -0,0 +1,63 @@ +/* + * 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.index.translog.transfer; + +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.opensearch.common.io.IndexIOStreamHandler; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +/** + * Handler for {@link TranslogTransferMetadata} + * + * @opensearch.internal + */ +public class TranslogTransferMetadataHandler implements IndexIOStreamHandler { + + /** + * Implements logic to read content from file input stream {@code indexInput} and parse into {@link TranslogTransferMetadata} + * + * @param indexInput file input stream + * @return content parsed to {@link TranslogTransferMetadata} + */ + @Override + public TranslogTransferMetadata readContent(IndexInput indexInput) throws IOException { + long primaryTerm = indexInput.readLong(); + long generation = indexInput.readLong(); + long minTranslogGeneration = indexInput.readLong(); + Map generationToPrimaryTermMapper = indexInput.readMapOfStrings(); + + int count = generationToPrimaryTermMapper.size(); + TranslogTransferMetadata metadata = new TranslogTransferMetadata(primaryTerm, generation, minTranslogGeneration, count); + metadata.setGenerationToPrimaryTermMapper(generationToPrimaryTermMapper); + + return metadata; + } + + /** + * Implements logic to write content from {@code content} to file output stream {@code indexOutput} + * + * @param indexOutput file input stream + * @param content metadata content to be written + */ + @Override + public void writeContent(IndexOutput indexOutput, TranslogTransferMetadata content) throws IOException { + indexOutput.writeLong(content.getPrimaryTerm()); + indexOutput.writeLong(content.getGeneration()); + indexOutput.writeLong(content.getMinTranslogGeneration()); + if (content.getGenerationToPrimaryTermMapper() != null) { + indexOutput.writeMapOfStrings(content.getGenerationToPrimaryTermMapper()); + } else { + indexOutput.writeMapOfStrings(new HashMap<>()); + } + } +} diff --git a/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferManagerTests.java b/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferManagerTests.java index 1c485dbc35c63..6f6b3622295b6 100644 --- a/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferManagerTests.java +++ b/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferManagerTests.java @@ -204,7 +204,7 @@ public void testReadMetadataSingleFile() throws IOException { TranslogTransferMetadata metadata = createTransferSnapshot().getTranslogTransferMetadata(); when(transferService.downloadBlob(any(BlobPath.class), eq("12__234"))).thenReturn( - new ByteArrayInputStream(metadata.createMetadataBytes()) + new ByteArrayInputStream(translogTransferManager.getMetadataBytes(metadata)) ); assertEquals(metadata, translogTransferManager.readMetadata()); @@ -222,7 +222,7 @@ public void testReadMetadataMultipleFiles() throws IOException { TranslogTransferMetadata metadata = createTransferSnapshot().getTranslogTransferMetadata(); when(transferService.downloadBlob(any(BlobPath.class), eq("12__235"))).thenReturn( - new ByteArrayInputStream(metadata.createMetadataBytes()) + new ByteArrayInputStream(translogTransferManager.getMetadataBytes(metadata)) ); assertEquals(metadata, translogTransferManager.readMetadata()); diff --git a/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferMetadataHandlerTests.java b/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferMetadataHandlerTests.java new file mode 100644 index 0000000000000..ccedd4a711433 --- /dev/null +++ b/server/src/test/java/org/opensearch/index/translog/transfer/TranslogTransferMetadataHandlerTests.java @@ -0,0 +1,93 @@ +/* + * 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.index.translog.transfer; + +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.OutputStreamIndexOutput; +import org.junit.Before; +import org.opensearch.common.bytes.BytesReference; +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.common.lucene.store.ByteArrayIndexInput; +import org.opensearch.test.OpenSearchTestCase; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +public class TranslogTransferMetadataHandlerTests extends OpenSearchTestCase { + private TranslogTransferMetadataHandler handler; + + @Before + public void setUp() throws Exception { + super.setUp(); + handler = new TranslogTransferMetadataHandler(); + } + + public void testReadContent() throws IOException { + TranslogTransferMetadata expectedMetadata = getTestMetadata(); + + // Operation: Read expected metadata from source input stream. + IndexInput indexInput = new ByteArrayIndexInput("metadata file", getTestMetadataBytes()); + TranslogTransferMetadata actualMetadata = handler.readContent(indexInput); + + // Verification: Compare actual metadata read from the source input stream. + assertEquals(expectedMetadata, actualMetadata); + } + + public void testWriteContent() throws IOException { + TranslogTransferMetadata expectedMetadata = getTestMetadata(); + + // Operation: Write expected metadata to the target output stream. + BytesStreamOutput output = new BytesStreamOutput(); + OutputStreamIndexOutput actualMetadataStream = new OutputStreamIndexOutput("dummy bytes", "dummy stream", output, 4096); + handler.writeContent(actualMetadataStream, expectedMetadata); + actualMetadataStream.close(); + + // Verification: Compare actual metadata written to the target output stream. + IndexInput indexInput = new ByteArrayIndexInput("metadata file", BytesReference.toBytes(output.bytes())); + long primaryTerm = indexInput.readLong(); + long generation = indexInput.readLong(); + long minTranslogGeneration = indexInput.readLong(); + Map generationToPrimaryTermMapper = indexInput.readMapOfStrings(); + int count = generationToPrimaryTermMapper.size(); + TranslogTransferMetadata actualMetadata = new TranslogTransferMetadata(primaryTerm, generation, minTranslogGeneration, count); + actualMetadata.setGenerationToPrimaryTermMapper(generationToPrimaryTermMapper); + assertEquals(expectedMetadata, actualMetadata); + } + + private TranslogTransferMetadata getTestMetadata() { + long primaryTerm = 3; + long generation = 500; + long minTranslogGeneration = 300; + Map generationToPrimaryTermMapper = new HashMap<>(); + generationToPrimaryTermMapper.put("300", "1"); + generationToPrimaryTermMapper.put("400", "2"); + generationToPrimaryTermMapper.put("500", "3"); + int count = generationToPrimaryTermMapper.size(); + TranslogTransferMetadata metadata = new TranslogTransferMetadata(primaryTerm, generation, minTranslogGeneration, count); + metadata.setGenerationToPrimaryTermMapper(generationToPrimaryTermMapper); + + return metadata; + } + + private byte[] getTestMetadataBytes() throws IOException { + TranslogTransferMetadata metadata = getTestMetadata(); + + BytesStreamOutput output = new BytesStreamOutput(); + OutputStreamIndexOutput indexOutput = new OutputStreamIndexOutput("dummy bytes", "dummy stream", output, 4096); + indexOutput.writeLong(metadata.getPrimaryTerm()); + indexOutput.writeLong(metadata.getGeneration()); + indexOutput.writeLong(metadata.getMinTranslogGeneration()); + Map generationToPrimaryTermMapper = metadata.getGenerationToPrimaryTermMapper(); + indexOutput.writeMapOfStrings(generationToPrimaryTermMapper); + indexOutput.close(); + + return BytesReference.toBytes(output.bytes()); + } +} From b85ac4c6cb506a229522dbcb4a7c6471efb09f57 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 13 Jun 2023 09:20:28 -0400 Subject: [PATCH 010/109] Bump com.diffplug.spotless from 6.18.0 to 6.19.0 (#8007) * Bump com.diffplug.spotless from 6.18.0 to 6.19.0 Bumps com.diffplug.spotless from 6.18.0 to 6.19.0. --- updated-dependencies: - dependency-name: com.diffplug.spotless dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] * Update changelog Signed-off-by: dependabot[bot] --------- Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: dependabot[bot] --- CHANGELOG.md | 1 + build.gradle | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index cf0c2518efbcd..beff5f90ac5c5 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -105,6 +105,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `com.netflix.nebula:gradle-extra-configurations-plugin` from 9.0.0 to 10.0.0 in /buildSrc ([#7068](https://github.com/opensearch-project/OpenSearch/pull/7068)) - Bump `com.google.guava:guava` from 32.0.0-jre to 32.0.1-jre (#8009) - Bump `commons-io:commons-io` from 2.12.0 to 2.13.0 (#8014, #8013) +- Bump `com.diffplug.spotless` from 6.18.0 to 6.19.0 (#8007) ### Changed - Replace jboss-annotations-api_1.2_spec with jakarta.annotation-api ([#7836](https://github.com/opensearch-project/OpenSearch/pull/7836)) diff --git a/build.gradle b/build.gradle index aa0a3ecd96d89..4b188718e5f30 100644 --- a/build.gradle +++ b/build.gradle @@ -53,7 +53,7 @@ plugins { id 'lifecycle-base' id 'opensearch.docker-support' id 'opensearch.global-build-info' - id "com.diffplug.spotless" version "6.18.0" apply false + id "com.diffplug.spotless" version "6.19.0" apply false id "org.gradle.test-retry" version "1.5.3" apply false id "test-report-aggregation" id 'jacoco-report-aggregation' From df517a6f69afe2a59f55daf5f3b0689128ee6879 Mon Sep 17 00:00:00 2001 From: Stephen Crawford <65832608+scrawfor99@users.noreply.github.com> Date: Tue, 13 Jun 2023 11:43:48 -0400 Subject: [PATCH 011/109] Fix flaky test `SearchRestCancellationIT.testAutomaticCancellationMultiSearchDuringFetchPhase` (#7985) * add latch.await Signed-off-by: Stephen Crawford * remove space Signed-off-by: Stephen Crawford * Fix await timing Signed-off-by: Stephen Crawford * fix line Signed-off-by: Stephen Crawford * shorten await Signed-off-by: Stephen Crawford --------- Signed-off-by: Stephen Crawford --- .../test/java/org/opensearch/http/SearchRestCancellationIT.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/qa/smoke-test-http/src/test/java/org/opensearch/http/SearchRestCancellationIT.java b/qa/smoke-test-http/src/test/java/org/opensearch/http/SearchRestCancellationIT.java index 9692ff895b062..4a898d816bbf4 100644 --- a/qa/smoke-test-http/src/test/java/org/opensearch/http/SearchRestCancellationIT.java +++ b/qa/smoke-test-http/src/test/java/org/opensearch/http/SearchRestCancellationIT.java @@ -83,6 +83,7 @@ import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertNoFailures; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.instanceOf; +import static org.opensearch.test.hamcrest.OpenSearchAssertions.awaitLatch; public class SearchRestCancellationIT extends HttpSmokeTestCase { @@ -183,6 +184,7 @@ public void onFailure(Exception exception) { } }); + latch.await(2, TimeUnit.SECONDS); awaitForBlock(plugins); cancellable.cancel(); ensureSearchTaskIsCancelled(searchAction, nodeIdToName::get); From fcfe30580f6803ba8dbc6e4020e86f8d239e75a5 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 13 Jun 2023 13:42:34 -0400 Subject: [PATCH 012/109] Bump commons-io:commons-io from 2.12.0 to 2.13.0 in /plugins/ingest-attachment (#8010) * Bump commons-io:commons-io in /plugins/ingest-attachment Bumps commons-io:commons-io from 2.12.0 to 2.13.0. --- updated-dependencies: - dependency-name: commons-io:commons-io dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] * Updating SHAs Signed-off-by: dependabot[bot] * Add changelog Signed-off-by: Kunal Kotwani --------- Signed-off-by: dependabot[bot] Signed-off-by: Kunal Kotwani Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: dependabot[bot] Co-authored-by: Kunal Kotwani --- CHANGELOG.md | 4 ++-- plugins/ingest-attachment/build.gradle | 2 +- plugins/ingest-attachment/licenses/commons-io-2.12.0.jar.sha1 | 1 - plugins/ingest-attachment/licenses/commons-io-2.13.0.jar.sha1 | 1 + 4 files changed, 4 insertions(+), 4 deletions(-) delete mode 100644 plugins/ingest-attachment/licenses/commons-io-2.12.0.jar.sha1 create mode 100644 plugins/ingest-attachment/licenses/commons-io-2.13.0.jar.sha1 diff --git a/CHANGELOG.md b/CHANGELOG.md index beff5f90ac5c5..1b76adc925210 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -104,7 +104,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `netty` from 4.1.91.Final to 4.1.93.Final ([#7901](https://github.com/opensearch-project/OpenSearch/pull/7901)) - Bump `com.netflix.nebula:gradle-extra-configurations-plugin` from 9.0.0 to 10.0.0 in /buildSrc ([#7068](https://github.com/opensearch-project/OpenSearch/pull/7068)) - Bump `com.google.guava:guava` from 32.0.0-jre to 32.0.1-jre (#8009) -- Bump `commons-io:commons-io` from 2.12.0 to 2.13.0 (#8014, #8013) +- Bump `commons-io:commons-io` from 2.12.0 to 2.13.0 (#8014, #8013, #8010) - Bump `com.diffplug.spotless` from 6.18.0 to 6.19.0 (#8007) ### Changed @@ -124,4 +124,4 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), ### Security [Unreleased 3.0]: https://github.com/opensearch-project/OpenSearch/compare/2.x...HEAD -[Unreleased 2.x]: https://github.com/opensearch-project/OpenSearch/compare/2.8...2.x \ No newline at end of file +[Unreleased 2.x]: https://github.com/opensearch-project/OpenSearch/compare/2.8...2.x diff --git a/plugins/ingest-attachment/build.gradle b/plugins/ingest-attachment/build.gradle index fe4b0fb4e90ee..5f8d7dbcd9ccd 100644 --- a/plugins/ingest-attachment/build.gradle +++ b/plugins/ingest-attachment/build.gradle @@ -57,7 +57,7 @@ dependencies { runtimeOnly "com.google.guava:guava:${versions.guava}" // Other dependencies api 'org.tukaani:xz:1.9' - api 'commons-io:commons-io:2.12.0' + api 'commons-io:commons-io:2.13.0' api "org.slf4j:slf4j-api:${versions.slf4j}" // character set detection diff --git a/plugins/ingest-attachment/licenses/commons-io-2.12.0.jar.sha1 b/plugins/ingest-attachment/licenses/commons-io-2.12.0.jar.sha1 deleted file mode 100644 index 5225b130fb817..0000000000000 --- a/plugins/ingest-attachment/licenses/commons-io-2.12.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -e5e3eb2ff05b494287f51476bc715161412c525f \ No newline at end of file diff --git a/plugins/ingest-attachment/licenses/commons-io-2.13.0.jar.sha1 b/plugins/ingest-attachment/licenses/commons-io-2.13.0.jar.sha1 new file mode 100644 index 0000000000000..c165136eb5822 --- /dev/null +++ b/plugins/ingest-attachment/licenses/commons-io-2.13.0.jar.sha1 @@ -0,0 +1 @@ +8bb2bc9b4df17e2411533a0708a69f983bf5e83b \ No newline at end of file From 8c7493203bbedc96efb88581a80198d8d26771db Mon Sep 17 00:00:00 2001 From: Andrey Pleskach Date: Tue, 13 Jun 2023 21:25:30 +0200 Subject: [PATCH 013/109] Remove COMPRESSOR variable from CompressorFactory (#7907) Removed a deprecated COMPRESSOR variable from CompressorFactory and use DEFLATE_COMPRESSOR instea Signed-off-by: Andrey Pleskach --- CHANGELOG.md | 1 + .../cluster/coordination/CompressedStreamUtils.java | 2 +- .../common/compress/CompressedXContent.java | 4 ++-- .../opensearch/common/compress/CompressorFactory.java | 11 ++++++----- .../repositories/blobstore/BlobStoreRepository.java | 4 ++-- .../transport/CompressibleBytesOutputStream.java | 2 +- .../opensearch/transport/TransportDecompressor.java | 8 +++++--- .../org/opensearch/transport/TransportLogger.java | 2 +- .../index/mapper/BinaryFieldMapperTests.java | 2 +- .../transport/CompressibleBytesOutputStreamTests.java | 10 ++++++---- .../transport/TransportDecompressorTests.java | 9 ++++++--- 11 files changed, 32 insertions(+), 23 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 1b76adc925210..f88b4c4b18a43 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -81,6 +81,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Remove LegacyESVersion.V_7_10_ Constants ([#5018](https://github.com/opensearch-project/OpenSearch/pull/5018)) - Remove Version.V_1_ Constants ([#5021](https://github.com/opensearch-project/OpenSearch/pull/5021)) - Remove custom Map, List and Set collection classes ([#6871](https://github.com/opensearch-project/OpenSearch/pull/6871)) +- Remove COMPRESSOR variable from CompressorFactory and use DEFLATE_COMPRESSOR instead ([7907](https://github.com/opensearch-project/OpenSearch/pull/7907)) ### Fixed - Fix 'org.apache.hc.core5.http.ParseException: Invalid protocol version' under JDK 16+ ([#4827](https://github.com/opensearch-project/OpenSearch/pull/4827)) diff --git a/server/src/main/java/org/opensearch/cluster/coordination/CompressedStreamUtils.java b/server/src/main/java/org/opensearch/cluster/coordination/CompressedStreamUtils.java index 57359f553b5a5..97a93a8fa99d4 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/CompressedStreamUtils.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/CompressedStreamUtils.java @@ -37,7 +37,7 @@ public final class CompressedStreamUtils { public static BytesReference createCompressedStream(Version version, CheckedConsumer outputConsumer) throws IOException { final BytesStreamOutput bStream = new BytesStreamOutput(); - try (StreamOutput stream = new OutputStreamStreamOutput(CompressorFactory.COMPRESSOR.threadLocalOutputStream(bStream))) { + try (StreamOutput stream = new OutputStreamStreamOutput(CompressorFactory.defaultCompressor().threadLocalOutputStream(bStream))) { stream.setVersion(version); outputConsumer.accept(stream); } diff --git a/server/src/main/java/org/opensearch/common/compress/CompressedXContent.java b/server/src/main/java/org/opensearch/common/compress/CompressedXContent.java index 462943e974f1e..f8c4dc3346b8d 100644 --- a/server/src/main/java/org/opensearch/common/compress/CompressedXContent.java +++ b/server/src/main/java/org/opensearch/common/compress/CompressedXContent.java @@ -85,7 +85,7 @@ private CompressedXContent(byte[] compressed, int crc32) { */ public CompressedXContent(ToXContent xcontent, ToXContent.Params params) throws IOException { BytesStreamOutput bStream = new BytesStreamOutput(); - OutputStream compressedStream = CompressorFactory.COMPRESSOR.threadLocalOutputStream(bStream); + OutputStream compressedStream = CompressorFactory.defaultCompressor().threadLocalOutputStream(bStream); CRC32 crc32 = new CRC32(); OutputStream checkedStream = new CheckedOutputStream(compressedStream, crc32); try (XContentBuilder builder = XContentFactory.jsonBuilder(checkedStream)) { @@ -113,7 +113,7 @@ public CompressedXContent(BytesReference data) throws IOException { this.bytes = BytesReference.toBytes(data); this.crc32 = crc32(uncompressed()); } else { - this.bytes = BytesReference.toBytes(CompressorFactory.COMPRESSOR.compress(data)); + this.bytes = BytesReference.toBytes(CompressorFactory.defaultCompressor().compress(data)); this.crc32 = crc32(data); } assertConsistent(); diff --git a/server/src/main/java/org/opensearch/common/compress/CompressorFactory.java b/server/src/main/java/org/opensearch/common/compress/CompressorFactory.java index 23a8cac349bd2..4b8d0cfea57f0 100644 --- a/server/src/main/java/org/opensearch/common/compress/CompressorFactory.java +++ b/server/src/main/java/org/opensearch/common/compress/CompressorFactory.java @@ -48,9 +48,6 @@ public class CompressorFactory { public static final Compressor DEFLATE_COMPRESSOR = new DeflateCompressor(); - @Deprecated - public static final Compressor COMPRESSOR = DEFLATE_COMPRESSOR; - public static final Compressor ZSTD_COMPRESSOR = new ZstdCompressor(); public static final Compressor NONE_COMPRESSOR = new NoneCompressor(); @@ -59,14 +56,18 @@ public static boolean isCompressed(BytesReference bytes) { return compressor(bytes) != null; } + public static Compressor defaultCompressor() { + return DEFLATE_COMPRESSOR; + } + @Nullable public static Compressor compressor(BytesReference bytes) { - if (COMPRESSOR.isCompressed(bytes)) { + if (DEFLATE_COMPRESSOR.isCompressed(bytes)) { // bytes should be either detected as compressed or as xcontent, // if we have bytes that can be either detected as compressed or // as a xcontent, we have a problem assert XContentHelper.xContentType(bytes) == null; - return COMPRESSOR; + return DEFLATE_COMPRESSOR; } else if (ZSTD_COMPRESSOR.isCompressed(bytes)) { assert XContentHelper.xContentType(bytes) == null; return ZSTD_COMPRESSOR; diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java index b67ffc8107d96..13550117a24a4 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java @@ -1720,7 +1720,7 @@ private void cacheRepositoryData(BytesReference updated, long generation) { if (cacheRepositoryData && bestEffortConsistency == false) { final BytesReference serialized; try { - serialized = CompressorFactory.COMPRESSOR.compress(updated); + serialized = CompressorFactory.defaultCompressor().compress(updated); final int len = serialized.length(); if (len > ByteSizeUnit.KB.toBytes(500)) { logger.debug( @@ -1756,7 +1756,7 @@ private void cacheRepositoryData(BytesReference updated, long generation) { } private RepositoryData repositoryDataFromCachedEntry(Tuple cacheEntry) throws IOException { - try (InputStream input = CompressorFactory.COMPRESSOR.threadLocalInputStream(cacheEntry.v2().streamInput())) { + try (InputStream input = CompressorFactory.defaultCompressor().threadLocalInputStream(cacheEntry.v2().streamInput())) { return RepositoryData.snapshotsFromXContent( XContentType.JSON.xContent().createParser(NamedXContentRegistry.EMPTY, LoggingDeprecationHandler.INSTANCE, input), cacheEntry.v1() diff --git a/server/src/main/java/org/opensearch/transport/CompressibleBytesOutputStream.java b/server/src/main/java/org/opensearch/transport/CompressibleBytesOutputStream.java index 6877d9679eb81..58cbc406612f4 100644 --- a/server/src/main/java/org/opensearch/transport/CompressibleBytesOutputStream.java +++ b/server/src/main/java/org/opensearch/transport/CompressibleBytesOutputStream.java @@ -68,7 +68,7 @@ final class CompressibleBytesOutputStream extends StreamOutput { this.bytesStreamOutput = bytesStreamOutput; this.shouldCompress = shouldCompress; if (shouldCompress) { - this.stream = CompressorFactory.COMPRESSOR.threadLocalOutputStream(Streams.flushOnCloseStream(bytesStreamOutput)); + this.stream = CompressorFactory.defaultCompressor().threadLocalOutputStream(Streams.flushOnCloseStream(bytesStreamOutput)); } else { this.stream = bytesStreamOutput; } diff --git a/server/src/main/java/org/opensearch/transport/TransportDecompressor.java b/server/src/main/java/org/opensearch/transport/TransportDecompressor.java index 7690bfdf35d8a..6ecb1dba7ad16 100644 --- a/server/src/main/java/org/opensearch/transport/TransportDecompressor.java +++ b/server/src/main/java/org/opensearch/transport/TransportDecompressor.java @@ -37,6 +37,7 @@ import org.opensearch.common.bytes.BytesArray; import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.bytes.ReleasableBytesReference; +import org.opensearch.common.compress.Compressor; import org.opensearch.common.compress.CompressorFactory; import org.opensearch.common.recycler.Recycler; import org.opensearch.common.util.PageCacheRecycler; @@ -69,7 +70,8 @@ public TransportDecompressor(PageCacheRecycler recycler) { public int decompress(BytesReference bytesReference) throws IOException { int bytesConsumed = 0; if (hasReadHeader == false) { - if (CompressorFactory.COMPRESSOR.isCompressed(bytesReference) == false) { + final Compressor compressor = CompressorFactory.defaultCompressor(); + if (compressor.isCompressed(bytesReference) == false) { int maxToRead = Math.min(bytesReference.length(), 10); StringBuilder sb = new StringBuilder("stream marked as compressed, but no compressor found, first [").append(maxToRead) .append("] content bytes out of [") @@ -85,7 +87,7 @@ public int decompress(BytesReference bytesReference) throws IOException { throw new IllegalStateException(sb.toString()); } hasReadHeader = true; - int headerLength = CompressorFactory.COMPRESSOR.headerLength(); + int headerLength = compressor.headerLength(); bytesReference = bytesReference.slice(headerLength, bytesReference.length() - headerLength); bytesConsumed += headerLength; } @@ -135,7 +137,7 @@ public int decompress(BytesReference bytesReference) throws IOException { } public boolean canDecompress(int bytesAvailable) { - return hasReadHeader || bytesAvailable >= CompressorFactory.COMPRESSOR.headerLength(); + return hasReadHeader || bytesAvailable >= CompressorFactory.defaultCompressor().headerLength(); } public boolean isEOS() { diff --git a/server/src/main/java/org/opensearch/transport/TransportLogger.java b/server/src/main/java/org/opensearch/transport/TransportLogger.java index c97b35a150e91..1f7facbca49ec 100644 --- a/server/src/main/java/org/opensearch/transport/TransportLogger.java +++ b/server/src/main/java/org/opensearch/transport/TransportLogger.java @@ -179,7 +179,7 @@ private static String format(TcpChannel channel, InboundMessage message, String private static StreamInput decompressingStream(byte status, StreamInput streamInput) throws IOException { if (TransportStatus.isCompress(status) && streamInput.available() > 0) { try { - return new InputStreamStreamInput(CompressorFactory.COMPRESSOR.threadLocalInputStream(streamInput)); + return new InputStreamStreamInput(CompressorFactory.defaultCompressor().threadLocalInputStream(streamInput)); } catch (IllegalArgumentException e) { throw new IllegalStateException("stream marked as compressed, but is missing deflate header"); } diff --git a/server/src/test/java/org/opensearch/index/mapper/BinaryFieldMapperTests.java b/server/src/test/java/org/opensearch/index/mapper/BinaryFieldMapperTests.java index 2ebbb5c841cbd..878fd48e3fb02 100644 --- a/server/src/test/java/org/opensearch/index/mapper/BinaryFieldMapperTests.java +++ b/server/src/test/java/org/opensearch/index/mapper/BinaryFieldMapperTests.java @@ -119,7 +119,7 @@ public void testStoredValue() throws IOException { // case 2: a value that looks compressed: this used to fail in 1.x BytesStreamOutput out = new BytesStreamOutput(); - try (OutputStream compressed = CompressorFactory.COMPRESSOR.threadLocalOutputStream(out)) { + try (OutputStream compressed = CompressorFactory.defaultCompressor().threadLocalOutputStream(out)) { new BytesArray(binaryValue1).writeTo(compressed); } final byte[] binaryValue2 = BytesReference.toBytes(out.bytes()); diff --git a/server/src/test/java/org/opensearch/transport/CompressibleBytesOutputStreamTests.java b/server/src/test/java/org/opensearch/transport/CompressibleBytesOutputStreamTests.java index 643b6665cd659..1c300bc5e2adf 100644 --- a/server/src/test/java/org/opensearch/transport/CompressibleBytesOutputStreamTests.java +++ b/server/src/test/java/org/opensearch/transport/CompressibleBytesOutputStreamTests.java @@ -56,7 +56,7 @@ public void testStreamWithoutCompression() throws IOException { // Closing compression stream does not close underlying stream stream.close(); - assertFalse(CompressorFactory.COMPRESSOR.isCompressed(bytesRef)); + assertFalse(CompressorFactory.defaultCompressor().isCompressed(bytesRef)); StreamInput streamInput = bytesRef.streamInput(); byte[] actualBytes = new byte[expectedBytes.length]; @@ -83,9 +83,11 @@ public void testStreamWithCompression() throws IOException { BytesReference bytesRef = stream.materializeBytes(); stream.close(); - assertTrue(CompressorFactory.COMPRESSOR.isCompressed(bytesRef)); + assertTrue(CompressorFactory.defaultCompressor().isCompressed(bytesRef)); - StreamInput streamInput = new InputStreamStreamInput(CompressorFactory.COMPRESSOR.threadLocalInputStream(bytesRef.streamInput())); + StreamInput streamInput = new InputStreamStreamInput( + CompressorFactory.defaultCompressor().threadLocalInputStream(bytesRef.streamInput()) + ); byte[] actualBytes = new byte[expectedBytes.length]; streamInput.readBytes(actualBytes, 0, expectedBytes.length); @@ -108,7 +110,7 @@ public void testCompressionWithCallingMaterializeFails() throws IOException { stream.write(expectedBytes); StreamInput streamInput = new InputStreamStreamInput( - CompressorFactory.COMPRESSOR.threadLocalInputStream(bStream.bytes().streamInput()) + CompressorFactory.defaultCompressor().threadLocalInputStream(bStream.bytes().streamInput()) ); byte[] actualBytes = new byte[expectedBytes.length]; EOFException e = expectThrows(EOFException.class, () -> streamInput.readBytes(actualBytes, 0, expectedBytes.length)); diff --git a/server/src/test/java/org/opensearch/transport/TransportDecompressorTests.java b/server/src/test/java/org/opensearch/transport/TransportDecompressorTests.java index 08308f80ab09d..4675f12b3e086 100644 --- a/server/src/test/java/org/opensearch/transport/TransportDecompressorTests.java +++ b/server/src/test/java/org/opensearch/transport/TransportDecompressorTests.java @@ -53,7 +53,10 @@ public class TransportDecompressorTests extends OpenSearchTestCase { public void testSimpleCompression() throws IOException { try (BytesStreamOutput output = new BytesStreamOutput()) { byte randomByte = randomByte(); - try (OutputStream deflateStream = CompressorFactory.COMPRESSOR.threadLocalOutputStream(Streams.flushOnCloseStream(output))) { + try ( + OutputStream deflateStream = CompressorFactory.defaultCompressor() + .threadLocalOutputStream(Streams.flushOnCloseStream(output)) + ) { deflateStream.write(randomByte); } @@ -74,7 +77,7 @@ public void testMultiPageCompression() throws IOException { try (BytesStreamOutput output = new BytesStreamOutput()) { try ( StreamOutput deflateStream = new OutputStreamStreamOutput( - CompressorFactory.COMPRESSOR.threadLocalOutputStream(Streams.flushOnCloseStream(output)) + CompressorFactory.defaultCompressor().threadLocalOutputStream(Streams.flushOnCloseStream(output)) ) ) { for (int i = 0; i < 10000; ++i) { @@ -106,7 +109,7 @@ public void testIncrementalMultiPageCompression() throws IOException { try (BytesStreamOutput output = new BytesStreamOutput()) { try ( StreamOutput deflateStream = new OutputStreamStreamOutput( - CompressorFactory.COMPRESSOR.threadLocalOutputStream(Streams.flushOnCloseStream(output)) + CompressorFactory.defaultCompressor().threadLocalOutputStream(Streams.flushOnCloseStream(output)) ) ) { for (int i = 0; i < 10000; ++i) { From c3ca78ce64f03a04fab1d30896817c5b07d3afc6 Mon Sep 17 00:00:00 2001 From: Poojita Raj Date: Tue, 13 Jun 2023 14:30:39 -0700 Subject: [PATCH 014/109] workflow added (#8033) Signed-off-by: Poojita Raj --- .github/workflows/poc-checklist.yml | 49 +++++++++++++++++++++++++++++ 1 file changed, 49 insertions(+) create mode 100644 .github/workflows/poc-checklist.yml diff --git a/.github/workflows/poc-checklist.yml b/.github/workflows/poc-checklist.yml new file mode 100644 index 0000000000000..2dfb1bbe5cdce --- /dev/null +++ b/.github/workflows/poc-checklist.yml @@ -0,0 +1,49 @@ +name: Add comment +on: + issues: + types: + - labeled +jobs: + add-comment: + if: github.event.label.name == 'poc' + runs-on: ubuntu-latest + permissions: + issues: write + steps: + - name: Add comment + uses: peter-evans/create-or-update-comment@v2 + with: + issue-number: ${{ github.event.issue.number }} + body: | + ### POC Checklist: + + Please go through the following checklist to ensure these items are taken into account while designing the POC. + + - [ ] Supports safe upgrade paths from all supported BWC versions to the current version + - [ ] Supports compatibility with all plugins + - [ ] opensearch-alerting + - [ ] opensearch-anomaly-detection + - [ ] opensearch-asynchronous-search + - [ ] opensearch-cross-cluster-replication + - [ ] opensearch-geospatial + - [ ] opensearch-index-management + - [ ] opensearch-job-scheduler + - [ ] opensearch-knn + - [ ] opensearch-ml + - [ ] opensearch-notifications + - [ ] opensearch-notifications-core + - [ ] opensearch-observability + - [ ] opensearch-performance-analyzer + - [ ] opensearch-reports-scheduler + - [ ] opensearch-security + - [ ] opensearch-sql + - [ ] Supports lucene upgrades across minor lucene versions + - [ ] Supports lucene upgrades across major lucene versions + - [ ] Supports lucene upgrades across underlying lucene codec bumps (Eg: Lucene95Codec -> Lucene96Codec) + - [ ] Supports wire compatibility of OpenSearch + - [ ] Plan to measure performance degradation/improvement (if any) + - [ ] Plan to document any user facing changes introduced by this feature + - [ ] Ensure working and passing CI + + Thank you for your contribution! + From 99ddee15df7a50ea575f578f8c7ca2f44b1a8d38 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 13 Jun 2023 17:44:58 -0400 Subject: [PATCH 015/109] Bump io.projectreactor:reactor-core in /plugins/repository-azure (#8016) Bumps [io.projectreactor:reactor-core](https://github.com/reactor/reactor-core) from 3.5.1 to 3.5.6. - [Release notes](https://github.com/reactor/reactor-core/releases) - [Commits](https://github.com/reactor/reactor-core/compare/v3.5.1...v3.5.6) --- updated-dependencies: - dependency-name: io.projectreactor:reactor-core dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Signed-off-by: Andriy Redko Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- CHANGELOG.md | 1 + plugins/repository-azure/build.gradle | 3 ++- plugins/repository-azure/licenses/reactor-core-3.5.1.jar.sha1 | 1 - plugins/repository-azure/licenses/reactor-core-3.5.6.jar.sha1 | 1 + 4 files changed, 4 insertions(+), 2 deletions(-) delete mode 100644 plugins/repository-azure/licenses/reactor-core-3.5.1.jar.sha1 create mode 100644 plugins/repository-azure/licenses/reactor-core-3.5.6.jar.sha1 diff --git a/CHANGELOG.md b/CHANGELOG.md index f88b4c4b18a43..ae670546a1f14 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -50,6 +50,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `com.networknt:json-schema-validator` from 1.0.81 to 1.0.83 ([7933](https://github.com/opensearch-project/OpenSearch/pull/7933/)) - Bump `com.google.http-client:google-http-client:1.43.2` from 1.42.0 to 1.43.2 ([7928](https://github.com/opensearch-project/OpenSearch/pull/7928))) - Bump `com.azure:azure-storage-blob` from 12.21.1 to 12.22.2 ([7930](https://github.com/opensearch-project/OpenSearch/pull/7930)) +- Bump `io.projectreactor:reactor-core` from 3.5.1 to 3.5.6 in /plugins/repository-azure ([#8016](https://github.com/opensearch-project/OpenSearch/pull/8016)) ### Changed - [CCR] Add getHistoryOperationsFromTranslog method to fetch the history snapshot from translogs ([#3948](https://github.com/opensearch-project/OpenSearch/pull/3948)) diff --git a/plugins/repository-azure/build.gradle b/plugins/repository-azure/build.gradle index df8f2dfb5922a..d711950f1392d 100644 --- a/plugins/repository-azure/build.gradle +++ b/plugins/repository-azure/build.gradle @@ -57,7 +57,7 @@ dependencies { implementation project(':modules:transport-netty4') api 'com.azure:azure-storage-blob:12.22.2' api 'org.reactivestreams:reactive-streams:1.0.4' - api 'io.projectreactor:reactor-core:3.5.1' + api 'io.projectreactor:reactor-core:3.5.6' api 'io.projectreactor.netty:reactor-netty:1.1.7' api 'io.projectreactor.netty:reactor-netty-core:1.1.7' api 'io.projectreactor.netty:reactor-netty-http:1.1.7' @@ -170,6 +170,7 @@ thirdPartyAudit { 'org.slf4j.impl.StaticMarkerBinder', 'reactor.blockhound.BlockHound$Builder', 'reactor.blockhound.integration.BlockHoundIntegration', + 'io.micrometer.context.ThreadLocalAccessor', 'io.micrometer.common.KeyValue', 'io.micrometer.common.KeyValues', 'io.micrometer.common.docs.KeyName', diff --git a/plugins/repository-azure/licenses/reactor-core-3.5.1.jar.sha1 b/plugins/repository-azure/licenses/reactor-core-3.5.1.jar.sha1 deleted file mode 100644 index 697203d58ff36..0000000000000 --- a/plugins/repository-azure/licenses/reactor-core-3.5.1.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -d1e833c13320d3a3133be6a70a4f1a82466f65fe \ No newline at end of file diff --git a/plugins/repository-azure/licenses/reactor-core-3.5.6.jar.sha1 b/plugins/repository-azure/licenses/reactor-core-3.5.6.jar.sha1 new file mode 100644 index 0000000000000..ad9b7263e7b38 --- /dev/null +++ b/plugins/repository-azure/licenses/reactor-core-3.5.6.jar.sha1 @@ -0,0 +1 @@ +027fdc551537b349389176a23a192f11a7a3d7de \ No newline at end of file From 0cd904b7030af58562a3586c8cf87bcf2c94f256 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 13 Jun 2023 15:45:29 -0700 Subject: [PATCH 016/109] Bump com.google.guava:guava in /distribution/tools/plugin-cli (#8012) Bumps [com.google.guava:guava](https://github.com/google/guava) from 32.0.0-jre to 32.0.1-jre. - [Release notes](https://github.com/google/guava/releases) - [Commits](https://github.com/google/guava/commits) --- updated-dependencies: - dependency-name: com.google.guava:guava dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- distribution/tools/plugin-cli/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/distribution/tools/plugin-cli/build.gradle b/distribution/tools/plugin-cli/build.gradle index 894e4fc67e019..477f66625e124 100644 --- a/distribution/tools/plugin-cli/build.gradle +++ b/distribution/tools/plugin-cli/build.gradle @@ -39,7 +39,7 @@ dependencies { api "org.bouncycastle:bc-fips:1.0.2.3" testImplementation project(":test:framework") testImplementation 'com.google.jimfs:jimfs:1.2' - testRuntimeOnly 'com.google.guava:guava:32.0.0-jre' + testRuntimeOnly 'com.google.guava:guava:32.0.1-jre' implementation 'org.apache.commons:commons-compress:1.23.0' } From 95135ec32123e0b60d3be90e31b095d62faa3033 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 13 Jun 2023 16:16:22 -0700 Subject: [PATCH 017/109] Bump org.jruby.joni:joni from 2.1.48 to 2.2.1 in /libs/grok (#8015) * Bump org.jruby.joni:joni from 2.1.48 to 2.2.1 in /libs/grok Bumps [org.jruby.joni:joni](https://github.com/jruby/joni) from 2.1.48 to 2.2.1. - [Commits](https://github.com/jruby/joni/compare/joni-2.1.48...joni-2.2.1) --- updated-dependencies: - dependency-name: org.jruby.joni:joni dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] * Updating SHAs Signed-off-by: dependabot[bot] * Update changelog Signed-off-by: dependabot[bot] * Update method calls to Region beginning and end The "beg" and "end" public members were marked as deprecated in v2.1.47, with direct access being removed in v2.2.0. "getBeg()" and "getEnd()" are now available instead. Signed-off-by: Kartik Ganesh * Spotless Signed-off-by: Kartik Ganesh --------- Signed-off-by: dependabot[bot] Signed-off-by: Kartik Ganesh Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: dependabot[bot] Co-authored-by: Kartik Ganesh --- CHANGELOG.md | 1 + libs/grok/build.gradle | 2 +- libs/grok/licenses/joni-2.1.48.jar.sha1 | 1 - libs/grok/licenses/joni-2.2.1.jar.sha1 | 1 + libs/grok/src/main/java/org/opensearch/grok/Grok.java | 11 ++++++++--- .../java/org/opensearch/grok/GrokCaptureType.java | 6 +++--- 6 files changed, 14 insertions(+), 8 deletions(-) delete mode 100644 libs/grok/licenses/joni-2.1.48.jar.sha1 create mode 100644 libs/grok/licenses/joni-2.2.1.jar.sha1 diff --git a/CHANGELOG.md b/CHANGELOG.md index ae670546a1f14..5a186ba702033 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -108,6 +108,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `com.google.guava:guava` from 32.0.0-jre to 32.0.1-jre (#8009) - Bump `commons-io:commons-io` from 2.12.0 to 2.13.0 (#8014, #8013, #8010) - Bump `com.diffplug.spotless` from 6.18.0 to 6.19.0 (#8007) +- Bump `org.jruby.joni:joni` from 2.1.48 to 2.2.1 (#8015) ### Changed - Replace jboss-annotations-api_1.2_spec with jakarta.annotation-api ([#7836](https://github.com/opensearch-project/OpenSearch/pull/7836)) diff --git a/libs/grok/build.gradle b/libs/grok/build.gradle index 2e3cf8d45ceab..97cf68a7971fa 100644 --- a/libs/grok/build.gradle +++ b/libs/grok/build.gradle @@ -29,7 +29,7 @@ */ dependencies { - api 'org.jruby.joni:joni:2.1.48' + api 'org.jruby.joni:joni:2.2.1' // joni dependencies: api 'org.jruby.jcodings:jcodings:1.0.58' diff --git a/libs/grok/licenses/joni-2.1.48.jar.sha1 b/libs/grok/licenses/joni-2.1.48.jar.sha1 deleted file mode 100644 index 4ac2f3842559b..0000000000000 --- a/libs/grok/licenses/joni-2.1.48.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -552f95e640553de15243c02bd97a6c0c3dd7a78f \ No newline at end of file diff --git a/libs/grok/licenses/joni-2.2.1.jar.sha1 b/libs/grok/licenses/joni-2.2.1.jar.sha1 new file mode 100644 index 0000000000000..d0338540f5512 --- /dev/null +++ b/libs/grok/licenses/joni-2.2.1.jar.sha1 @@ -0,0 +1 @@ +23d2f2eff7fa0cda465d86ec9d8bab53e496d9e6 \ No newline at end of file diff --git a/libs/grok/src/main/java/org/opensearch/grok/Grok.java b/libs/grok/src/main/java/org/opensearch/grok/Grok.java index e80bde02360ed..2585916aae38b 100644 --- a/libs/grok/src/main/java/org/opensearch/grok/Grok.java +++ b/libs/grok/src/main/java/org/opensearch/grok/Grok.java @@ -214,8 +214,8 @@ private String groupMatch(String name, Region region, String pattern) { name.getBytes(StandardCharsets.UTF_8).length, region ); - int begin = region.beg[number]; - int end = region.end[number]; + int begin = region.getBeg(number); + int end = region.getEnd(number); return new String(pattern.getBytes(StandardCharsets.UTF_8), begin, end - begin, StandardCharsets.UTF_8); } catch (StringIndexOutOfBoundsException e) { return null; @@ -270,7 +270,12 @@ protected String toRegex(String grokPattern) { grokPart = String.format(Locale.US, "(?<%s>%s)", patternName + "_" + result, pattern); } String start = new String(grokPatternBytes, 0, result, StandardCharsets.UTF_8); - String rest = new String(grokPatternBytes, region.end[0], grokPatternBytes.length - region.end[0], StandardCharsets.UTF_8); + String rest = new String( + grokPatternBytes, + region.getEnd(0), + grokPatternBytes.length - region.getEnd(0), + StandardCharsets.UTF_8 + ); grokPattern = grokPart + rest; res.append(start); } diff --git a/libs/grok/src/main/java/org/opensearch/grok/GrokCaptureType.java b/libs/grok/src/main/java/org/opensearch/grok/GrokCaptureType.java index f5898414827ff..7f9555d01e386 100644 --- a/libs/grok/src/main/java/org/opensearch/grok/GrokCaptureType.java +++ b/libs/grok/src/main/java/org/opensearch/grok/GrokCaptureType.java @@ -105,9 +105,9 @@ protected final GrokCaptureExtracter rawExtracter(int[] backRefs, Consumer= 0) { - int matchOffset = offset + region.beg[number]; - int matchLength = region.end[number] - region.beg[number]; + if (region.getBeg(number) >= 0) { + int matchOffset = offset + region.getBeg(number); + int matchLength = region.getEnd(number) - region.getBeg(number); emit.accept(new String(utf8Bytes, matchOffset, matchLength, StandardCharsets.UTF_8)); return; // Capture only the first value. } From a81ef5aa322faf46e7c8be776571612b93c646f5 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 13 Jun 2023 16:51:35 -0700 Subject: [PATCH 018/109] Bump com.google.guava:guava from 32.0.0-jre to 32.0.1-jre in /distribution/tools/upgrade-cli (#8011) * Bump com.google.guava:guava in /distribution/tools/upgrade-cli Bumps [com.google.guava:guava](https://github.com/google/guava) from 32.0.0-jre to 32.0.1-jre. - [Release notes](https://github.com/google/guava/releases) - [Commits](https://github.com/google/guava/commits) --- updated-dependencies: - dependency-name: com.google.guava:guava dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] * Update CHANGELOG Signed-off-by: Kartik Ganesh --------- Signed-off-by: dependabot[bot] Signed-off-by: Kartik Ganesh Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Kartik Ganesh --- CHANGELOG.md | 2 ++ distribution/tools/upgrade-cli/build.gradle | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 5a186ba702033..8e7359cd93172 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -51,6 +51,8 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `com.google.http-client:google-http-client:1.43.2` from 1.42.0 to 1.43.2 ([7928](https://github.com/opensearch-project/OpenSearch/pull/7928))) - Bump `com.azure:azure-storage-blob` from 12.21.1 to 12.22.2 ([7930](https://github.com/opensearch-project/OpenSearch/pull/7930)) - Bump `io.projectreactor:reactor-core` from 3.5.1 to 3.5.6 in /plugins/repository-azure ([#8016](https://github.com/opensearch-project/OpenSearch/pull/8016)) +- Bump `com.google.guava:guava` from 32.0.0-jre to 32.0.1-jre in /distribution/tools/upgrade-cli ([#8011](https://github.com/opensearch-project/OpenSearch/pull/8011)) +- Bump `com.google.guava:guava` from 32.0.0-jre to 32.0.1-jre in /distribution/tools/plugin-cli ([#8012](https://github.com/opensearch-project/OpenSearch/pull/8012)) ### Changed - [CCR] Add getHistoryOperationsFromTranslog method to fetch the history snapshot from translogs ([#3948](https://github.com/opensearch-project/OpenSearch/pull/3948)) diff --git a/distribution/tools/upgrade-cli/build.gradle b/distribution/tools/upgrade-cli/build.gradle index 5ba3ab22139a3..cb87faf7788fa 100644 --- a/distribution/tools/upgrade-cli/build.gradle +++ b/distribution/tools/upgrade-cli/build.gradle @@ -19,7 +19,7 @@ dependencies { implementation "com.fasterxml.jackson.core:jackson-annotations:${versions.jackson}" testImplementation project(":test:framework") testImplementation 'com.google.jimfs:jimfs:1.2' - testRuntimeOnly 'com.google.guava:guava:32.0.0-jre' + testRuntimeOnly 'com.google.guava:guava:32.0.1-jre' } tasks.named("dependencyLicenses").configure { From bf3cf7134447ae1cdfbef51e5bbf2bfae44290b8 Mon Sep 17 00:00:00 2001 From: Gaurav Bafna <85113518+gbbafna@users.noreply.github.com> Date: Wed, 14 Jun 2023 16:16:50 +0530 Subject: [PATCH 019/109] [Remote Store] Deletion of Remote Segments and Translog upon Index Deletion (#7682) Signed-off-by: Gaurav Bafna --- .../shard/GlobalCheckpointListenersIT.java | 2 +- .../opensearch/index/shard/IndexShardIT.java | 4 +- .../RemoteStoreBaseIntegTestCase.java | 28 ++++++++++++- .../opensearch/remotestore/RemoteStoreIT.java | 36 +++++++++++++++++ .../org/opensearch/index/IndexService.java | 8 +++- .../opensearch/index/shard/IndexShard.java | 30 +++++++++----- .../index/store/RemoteDirectory.java | 4 ++ .../store/RemoteSegmentStoreDirectory.java | 40 ++++++++++++++++--- .../lockmanager/RemoteStoreLockManager.java | 7 +++- .../RemoteStoreMetadataLockManager.java | 4 ++ .../translog/InternalTranslogManager.java | 4 ++ .../index/translog/NoOpTranslogManager.java | 2 + .../index/translog/RemoteFsTranslog.java | 10 +++++ .../opensearch/index/translog/Translog.java | 2 + .../index/translog/TranslogManager.java | 5 +++ .../transfer/TranslogTransferManager.java | 23 +++++++++-- .../index/engine/NoOpEngineRecoveryTests.java | 2 +- .../RecoveryDuringReplicationTests.java | 10 ++--- .../index/shard/IndexShardTests.java | 12 +++--- .../SegmentReplicationIndexShardTests.java | 12 +++--- .../RemoteSegmentStoreDirectoryTests.java | 2 +- .../RemoteSegmentMetadataHandlerTests.java | 2 +- .../IndexingMemoryControllerTests.java | 2 +- .../PeerRecoveryTargetServiceTests.java | 2 +- .../indices/recovery/RecoveryStatusTests.java | 2 +- .../indices/recovery/RecoveryTests.java | 6 +-- .../BlobStoreRepositoryRestoreTests.java | 4 +- .../index/shard/IndexShardTestCase.java | 2 +- 28 files changed, 214 insertions(+), 53 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/index/shard/GlobalCheckpointListenersIT.java b/server/src/internalClusterTest/java/org/opensearch/index/shard/GlobalCheckpointListenersIT.java index a2c5c0333bbfe..76ff2f809cb83 100644 --- a/server/src/internalClusterTest/java/org/opensearch/index/shard/GlobalCheckpointListenersIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/index/shard/GlobalCheckpointListenersIT.java @@ -126,7 +126,7 @@ public void accept(final long g, final Exception e) { } }, null); - shard.close("closed", randomBoolean()); + shard.close("closed", randomBoolean(), false); assertBusy(() -> assertTrue(invoked.get())); } diff --git a/server/src/internalClusterTest/java/org/opensearch/index/shard/IndexShardIT.java b/server/src/internalClusterTest/java/org/opensearch/index/shard/IndexShardIT.java index ba567c125c6e9..1524acc4e7dd7 100644 --- a/server/src/internalClusterTest/java/org/opensearch/index/shard/IndexShardIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/index/shard/IndexShardIT.java @@ -620,7 +620,7 @@ public void testShardHasMemoryBufferOnTranslogRecover() throws Throwable { client().prepareIndex("test").setId("1").setSource("{\"foo\" : \"bar\"}", XContentType.JSON).setRefreshPolicy(IMMEDIATE).get(); CheckedFunction wrapper = directoryReader -> directoryReader; - shard.close("simon says", false); + shard.close("simon says", false, false); AtomicReference shardRef = new AtomicReference<>(); List failures = new ArrayList<>(); IndexingOperationListener listener = new IndexingOperationListener() { @@ -658,7 +658,7 @@ public void postDelete(ShardId shardId, Engine.Delete delete, Engine.DeleteResul try { ExceptionsHelper.rethrowAndSuppress(failures); } finally { - newShard.close("just do it", randomBoolean()); + newShard.close("just do it", randomBoolean(), false); } } diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java index 42850fc59c8ad..0ffa5ab23e0b6 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java @@ -18,7 +18,13 @@ import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.test.OpenSearchIntegTestCase; +import java.io.IOException; +import java.nio.file.FileVisitResult; +import java.nio.file.Files; import java.nio.file.Path; +import java.nio.file.SimpleFileVisitor; +import java.nio.file.attribute.BasicFileAttributes; +import java.util.concurrent.atomic.AtomicInteger; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; @@ -27,6 +33,8 @@ public class RemoteStoreBaseIntegTestCase extends OpenSearchIntegTestCase { protected static final int SHARD_COUNT = 1; protected static final int REPLICA_COUNT = 1; + protected Path absolutePath; + @Override protected boolean addMockInternalEngine() { return false; @@ -73,7 +81,7 @@ protected Settings remoteTranslogIndexSettings(int numberOfReplicas) { @Before public void setup() { internalCluster().startClusterManagerOnlyNode(); - Path absolutePath = randomRepoPath().toAbsolutePath(); + absolutePath = randomRepoPath().toAbsolutePath(); assertAcked( clusterAdmin().preparePutRepository(REPOSITORY_NAME).setType("fs").setSettings(Settings.builder().put("location", absolutePath)) ); @@ -84,4 +92,22 @@ public void teardown() { assertAcked(clusterAdmin().prepareDeleteRepository(REPOSITORY_NAME)); } + public int getFileCount(Path path) throws Exception { + final AtomicInteger filesExisting = new AtomicInteger(0); + Files.walkFileTree(path, new SimpleFileVisitor<>() { + @Override + public FileVisitResult postVisitDirectory(Path dir, IOException impossible) throws IOException { + return FileVisitResult.CONTINUE; + } + + @Override + public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) { + filesExisting.incrementAndGet(); + return FileVisitResult.CONTINUE; + } + }); + + return filesExisting.get(); + } + } diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java index 290f0df591c64..70a41d74a57c5 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java @@ -9,6 +9,7 @@ package org.opensearch.remotestore; import org.opensearch.action.admin.cluster.remotestore.restore.RestoreRemoteStoreRequest; +import org.opensearch.action.admin.indices.delete.DeleteIndexRequest; import org.opensearch.action.admin.indices.recovery.RecoveryResponse; import org.opensearch.action.index.IndexResponse; import org.opensearch.action.support.PlainActionFuture; @@ -23,6 +24,7 @@ import org.opensearch.test.transport.MockTransportService; import java.io.IOException; +import java.nio.file.Path; import java.util.Arrays; import java.util.Collection; import java.util.HashMap; @@ -30,6 +32,7 @@ import java.util.Optional; import java.util.concurrent.TimeUnit; +import static org.hamcrest.Matchers.comparesEqualTo; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertHitCount; @@ -241,4 +244,37 @@ public void testPeerRecoveryWithRemoteStoreAndRemoteTranslogNoDataRefresh() thro public void testPeerRecoveryWithRemoteStoreAndRemoteTranslogRefresh() throws Exception { testPeerRecovery(true, randomIntBetween(2, 5), false); } + + private void verifyRemoteStoreCleanup(boolean remoteTranslog) throws Exception { + internalCluster().startDataOnlyNodes(3); + if (remoteTranslog) { + createIndex(INDEX_NAME, remoteTranslogIndexSettings(1)); + } else { + createIndex(INDEX_NAME, remoteStoreIndexSettings(1)); + } + + indexData(5, randomBoolean()); + String indexUUID = client().admin() + .indices() + .prepareGetSettings(INDEX_NAME) + .get() + .getSetting(INDEX_NAME, IndexMetadata.SETTING_INDEX_UUID); + Path indexPath = Path.of(String.valueOf(absolutePath), indexUUID); + assertTrue(getFileCount(indexPath) > 0); + assertAcked(client().admin().indices().delete(new DeleteIndexRequest(INDEX_NAME)).get()); + // Delete is async. Give time for it + assertBusy(() -> { + try { + assertThat(getFileCount(indexPath), comparesEqualTo(0)); + } catch (Exception e) {} + }, 30, TimeUnit.SECONDS); + } + + public void testRemoteSegmentCleanup() throws Exception { + verifyRemoteStoreCleanup(false); + } + + public void testRemoteTranslogCleanup() throws Exception { + verifyRemoteStoreCleanup(true); + } } diff --git a/server/src/main/java/org/opensearch/index/IndexService.java b/server/src/main/java/org/opensearch/index/IndexService.java index 4e808ebb838e7..73797106bb66f 100644 --- a/server/src/main/java/org/opensearch/index/IndexService.java +++ b/server/src/main/java/org/opensearch/index/IndexService.java @@ -603,6 +603,7 @@ public synchronized void removeShard(int shardId, String reason) { private void closeShard(String reason, ShardId sId, IndexShard indexShard, Store store, IndexEventListener listener) { final int shardId = sId.id(); final Settings indexSettings = this.getIndexSettings().getSettings(); + Store remoteStore = indexShard.remoteStore(); if (store != null) { store.beforeClose(); } @@ -616,7 +617,7 @@ private void closeShard(String reason, ShardId sId, IndexShard indexShard, Store try { // only flush if we are closed (closed index or shutdown) and if we are not deleted final boolean flushEngine = deleted.get() == false && closed.get(); - indexShard.close(reason, flushEngine); + indexShard.close(reason, flushEngine, deleted.get()); } catch (Exception e) { logger.debug(() -> new ParameterizedMessage("[{}] failed to close index shard", shardId), e); // ignore @@ -632,6 +633,11 @@ private void closeShard(String reason, ShardId sId, IndexShard indexShard, Store } else { logger.trace("[{}] store not initialized prior to closing shard, nothing to close", shardId); } + + if (remoteStore != null && indexShard.isPrimaryMode() && deleted.get()) { + remoteStore.close(); + } + } catch (Exception e) { logger.warn( () -> new ParameterizedMessage("[{}] failed to close store on shard removal (reason: [{}])", shardId, reason), diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index ed38c561c7e29..946488bf441bc 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -1882,7 +1882,7 @@ public CacheHelper getReaderCacheHelper() { } - public void close(String reason, boolean flushEngine) throws IOException { + public void close(String reason, boolean flushEngine, boolean deleted) throws IOException { synchronized (engineMutex) { try { synchronized (mutex) { @@ -1898,12 +1898,30 @@ public void close(String reason, boolean flushEngine) throws IOException { // playing safe here and close the engine even if the above succeeds - close can be called multiple times // Also closing refreshListeners to prevent us from accumulating any more listeners IOUtils.close(engine, globalCheckpointListeners, refreshListeners, pendingReplicationActions); + + if (deleted && engine != null && isPrimaryMode()) { + // Translog Clean up + engine.translogManager().onDelete(); + } + indexShardOperationPermits.close(); } } } } + /* + ToDo : Fix this https://github.com/opensearch-project/OpenSearch/issues/8003 + */ + private RemoteSegmentStoreDirectory getRemoteDirectory() { + assert indexSettings.isRemoteStoreEnabled(); + assert remoteStore.directory() instanceof FilterDirectory : "Store.directory is not an instance of FilterDirectory"; + FilterDirectory remoteStoreDirectory = (FilterDirectory) remoteStore.directory(); + FilterDirectory byteSizeCachingStoreDirectory = (FilterDirectory) remoteStoreDirectory.getDelegate(); + final Directory remoteDirectory = byteSizeCachingStoreDirectory.getDelegate(); + return ((RemoteSegmentStoreDirectory) remoteDirectory); + } + public void preRecovery() { final IndexShardState currentState = this.state; // single volatile read if (currentState == IndexShardState.CLOSED) { @@ -4520,16 +4538,10 @@ public void syncSegmentsFromRemoteSegmentStore(boolean overrideLocal, boolean re throws IOException { assert indexSettings.isRemoteStoreEnabled(); logger.info("Downloading segments from remote segment store"); - assert remoteStore.directory() instanceof FilterDirectory : "Store.directory is not an instance of FilterDirectory"; - FilterDirectory remoteStoreDirectory = (FilterDirectory) remoteStore.directory(); - assert remoteStoreDirectory.getDelegate() instanceof FilterDirectory - : "Store.directory is not enclosing an instance of FilterDirectory"; - FilterDirectory byteSizeCachingStoreDirectory = (FilterDirectory) remoteStoreDirectory.getDelegate(); - final Directory remoteDirectory = byteSizeCachingStoreDirectory.getDelegate(); + RemoteSegmentStoreDirectory remoteDirectory = getRemoteDirectory(); // We need to call RemoteSegmentStoreDirectory.init() in order to get latest metadata of the files that // are uploaded to the remote segment store. - assert remoteDirectory instanceof RemoteSegmentStoreDirectory : "remoteDirectory is not an instance of RemoteSegmentStoreDirectory"; - RemoteSegmentMetadata remoteSegmentMetadata = ((RemoteSegmentStoreDirectory) remoteDirectory).init(); + RemoteSegmentMetadata remoteSegmentMetadata = remoteDirectory.init(); Map uploadedSegments = ((RemoteSegmentStoreDirectory) remoteDirectory) .getSegmentsUploadedToRemoteStore(); store.incRef(); diff --git a/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java b/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java index 5192fd49b91f6..be4b4e910bb4d 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java @@ -210,4 +210,8 @@ public void rename(String source, String dest) throws IOException { public Lock obtainLock(String name) throws IOException { throw new UnsupportedOperationException(); } + + public void delete() throws IOException { + blobContainer.delete(); + } } diff --git a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java index 93fa4b7eff7b7..ca6834438ed61 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java @@ -20,24 +20,24 @@ import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.IndexOutput; import org.opensearch.common.UUIDs; +import org.opensearch.common.io.VersionedCodecStreamWrapper; import org.opensearch.common.lucene.store.ByteArrayIndexInput; +import org.opensearch.index.store.lockmanager.FileLockInfo; import org.opensearch.index.store.lockmanager.RemoteStoreCommitLevelLockManager; import org.opensearch.index.store.lockmanager.RemoteStoreLockManager; -import org.opensearch.index.store.lockmanager.FileLockInfo; import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadata; -import org.opensearch.common.io.VersionedCodecStreamWrapper; import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadataHandler; import java.io.IOException; import java.nio.file.NoSuchFileException; -import java.util.Map; -import java.util.HashSet; -import java.util.Optional; -import java.util.HashMap; import java.util.Collection; import java.util.Collections; import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; import java.util.List; +import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicBoolean; @@ -617,4 +617,32 @@ public void deleteStaleSegments(int lastNMetadataFilesToKeep) throws IOException } } } + + /* + Tries to delete shard level directory if it is empty + Return true if it deleted it successfully + */ + private boolean deleteIfEmpty() throws IOException { + Collection metadataFiles = remoteMetadataDirectory.listFilesByPrefix(MetadataFilenameUtils.METADATA_PREFIX); + if (metadataFiles.size() != 0) { + logger.info("Remote directory still has files , not deleting the path"); + return false; + } + + try { + remoteDataDirectory.delete(); + remoteMetadataDirectory.delete(); + mdLockManager.delete(); + } catch (Exception e) { + logger.error("Exception occurred while deleting directory", e); + return false; + } + + return true; + } + + public void close() throws IOException { + deleteStaleSegments(0); + deleteIfEmpty(); + } } diff --git a/server/src/main/java/org/opensearch/index/store/lockmanager/RemoteStoreLockManager.java b/server/src/main/java/org/opensearch/index/store/lockmanager/RemoteStoreLockManager.java index ce657627fcfc6..c30be082b4795 100644 --- a/server/src/main/java/org/opensearch/index/store/lockmanager/RemoteStoreLockManager.java +++ b/server/src/main/java/org/opensearch/index/store/lockmanager/RemoteStoreLockManager.java @@ -22,7 +22,7 @@ public interface RemoteStoreLockManager { * @param lockInfo lock info instance for which we need to acquire lock. * @throws IOException throws exception in case there is a problem with acquiring lock. */ - public void acquire(LockInfo lockInfo) throws IOException; + void acquire(LockInfo lockInfo) throws IOException; /** * @@ -38,4 +38,9 @@ public interface RemoteStoreLockManager { * @throws IOException throws exception in case there is a problem in checking if a given file is locked or not. */ Boolean isAcquired(LockInfo lockInfo) throws IOException; + + /* + Deletes all lock related files and directories + */ + void delete() throws IOException; } diff --git a/server/src/main/java/org/opensearch/index/store/lockmanager/RemoteStoreMetadataLockManager.java b/server/src/main/java/org/opensearch/index/store/lockmanager/RemoteStoreMetadataLockManager.java index 41665ebe47600..7df20cae10664 100644 --- a/server/src/main/java/org/opensearch/index/store/lockmanager/RemoteStoreMetadataLockManager.java +++ b/server/src/main/java/org/opensearch/index/store/lockmanager/RemoteStoreMetadataLockManager.java @@ -83,4 +83,8 @@ public Boolean isAcquired(LockInfo lockInfo) throws IOException { Collection lockFiles = lockDirectory.listFilesByPrefix(((FileLockInfo) lockInfo).getLockPrefix()); return !lockFiles.isEmpty(); } + + public void delete() throws IOException { + lockDirectory.delete(); + } } diff --git a/server/src/main/java/org/opensearch/index/translog/InternalTranslogManager.java b/server/src/main/java/org/opensearch/index/translog/InternalTranslogManager.java index 0eb133eb464d2..7eaab67ddb5a5 100644 --- a/server/src/main/java/org/opensearch/index/translog/InternalTranslogManager.java +++ b/server/src/main/java/org/opensearch/index/translog/InternalTranslogManager.java @@ -296,6 +296,10 @@ public void setMinSeqNoToKeep(long seqNo) { translog.setMinSeqNoToKeep(seqNo); } + public void onDelete() { + translog.onDelete(); + } + /** * Reads operations from the translog * @param location location of translog diff --git a/server/src/main/java/org/opensearch/index/translog/NoOpTranslogManager.java b/server/src/main/java/org/opensearch/index/translog/NoOpTranslogManager.java index cea38b4fbc781..58ee8c0fd39e7 100644 --- a/server/src/main/java/org/opensearch/index/translog/NoOpTranslogManager.java +++ b/server/src/main/java/org/opensearch/index/translog/NoOpTranslogManager.java @@ -120,4 +120,6 @@ public Translog.Location add(Translog.Operation operation) throws IOException { public Translog.Snapshot newChangesSnapshot(long fromSeqNo, long toSeqNo, boolean requiredFullRange) throws IOException { throw new UnsupportedOperationException("Translog snapshot unsupported with no-op translogs"); } + + public void onDelete() {} } diff --git a/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java b/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java index 190ca6948f42a..939402058e048 100644 --- a/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java +++ b/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java @@ -422,4 +422,14 @@ private void deleteStaleRemotePrimaryTermsAndMetadataFiles() { translogTransferManager.deleteStaleTranslogMetadataFilesAsync(); } } + + protected void onDelete() { + if (primaryModeSupplier.getAsBoolean() == false) { + logger.trace("skipped delete translog"); + // NO-OP + return; + } + // clean up all remote translog files + translogTransferManager.delete(); + } } 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 422864219745b..5816fdfaff754 100644 --- a/server/src/main/java/org/opensearch/index/translog/Translog.java +++ b/server/src/main/java/org/opensearch/index/translog/Translog.java @@ -1807,6 +1807,8 @@ protected long getMinReferencedGen() throws IOException { */ protected void setMinSeqNoToKeep(long seqNo) {} + protected void onDelete() {} + /** * deletes all files associated with a reader. package-private to be able to simulate node failures at this point */ diff --git a/server/src/main/java/org/opensearch/index/translog/TranslogManager.java b/server/src/main/java/org/opensearch/index/translog/TranslogManager.java index e2818dd702d87..420d6cdc43bbf 100644 --- a/server/src/main/java/org/opensearch/index/translog/TranslogManager.java +++ b/server/src/main/java/org/opensearch/index/translog/TranslogManager.java @@ -126,4 +126,9 @@ public interface TranslogManager { * This might be required when segments are persisted via other mechanism than flush. */ void setMinSeqNoToKeep(long seqNo); + + /* + Clean up if any needed on deletion of index + */ + void onDelete(); } diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java index 489c81f802695..58aca00d2e9d3 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java @@ -56,6 +56,7 @@ public class TranslogTransferManager { private final TransferService transferService; private final BlobPath remoteDataTransferPath; private final BlobPath remoteMetadataTransferPath; + private final BlobPath remoteBaseTransferPath; private final FileTransferTracker fileTransferTracker; private static final long TRANSFER_TIMEOUT_IN_MILLIS = 30000; @@ -74,13 +75,14 @@ public class TranslogTransferManager { public TranslogTransferManager( ShardId shardId, TransferService transferService, - BlobPath remoteDataTransferPath, + BlobPath remoteBaseTransferPath, FileTransferTracker fileTransferTracker ) { this.shardId = shardId; this.transferService = transferService; - this.remoteDataTransferPath = remoteDataTransferPath.add(DATA_DIR); - this.remoteMetadataTransferPath = remoteDataTransferPath.add(METADATA_DIR); + this.remoteBaseTransferPath = remoteBaseTransferPath; + this.remoteDataTransferPath = remoteBaseTransferPath.add(DATA_DIR); + this.remoteMetadataTransferPath = remoteBaseTransferPath.add(METADATA_DIR); this.fileTransferTracker = fileTransferTracker; } @@ -324,6 +326,21 @@ public void onFailure(Exception e) { ); } + public void delete() { + // cleans up all the translog contents in async fashion + transferService.deleteAsync(ThreadPool.Names.REMOTE_PURGE, remoteBaseTransferPath, new ActionListener<>() { + @Override + public void onResponse(Void unused) { + logger.info("Deleted all remote translog data for {}", shardId); + } + + @Override + public void onFailure(Exception e) { + logger.error("Exception occurred while cleaning translog ", e); + } + }); + } + public void deleteStaleTranslogMetadataFilesAsync() { transferService.listAllAsync(ThreadPool.Names.REMOTE_PURGE, remoteMetadataTransferPath, new ActionListener<>() { @Override diff --git a/server/src/test/java/org/opensearch/index/engine/NoOpEngineRecoveryTests.java b/server/src/test/java/org/opensearch/index/engine/NoOpEngineRecoveryTests.java index 3162f7915c994..d8b55815b5b05 100644 --- a/server/src/test/java/org/opensearch/index/engine/NoOpEngineRecoveryTests.java +++ b/server/src/test/java/org/opensearch/index/engine/NoOpEngineRecoveryTests.java @@ -50,7 +50,7 @@ public void testRecoverFromNoOp() throws IOException { for (int i = 0; i < nbDocs; i++) { indexDoc(indexShard, "_doc", String.valueOf(i)); } - indexShard.close("test", true); + indexShard.close("test", true, false); final ShardRouting shardRouting = indexShard.routingEntry(); IndexShard primary = reinitShard( diff --git a/server/src/test/java/org/opensearch/index/replication/RecoveryDuringReplicationTests.java b/server/src/test/java/org/opensearch/index/replication/RecoveryDuringReplicationTests.java index 5ac10824e14e2..d73cdfd3fe93f 100644 --- a/server/src/test/java/org/opensearch/index/replication/RecoveryDuringReplicationTests.java +++ b/server/src/test/java/org/opensearch/index/replication/RecoveryDuringReplicationTests.java @@ -161,10 +161,10 @@ public void testRecoveryToReplicaThatReceivedExtraDocument() throws Exception { new SourceToParse("index", "replica", new BytesArray("{}"), XContentType.JSON) ); shards.promoteReplicaToPrimary(promotedReplica).get(); - oldPrimary.close("demoted", randomBoolean()); + oldPrimary.close("demoted", randomBoolean(), false); oldPrimary.store().close(); shards.removeReplica(remainingReplica); - remainingReplica.close("disconnected", false); + remainingReplica.close("disconnected", false, false); remainingReplica.store().close(); // randomly introduce a conflicting document final boolean extra = randomBoolean(); @@ -289,7 +289,7 @@ public void testRecoveryAfterPrimaryPromotion() throws Exception { newPrimary.flush(new FlushRequest()); } - oldPrimary.close("demoted", false); + oldPrimary.close("demoted", false, false); oldPrimary.store().close(); IndexShard newReplica = shards.addReplicaWithExistingPath(oldPrimary.shardPath(), oldPrimary.routingEntry().currentNodeId()); @@ -335,7 +335,7 @@ public void testReplicaRollbackStaleDocumentsInPeerRecovery() throws Exception { shards.promoteReplicaToPrimary(newPrimary).get(); // Recover a replica should rollback the stale documents shards.removeReplica(replica); - replica.close("recover replica - first time", false); + replica.close("recover replica - first time", false, false); replica.store().close(); replica = shards.addReplicaWithExistingPath(replica.shardPath(), replica.routingEntry().currentNodeId()); shards.recoverReplica(replica); @@ -346,7 +346,7 @@ public void testReplicaRollbackStaleDocumentsInPeerRecovery() throws Exception { assertThat(replica.getLastSyncedGlobalCheckpoint(), equalTo(replica.seqNoStats().getMaxSeqNo())); // Recover a replica again should also rollback the stale documents. shards.removeReplica(replica); - replica.close("recover replica - second time", false); + replica.close("recover replica - second time", false, false); replica.store().close(); IndexShard anotherReplica = shards.addReplicaWithExistingPath(replica.shardPath(), replica.routingEntry().currentNodeId()); shards.recoverReplica(anotherReplica); diff --git a/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java index eb7ff360ec5d9..67f149c3cb5ae 100644 --- a/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java @@ -291,7 +291,7 @@ public void testFailShard() throws Exception { assertNotNull(shardPath); // fail shard shard.failShard("test shard fail", new CorruptIndexException("", "")); - shard.close("do not assert history", false); + shard.close("do not assert history", false, false); shard.store().close(); // check state file still exists ShardStateMetadata shardStateMetadata = load(logger, shardPath.getShardStatePath()); @@ -1614,7 +1614,7 @@ public void testSnapshotStore() throws IOException { snapshot = newShard.snapshotStoreMetadata(); assertThat(snapshot.getSegmentsFile().name(), equalTo("segments_3")); - newShard.close("test", false); + newShard.close("test", false, false); snapshot = newShard.snapshotStoreMetadata(); assertThat(snapshot.getSegmentsFile().name(), equalTo("segments_3")); @@ -1874,7 +1874,7 @@ public void testIndexingOperationsListeners() throws IOException { AtomicInteger preDelete = new AtomicInteger(); AtomicInteger postDelete = new AtomicInteger(); AtomicInteger postDeleteException = new AtomicInteger(); - shard.close("simon says", true); + shard.close("simon says", true, false); shard = reinitShard(shard, new IndexingOperationListener() { @Override public Engine.Index preIndex(ShardId shardId, Engine.Index operation) { @@ -1961,7 +1961,7 @@ public void postDelete(ShardId shardId, Engine.Delete delete, Exception ex) { assertEquals(1, postDelete.get()); assertEquals(0, postDeleteException.get()); - shard.close("Unexpected close", true); + shard.close("Unexpected close", true, false); shard.state = IndexShardState.STARTED; // It will generate exception try { @@ -4362,7 +4362,7 @@ public void onBeginTranslogRecovery() { Thread closeShardThread = new Thread(() -> { try { readyToCloseLatch.await(); - shard.close("testing", false); + shard.close("testing", false, false); // in integration tests, this is done as a listener on IndexService. MockFSDirectoryFactory.checkIndex(logger, shard.store(), shard.shardId); } catch (InterruptedException | IOException e) { @@ -4811,7 +4811,7 @@ public void testCloseShardWhileEngineIsWarming() throws Exception { recoveryThread.start(); try { warmerStarted.await(); - shard.close("testing", false); + shard.close("testing", false, false); assertThat(shard.state, equalTo(IndexShardState.CLOSED)); } finally { warmerBlocking.countDown(); diff --git a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java index 0ee3e81678511..aba04e9d30159 100644 --- a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java @@ -579,7 +579,7 @@ public void testReplicaReceivesLowerGeneration() throws Exception { assertEqualCommittedSegments(primary, replica_1); shards.promoteReplicaToPrimary(replica_2).get(); - primary.close("demoted", false); + primary.close("demoted", false, false); primary.store().close(); IndexShard oldPrimary = shards.addReplicaWithExistingPath(primary.shardPath(), primary.routingEntry().currentNodeId()); shards.recoverReplica(oldPrimary); @@ -618,7 +618,7 @@ public void testReplicaRestarts() throws Exception { // randomly resetart a replica final IndexShard replicaToRestart = getRandomReplica(shards); - replicaToRestart.close("restart", false); + replicaToRestart.close("restart", false, false); replicaToRestart.store().close(); shards.removeReplica(replicaToRestart); final IndexShard newReplica = shards.addReplicaWithExistingPath( @@ -716,7 +716,7 @@ private void testNRTReplicaWithRemoteStorePromotedAsPrimary(boolean performFlush shards.promoteReplicaToPrimary(nextPrimary).get(); // close oldPrimary. - oldPrimary.close("demoted", false); + oldPrimary.close("demoted", false, false); oldPrimary.store().close(); assertEquals(InternalEngine.class, nextPrimary.getEngine().getClass()); @@ -783,7 +783,7 @@ public void testNRTReplicaPromotedAsPrimary() throws Exception { shards.promoteReplicaToPrimary(nextPrimary); // close and start the oldPrimary as a replica. - oldPrimary.close("demoted", false); + oldPrimary.close("demoted", false, false); oldPrimary.store().close(); oldPrimary = shards.addReplicaWithExistingPath(oldPrimary.shardPath(), oldPrimary.routingEntry().currentNodeId()); shards.recoverReplica(oldPrimary); @@ -866,7 +866,7 @@ public void onFailure(Exception e) { assertEquals(nextPrimary.getEngine().getClass(), InternalEngine.class); nextPrimary.refresh("test"); - oldPrimary.close("demoted", false); + oldPrimary.close("demoted", false, false); oldPrimary.store().close(); IndexShard newReplica = shards.addReplicaWithExistingPath(oldPrimary.shardPath(), oldPrimary.routingEntry().currentNodeId()); shards.recoverReplica(newReplica); @@ -1074,7 +1074,7 @@ private IndexShard failAndPromoteRandomReplica(ReplicationGroup shards) throws I IndexShard primary = shards.getPrimary(); final IndexShard newPrimary = getRandomReplica(shards); shards.promoteReplicaToPrimary(newPrimary); - primary.close("demoted", true); + primary.close("demoted", true, false); primary.store().close(); primary = shards.addReplicaWithExistingPath(primary.shardPath(), primary.routingEntry().currentNodeId()); shards.recoverReplica(primary); diff --git a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java index f5f24402c1646..fec9b04d6e371 100644 --- a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java @@ -84,7 +84,7 @@ public void setup() throws IOException { @After public void tearDown() throws Exception { - indexShard.close("test tearDown", true); + indexShard.close("test tearDown", true, false); super.tearDown(); } diff --git a/server/src/test/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerTests.java b/server/src/test/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerTests.java index 3e824c0afee25..3bf7781fb909f 100644 --- a/server/src/test/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerTests.java +++ b/server/src/test/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerTests.java @@ -51,7 +51,7 @@ public void setup() throws IOException { @After public void tearDown() throws Exception { - indexShard.close("test tearDown", true); + indexShard.close("test tearDown", true, false); super.tearDown(); } diff --git a/server/src/test/java/org/opensearch/indices/IndexingMemoryControllerTests.java b/server/src/test/java/org/opensearch/indices/IndexingMemoryControllerTests.java index 7a362ce8ded74..4a9f15f7128ad 100644 --- a/server/src/test/java/org/opensearch/indices/IndexingMemoryControllerTests.java +++ b/server/src/test/java/org/opensearch/indices/IndexingMemoryControllerTests.java @@ -369,7 +369,7 @@ public void testTranslogRecoveryWorksWithIMC() throws IOException { for (int i = 0; i < 100; i++) { indexDoc(shard, Integer.toString(i), "{\"foo\" : \"bar\"}", XContentType.JSON, null); } - shard.close("simon says", false); + shard.close("simon says", false, false); AtomicReference shardRef = new AtomicReference<>(); Settings settings = Settings.builder().put("indices.memory.index_buffer_size", "50kb").build(); Iterable iterable = () -> (shardRef.get() == null) diff --git a/server/src/test/java/org/opensearch/indices/recovery/PeerRecoveryTargetServiceTests.java b/server/src/test/java/org/opensearch/indices/recovery/PeerRecoveryTargetServiceTests.java index 730d9b4215b73..9faa8ddff8183 100644 --- a/server/src/test/java/org/opensearch/indices/recovery/PeerRecoveryTargetServiceTests.java +++ b/server/src/test/java/org/opensearch/indices/recovery/PeerRecoveryTargetServiceTests.java @@ -380,7 +380,7 @@ public void testResetStartRequestIfTranslogIsCorrupted() throws Exception { ); IndexShard shard = newStartedShard(false); final SeqNoStats seqNoStats = populateRandomData(shard); - shard.close("test", false); + shard.close("test", false, false); if (randomBoolean()) { shard.store().associateIndexWithNewTranslog(UUIDs.randomBase64UUID()); } else if (randomBoolean()) { diff --git a/server/src/test/java/org/opensearch/indices/recovery/RecoveryStatusTests.java b/server/src/test/java/org/opensearch/indices/recovery/RecoveryStatusTests.java index 73caa611dbcdb..3038d11e6ad91 100644 --- a/server/src/test/java/org/opensearch/indices/recovery/RecoveryStatusTests.java +++ b/server/src/test/java/org/opensearch/indices/recovery/RecoveryStatusTests.java @@ -94,7 +94,7 @@ public void testRenameTempFiles() throws IOException { } } assertNotNull(expectedFile); - indexShard.close("foo", false);// we have to close it here otherwise rename fails since the write.lock is held by the engine + indexShard.close("foo", false, false);// we have to close it here otherwise rename fails since the write.lock is held by the engine multiFileWriter.renameAllTempFiles(); strings = Sets.newHashSet(indexShard.store().directory().listAll()); assertTrue(strings.toString(), strings.contains("foo.bar")); diff --git a/server/src/test/java/org/opensearch/indices/recovery/RecoveryTests.java b/server/src/test/java/org/opensearch/indices/recovery/RecoveryTests.java index eae070b98c4a1..97772564acc88 100644 --- a/server/src/test/java/org/opensearch/indices/recovery/RecoveryTests.java +++ b/server/src/test/java/org/opensearch/indices/recovery/RecoveryTests.java @@ -267,7 +267,7 @@ public void testDifferentHistoryUUIDDisablesOPsRecovery() throws Exception { final String historyUUID = replica.getHistoryUUID(); Translog.TranslogGeneration translogGeneration = getTranslog(replica).getGeneration(); shards.removeReplica(replica); - replica.close("test", false); + replica.close("test", false, false); IndexWriterConfig iwc = new IndexWriterConfig(null).setCommitOnClose(false) // we don't want merges to happen here - we call maybe merge on the engine // later once we stared it up otherwise we would need to wait for it here @@ -391,7 +391,7 @@ public void testSequenceBasedRecoveryKeepsTranslog() throws Exception { if (randomBoolean()) { shards.flush(); } - replica.close("test", randomBoolean()); + replica.close("test", randomBoolean(), false); replica.store().close(); final IndexShard newReplica = shards.addReplicaWithExistingPath(replica.shardPath(), replica.routingEntry().currentNodeId()); shards.recoverReplica(newReplica); @@ -509,7 +509,7 @@ public void testRecoveryTrimsLocalTranslog() throws Exception { } shards.syncGlobalCheckpoint(); shards.promoteReplicaToPrimary(randomFrom(shards.getReplicas())).get(); - oldPrimary.close("demoted", false); + oldPrimary.close("demoted", false, false); oldPrimary.store().close(); oldPrimary = shards.addReplicaWithExistingPath(oldPrimary.shardPath(), oldPrimary.routingEntry().currentNodeId()); shards.recoverReplica(oldPrimary); diff --git a/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryRestoreTests.java b/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryRestoreTests.java index c3bd4dcaf530d..6797a1db23b2d 100644 --- a/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryRestoreTests.java +++ b/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryRestoreTests.java @@ -162,7 +162,7 @@ public void testRestoreSnapshotWithExistingFiles() throws IOException { } finally { if (shard != null && shard.state() != IndexShardState.CLOSED) { try { - shard.close("test", false); + shard.close("test", false, false); } finally { IOUtils.close(shard.store()); } @@ -228,7 +228,7 @@ public void testSnapshotWithConflictingName() throws Exception { } finally { if (shard != null && shard.state() != IndexShardState.CLOSED) { try { - shard.close("test", false); + shard.close("test", false, false); } finally { IOUtils.close(shard.store()); } diff --git a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java index 5a01fcaa1ddaf..d3d81083e72cc 100644 --- a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java @@ -806,7 +806,7 @@ protected void closeShard(IndexShard shard, boolean assertConsistencyBetweenTran EngineTestCase.assertAtMostOneLuceneDocumentPerSequenceNumber(engine); } } finally { - IOUtils.close(() -> shard.close("test", false), shard.store()); + IOUtils.close(() -> shard.close("test", false, false), shard.store()); } } From 00526f2477d229b3a7ef4bb81bd8c9a32fb7415d Mon Sep 17 00:00:00 2001 From: Varun Bansal Date: Wed, 14 Jun 2023 19:02:40 +0530 Subject: [PATCH 020/109] [Remote Store] Ensure metadata file is deleted from local fs store during metadata upload failures (#7458) * Ensure metadata file is deleted from local store during metadata upload failures or success Signed-off-by: bansvaru * move finally inside synchonized block Signed-off-by: bansvaru --------- Signed-off-by: bansvaru --- .../store/RemoteSegmentStoreDirectory.java | 65 ++++++++++++------- .../RemoteStoreRefreshListenerTests.java | 53 ++++++++++++++- 2 files changed, 94 insertions(+), 24 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java index ca6834438ed61..15c6fbea99148 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java @@ -28,6 +28,7 @@ import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadata; import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadataHandler; +import java.io.FileNotFoundException; import java.io.IOException; import java.nio.file.NoSuchFileException; import java.util.Collection; @@ -464,32 +465,50 @@ public void uploadMetadata( segmentInfosSnapshot.getGeneration(), this.commonFilenameSuffix ); - IndexOutput indexOutput = storeDirectory.createOutput(metadataFilename, IOContext.DEFAULT); - Map uploadedSegments = new HashMap<>(); - for (String file : segmentFiles) { - if (segmentsUploadedToRemoteStore.containsKey(file)) { - uploadedSegments.put(file, segmentsUploadedToRemoteStore.get(file).toString()); - } else { - throw new NoSuchFileException(file); + try { + IndexOutput indexOutput = storeDirectory.createOutput(metadataFilename, IOContext.DEFAULT); + Map uploadedSegments = new HashMap<>(); + for (String file : segmentFiles) { + if (segmentsUploadedToRemoteStore.containsKey(file)) { + uploadedSegments.put(file, segmentsUploadedToRemoteStore.get(file).toString()); + } else { + throw new NoSuchFileException(file); + } } + + ByteBuffersDataOutput byteBuffersIndexOutput = new ByteBuffersDataOutput(); + segmentInfosSnapshot.write(new ByteBuffersIndexOutput(byteBuffersIndexOutput, "Snapshot of SegmentInfos", "SegmentInfos")); + byte[] segmentInfoSnapshotByteArray = byteBuffersIndexOutput.toArrayCopy(); + + metadataStreamWrapper.writeStream( + indexOutput, + new RemoteSegmentMetadata( + RemoteSegmentMetadata.fromMapOfStrings(uploadedSegments), + segmentInfoSnapshotByteArray, + segmentInfosSnapshot.getGeneration() + ) + ); + indexOutput.close(); + storeDirectory.sync(Collections.singleton(metadataFilename)); + remoteMetadataDirectory.copyFrom(storeDirectory, metadataFilename, metadataFilename, IOContext.DEFAULT); + } finally { + tryAndDeleteLocalFile(metadataFilename, storeDirectory); } + } + } - ByteBuffersDataOutput byteBuffersIndexOutput = new ByteBuffersDataOutput(); - segmentInfosSnapshot.write(new ByteBuffersIndexOutput(byteBuffersIndexOutput, "Snapshot of SegmentInfos", "SegmentInfos")); - byte[] segmentInfoSnapshotByteArray = byteBuffersIndexOutput.toArrayCopy(); - - metadataStreamWrapper.writeStream( - indexOutput, - new RemoteSegmentMetadata( - RemoteSegmentMetadata.fromMapOfStrings(uploadedSegments), - segmentInfoSnapshotByteArray, - segmentInfosSnapshot.getGeneration() - ) - ); - indexOutput.close(); - storeDirectory.sync(Collections.singleton(metadataFilename)); - remoteMetadataDirectory.copyFrom(storeDirectory, metadataFilename, metadataFilename, IOContext.DEFAULT); - storeDirectory.deleteFile(metadataFilename); + /** + * Try to delete file from local store. Fails silently on failures + * @param filename: name of the file to be deleted + */ + private void tryAndDeleteLocalFile(String filename, Directory directory) { + try { + logger.trace("Deleting file: " + filename); + directory.deleteFile(filename); + } catch (NoSuchFileException | FileNotFoundException e) { + logger.trace("Exception while deleting. Missing file : " + filename, e); + } catch (IOException e) { + logger.warn("Exception while deleting: " + filename, e); } } 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 8dea8d272b4e8..f70eec877dbc9 100644 --- a/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java +++ b/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java @@ -40,8 +40,10 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicLong; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REPLICATION_TYPE; @@ -277,6 +279,32 @@ public void testRefreshSuccessOnSecondAttempt() throws Exception { assertEquals(1, segmentTracker.getTotalUploadsFailed()); } + /** + * Tests retry flow after snapshot and metadata files have been uploaded to remote store in the failed attempt. + * Snapshot and metadata files created in failed attempt should not break retry. + * @throws Exception + */ + public void testRefreshSuccessAfterFailureInFirstAttemptAfterSnapshotAndMetadataUpload() throws Exception { + int succeedOnAttempt = 1; + int checkpointPublishSucceedOnAttempt = 2; + // We spy on IndexShard.getReplicationTracker() to validate that we have tried running remote time as per the expectation. + CountDownLatch refreshCountLatch = new CountDownLatch(succeedOnAttempt); + // We spy on IndexShard.getEngine() to validate that we have successfully hit the terminal code for ascertaining successful upload. + // Value has been set as 6 as during a successful upload IndexShard.getEngine() is hit thrice and here we are running the flow twice + CountDownLatch successLatch = new CountDownLatch(3); + CountDownLatch reachedCheckpointPublishLatch = new CountDownLatch(0); + mockIndexShardWithRetryAndScheduleRefresh( + succeedOnAttempt, + refreshCountLatch, + successLatch, + checkpointPublishSucceedOnAttempt, + reachedCheckpointPublishLatch + ); + assertBusy(() -> assertEquals(0, refreshCountLatch.getCount())); + assertBusy(() -> assertEquals(0, successLatch.getCount())); + assertBusy(() -> assertEquals(0, reachedCheckpointPublishLatch.getCount())); + } + public void testRefreshSuccessOnThirdAttemptAttempt() throws Exception { // This covers 3 cases - 1) isRetry=false, shouldRetry=true 2) isRetry=true, shouldRetry=false 3) isRetry=True, shouldRetry=true // Succeed on 3rd attempt @@ -338,6 +366,17 @@ private Tuple m int succeedOnAttempt, CountDownLatch refreshCountLatch, CountDownLatch successLatch + ) throws IOException { + CountDownLatch noOpLatch = new CountDownLatch(0); + return mockIndexShardWithRetryAndScheduleRefresh(succeedOnAttempt, refreshCountLatch, successLatch, 1, noOpLatch); + } + + private Tuple mockIndexShardWithRetryAndScheduleRefresh( + int succeedOnAttempt, + CountDownLatch refreshCountLatch, + CountDownLatch successLatch, + int succeedCheckpointPublishOnAttempt, + CountDownLatch reachedCheckpointPublishLatch ) throws IOException { // Create index shard that we will be using to mock different methods in IndexShard for the unit test indexShard = newStartedShard( @@ -400,6 +439,18 @@ private Tuple m return indexShard.getEngine(); }).when(shard).getEngine(); + SegmentReplicationCheckpointPublisher emptyCheckpointPublisher = spy(SegmentReplicationCheckpointPublisher.EMPTY); + AtomicLong checkpointPublisherCounter = new AtomicLong(); + doAnswer(invocation -> { + if (checkpointPublisherCounter.incrementAndGet() <= succeedCheckpointPublishOnAttempt - 1) { + throw new RuntimeException("Inducing failure after snapshot info snapshot to test if snapshot info file is deleted"); + } + if (Objects.nonNull(reachedCheckpointPublishLatch)) { + reachedCheckpointPublishLatch.countDown(); + } + return null; + }).when(emptyCheckpointPublisher).publish(any(), any()); + clusterService = new ClusterService( Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), @@ -414,7 +465,7 @@ private Tuple m remoteRefreshSegmentPressureService.afterIndexShardCreated(shard); RemoteStoreRefreshListener refreshListener = new RemoteStoreRefreshListener( shard, - SegmentReplicationCheckpointPublisher.EMPTY, + emptyCheckpointPublisher, remoteRefreshSegmentPressureService.getRemoteRefreshSegmentTracker(indexShard.shardId()) ); refreshListener.afterRefresh(true); From d15ad5efd12073460fcbd2791bf3b1ee845cb2fc Mon Sep 17 00:00:00 2001 From: Ankit Kala Date: Wed, 14 Jun 2023 21:06:45 +0530 Subject: [PATCH 021/109] =?UTF-8?q?SegRep=20with=20Remote:=20Update=20comp?= =?UTF-8?q?onents=20of=20segrep=20backpressure=20to=20suppo=E2=80=A6=20(#8?= =?UTF-8?q?020)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * SegRep with Remote: Update components of segrep backpressure to support remote store. Signed-off-by: Ankit Kala * Adding tests Signed-off-by: Ankit Kala * Addressed comments Signed-off-by: Ankit Kala --------- Signed-off-by: Ankit Kala --- CHANGELOG.md | 2 +- .../index/SegmentReplicationPressureIT.java | 12 ++- .../replication/SegmentReplicationBaseIT.java | 13 +++- .../replication/SegmentReplicationIT.java | 4 - ...tReplicationWithRemoteStorePressureIT.java | 67 ++++++++++++++++ .../index/seqno/ReplicationTracker.java | 9 ++- .../OngoingSegmentReplications.java | 4 - .../SegmentReplicationSourceHandler.java | 1 - .../SegmentReplicationSourceService.java | 22 ++++++ .../SegmentReplicationTargetService.java | 77 ++++++++++++++++++- .../UpdateVisibleCheckpointRequest.java | 62 +++++++++++++++ .../main/java/org/opensearch/node/Node.java | 3 +- .../index/seqno/ReplicationTrackerTests.java | 58 ++++++++++++++ .../SegmentReplicationIndexShardTests.java | 10 ++- .../SegmentReplicationSourceServiceTests.java | 66 +++++++++++++++- .../SegmentReplicationTargetServiceTests.java | 69 ++++++++++++++++- .../snapshots/SnapshotResiliencyTests.java | 3 +- .../index/shard/IndexShardTestCase.java | 9 ++- 18 files changed, 464 insertions(+), 27 deletions(-) create mode 100644 server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationWithRemoteStorePressureIT.java create mode 100644 server/src/main/java/org/opensearch/indices/replication/UpdateVisibleCheckpointRequest.java diff --git a/CHANGELOG.md b/CHANGELOG.md index 8e7359cd93172..ec249afbc0bb5 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -101,7 +101,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Add TokenManager Interface ([#7452](https://github.com/opensearch-project/OpenSearch/pull/7452)) - Add Remote store as a segment replication source ([#7653](https://github.com/opensearch-project/OpenSearch/pull/7653)) - Add descending order search optimization through reverse segment read. ([#7967](https://github.com/opensearch-project/OpenSearch/pull/7967)) - +- Update components of segrep backpressure to support remote store. ([#8020](https://github.com/opensearch-project/OpenSearch/pull/8020)) ### Dependencies - Bump `jackson` from 2.15.1 to 2.15.2 ([#7897](https://github.com/opensearch-project/OpenSearch/pull/7897)) diff --git a/server/src/internalClusterTest/java/org/opensearch/index/SegmentReplicationPressureIT.java b/server/src/internalClusterTest/java/org/opensearch/index/SegmentReplicationPressureIT.java index c04738dce8e8e..7b8c6a7feb1fd 100644 --- a/server/src/internalClusterTest/java/org/opensearch/index/SegmentReplicationPressureIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/index/SegmentReplicationPressureIT.java @@ -123,9 +123,9 @@ public void testWritesRejected() throws Exception { assertEquals(perGroupStats.getRejectedRequestCount(), 2L); } refresh(INDEX_NAME); - // wait for the replicas to catch up after block is released. - waitForSearchableDocs(totalDocs.get(), replicaNodes.toArray(new String[] {})); + // wait for the replicas to catch up after block is released. + assertReplicaCheckpointUpdated(primaryShard); // index another doc showing there is no pressure enforced. indexDoc(); refresh(INDEX_NAME); @@ -179,7 +179,7 @@ public void testAddReplicaWhileWritesBlocked() throws Exception { } refresh(INDEX_NAME); // wait for the replicas to catch up after block is released. - waitForSearchableDocs(totalDocs.get(), replicaNodes.toArray(new String[] {})); + assertReplicaCheckpointUpdated(primaryShard); // index another doc showing there is no pressure enforced. indexDoc(); @@ -258,6 +258,10 @@ public void testFailStaleReplica() throws Exception { } public void testWithDocumentReplicationEnabledIndex() throws Exception { + assumeTrue( + "Can't create DocRep index with remote store enabled. Skipping.", + indexSettings().getAsBoolean(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, false) == false + ); Settings settings = Settings.builder().put(MAX_REPLICATION_TIME_SETTING.getKey(), TimeValue.timeValueMillis(500)).build(); // Starts a primary and replica node. final String primaryNode = internalCluster().startNode(settings); @@ -313,7 +317,7 @@ public void testBulkWritesRejected() throws Exception { } refresh(INDEX_NAME); // wait for the replicas to catch up after block is released. - waitForSearchableDocs(totalDocs, replicaNodes.toArray(new String[] {})); + assertReplicaCheckpointUpdated(primaryShard); // index another doc showing there is no pressure enforced. executeBulkRequest(nodes, totalDocs); diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationBaseIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationBaseIT.java index f88b9f2777da4..385f54dd4fd5e 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationBaseIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationBaseIT.java @@ -23,6 +23,7 @@ import org.opensearch.index.IndexModule; import org.opensearch.index.IndexService; import org.opensearch.index.SegmentReplicationPerGroupStats; +import org.opensearch.index.SegmentReplicationShardStats; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.store.Store; import org.opensearch.index.store.StoreFileMetadata; @@ -38,6 +39,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -204,7 +206,7 @@ protected Releasable blockReplication(List nodes, CountDownLatch latch) node )); mockTargetTransportService.addSendBehavior((connection, requestId, action, request, options) -> { - if (action.equals(SegmentReplicationSourceService.Actions.GET_SEGMENT_FILES)) { + if (action.equals(SegmentReplicationSourceService.Actions.UPDATE_VISIBLE_CHECKPOINT)) { try { latch.countDown(); pauseReplicationLatch.await(); @@ -222,4 +224,13 @@ protected Releasable blockReplication(List nodes, CountDownLatch latch) }; } + protected void assertReplicaCheckpointUpdated(IndexShard primaryShard) throws Exception { + assertBusy(() -> { + Set groupStats = primaryShard.getReplicationStats(); + assertEquals(primaryShard.indexSettings().getNumberOfReplicas(), groupStats.size()); + for (SegmentReplicationShardStats shardStat : groupStats) { + assertEquals(0, shardStat.getCheckpointsBehindCount()); + } + }, 30, TimeUnit.SECONDS); + } } diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationIT.java index 1daac295630ff..153a7802a8f0e 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationIT.java @@ -796,10 +796,6 @@ public void testReplicaHasDiffFilesThanPrimary() throws Exception { } public void testPressureServiceStats() throws Exception { - assumeFalse( - "Skipping the test as pressure service is not compatible with SegRep and Remote store yet.", - segmentReplicationWithRemoteEnabled() - ); final String primaryNode = internalCluster().startNode(); createIndex(INDEX_NAME); final String replicaNode = internalCluster().startNode(); diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationWithRemoteStorePressureIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationWithRemoteStorePressureIT.java new file mode 100644 index 0000000000000..e7c9c811b7734 --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationWithRemoteStorePressureIT.java @@ -0,0 +1,67 @@ +/* + * 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.remotestore; + +import org.junit.After; +import org.junit.Before; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.FeatureFlags; +import org.opensearch.index.SegmentReplicationPressureIT; +import org.opensearch.indices.replication.common.ReplicationType; +import org.opensearch.test.OpenSearchIntegTestCase; + +import java.nio.file.Path; + +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; + +/** + * This class executes the SegmentReplicationPressureIT suite with remote store integration enabled. + * Setup is similar to SegmentReplicationPressureIT but this also enables the segment replication using remote store which + * is behind SEGMENT_REPLICATION_EXPERIMENTAL flag. + */ +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) +public class SegmentReplicationWithRemoteStorePressureIT extends SegmentReplicationPressureIT { + + private static final String REPOSITORY_NAME = "test-remote-store-repo"; + + @Override + public Settings indexSettings() { + return Settings.builder() + .put(super.indexSettings()) + .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, true) + .put(IndexMetadata.SETTING_REMOTE_STORE_REPOSITORY, REPOSITORY_NAME) + .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_ENABLED, false) + .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) + .build(); + } + + @Override + protected Settings featureFlagSettings() { + return Settings.builder() + .put(super.featureFlagSettings()) + .put(FeatureFlags.REMOTE_STORE, "true") + .put(FeatureFlags.SEGMENT_REPLICATION_EXPERIMENTAL, "true") + .build(); + } + + @Before + public void setup() { + internalCluster().startClusterManagerOnlyNode(); + Path absolutePath = randomRepoPath().toAbsolutePath(); + assertAcked( + clusterAdmin().preparePutRepository(REPOSITORY_NAME).setType("fs").setSettings(Settings.builder().put("location", absolutePath)) + ); + } + + @After + public void teardown() { + assertAcked(clusterAdmin().prepareDeleteRepository(REPOSITORY_NAME)); + } +} diff --git a/server/src/main/java/org/opensearch/index/seqno/ReplicationTracker.java b/server/src/main/java/org/opensearch/index/seqno/ReplicationTracker.java index e714de9814f1c..38288af4a57b6 100644 --- a/server/src/main/java/org/opensearch/index/seqno/ReplicationTracker.java +++ b/server/src/main/java/org/opensearch/index/seqno/ReplicationTracker.java @@ -1174,13 +1174,18 @@ public synchronized void updateVisibleCheckpointForShard(final String allocation assert handoffInProgress == false; assert invariant(); final CheckpointState cps = checkpoints.get(allocationId); - assert !this.shardAllocationId.equals(allocationId) && cps != null; + assert !this.shardAllocationId.equals(allocationId); + // Ignore if the cps is null (i.e. replica shard not in active state). + if (cps == null) { + logger.warn("Ignoring the checkpoint update for allocation ID {} as its not being tracked by primary", allocationId); + return; + } if (cps.checkpointTimers.isEmpty() == false) { // stop any timers for checkpoints up to the received cp and remove from cps.checkpointTimers. // Compute the max lag from the set of completed timers. final AtomicLong lastFinished = new AtomicLong(0L); cps.checkpointTimers.entrySet().removeIf((entry) -> { - boolean result = visibleCheckpoint.equals(entry.getKey()) || visibleCheckpoint.isAheadOf(entry.getKey()); + boolean result = entry.getKey().isAheadOf(visibleCheckpoint) == false; if (result) { final ReplicationTimer timer = entry.getValue(); timer.stop(); diff --git a/server/src/main/java/org/opensearch/indices/replication/OngoingSegmentReplications.java b/server/src/main/java/org/opensearch/indices/replication/OngoingSegmentReplications.java index 6f04c6cf6f665..050a66bedcf5d 100644 --- a/server/src/main/java/org/opensearch/indices/replication/OngoingSegmentReplications.java +++ b/server/src/main/java/org/opensearch/indices/replication/OngoingSegmentReplications.java @@ -121,10 +121,6 @@ void startSegmentCopy(GetSegmentFilesRequest request, ActionListener { try { - shard.updateVisibleCheckpointForShard(allocationId, copyState.getCheckpoint()); future.onResponse(new GetSegmentFilesResponse(List.of(storeFileMetadata))); } finally { IOUtils.close(resources); diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSourceService.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSourceService.java index 0e62a4320e3f3..79186deeeaf0f 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSourceService.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationSourceService.java @@ -34,6 +34,7 @@ import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportChannel; import org.opensearch.transport.TransportRequestHandler; +import org.opensearch.transport.TransportResponse; import org.opensearch.transport.TransportService; import java.io.IOException; @@ -63,6 +64,7 @@ public static class Actions { public static final String GET_CHECKPOINT_INFO = "internal:index/shard/replication/get_checkpoint_info"; public static final String GET_SEGMENT_FILES = "internal:index/shard/replication/get_segment_files"; + public static final String UPDATE_VISIBLE_CHECKPOINT = "internal:index/shard/replication/update_visible_checkpoint"; } private final OngoingSegmentReplications ongoingSegmentReplications; @@ -89,6 +91,12 @@ protected SegmentReplicationSourceService( GetSegmentFilesRequest::new, new GetSegmentFilesRequestHandler() ); + transportService.registerRequestHandler( + Actions.UPDATE_VISIBLE_CHECKPOINT, + ThreadPool.Names.GENERIC, + UpdateVisibleCheckpointRequest::new, + new UpdateVisibleCheckpointRequestHandler() + ); } public SegmentReplicationSourceService( @@ -142,6 +150,20 @@ public void messageReceived(GetSegmentFilesRequest request, TransportChannel cha } } + private class UpdateVisibleCheckpointRequestHandler implements TransportRequestHandler { + @Override + public void messageReceived(UpdateVisibleCheckpointRequest request, TransportChannel channel, Task task) throws Exception { + try { + IndexService indexService = indicesService.indexServiceSafe(request.getPrimaryShardId().getIndex()); + IndexShard indexShard = indexService.getShard(request.getPrimaryShardId().id()); + indexShard.updateVisibleCheckpointForShard(request.getTargetAllocationId(), request.getCheckpoint()); + channel.sendResponse(TransportResponse.Empty.INSTANCE); + } catch (Exception e) { + channel.sendResponse(e); + } + } + } + @Override public void clusterChanged(ClusterChangedEvent event) { if (event.nodesRemoved()) { 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 1ce208a9a8234..2931caa353bc6 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTargetService.java @@ -13,7 +13,9 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.BaseExceptionsHelper; import org.opensearch.action.ActionListener; +import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.Nullable; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.CancellableThreads; @@ -26,6 +28,7 @@ import org.opensearch.indices.recovery.FileChunkRequest; import org.opensearch.indices.recovery.ForceSyncRequest; import org.opensearch.indices.recovery.RecoverySettings; +import org.opensearch.indices.recovery.RetryableTransportClient; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.indices.replication.common.ReplicationCollection; import org.opensearch.indices.replication.common.ReplicationCollection.ReplicationRef; @@ -36,6 +39,7 @@ import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportChannel; import org.opensearch.transport.TransportRequestHandler; +import org.opensearch.transport.TransportRequestOptions; import org.opensearch.transport.TransportResponse; import org.opensearch.transport.TransportService; @@ -45,6 +49,8 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicLong; +import static org.opensearch.indices.replication.SegmentReplicationSourceService.Actions.UPDATE_VISIBLE_CHECKPOINT; + /** * Service class that orchestrates replication events on replicas. * @@ -66,6 +72,8 @@ public class SegmentReplicationTargetService implements IndexEventListener { protected final Map latestReceivedCheckpoint = ConcurrentCollections.newConcurrentMap(); private final IndicesService indicesService; + private final ClusterService clusterService; + private final TransportService transportService; public ReplicationRef get(long replicationId) { return onGoingReplications.get(replicationId); @@ -86,7 +94,8 @@ public SegmentReplicationTargetService( final RecoverySettings recoverySettings, final TransportService transportService, final SegmentReplicationSourceFactory sourceFactory, - final IndicesService indicesService + final IndicesService indicesService, + final ClusterService clusterService ) { this( threadPool, @@ -94,6 +103,7 @@ public SegmentReplicationTargetService( transportService, sourceFactory, indicesService, + clusterService, new ReplicationCollection<>(logger, threadPool) ); } @@ -104,6 +114,7 @@ public SegmentReplicationTargetService( final TransportService transportService, final SegmentReplicationSourceFactory sourceFactory, final IndicesService indicesService, + final ClusterService clusterService, final ReplicationCollection ongoingSegmentReplications ) { this.threadPool = threadPool; @@ -111,6 +122,8 @@ public SegmentReplicationTargetService( this.onGoingReplications = ongoingSegmentReplications; this.sourceFactory = sourceFactory; this.indicesService = indicesService; + this.clusterService = clusterService; + this.transportService = transportService; transportService.registerRequestHandler( Actions.FILE_CHUNK, @@ -240,6 +253,10 @@ public void onReplicationDone(SegmentReplicationState state) { state.getTimingData() ) ); + + // update visible checkpoint to primary + updateVisibleCheckpoint(state.getReplicationId(), replicaShard); + // if we received a checkpoint during the copy event that is ahead of this // try and process it. processLatestReceivedCheckpoint(replicaShard, thread); @@ -274,6 +291,61 @@ public void onReplicationFailure( } } + protected void updateVisibleCheckpoint(long replicationId, IndexShard replicaShard) { + ShardRouting primaryShard = clusterService.state().routingTable().shardRoutingTable(replicaShard.shardId()).primaryShard(); + + final UpdateVisibleCheckpointRequest request = new UpdateVisibleCheckpointRequest( + replicationId, + replicaShard.routingEntry().allocationId().getId(), + primaryShard.shardId(), + getPrimaryNode(primaryShard), + replicaShard.getLatestReplicationCheckpoint() + ); + + final TransportRequestOptions options = TransportRequestOptions.builder() + .withTimeout(recoverySettings.internalActionTimeout()) + .build(); + logger.debug("Updating replication checkpoint to {}", request.getCheckpoint()); + RetryableTransportClient transportClient = new RetryableTransportClient( + transportService, + getPrimaryNode(primaryShard), + recoverySettings.internalActionRetryTimeout(), + logger + ); + final ActionListener listener = new ActionListener<>() { + @Override + public void onResponse(Void unused) { + logger.debug( + "Successfully updated replication checkpoint {} for replica {}", + replicaShard.shardId(), + request.getCheckpoint() + ); + } + + @Override + public void onFailure(Exception e) { + logger.error( + "Failed to update visible checkpoint for replica {}, {}: {}", + replicaShard.shardId(), + request.getCheckpoint(), + e + ); + } + }; + + transportClient.executeRetryableAction( + UPDATE_VISIBLE_CHECKPOINT, + request, + options, + ActionListener.map(listener, r -> null), + in -> TransportResponse.Empty.INSTANCE + ); + } + + private DiscoveryNode getPrimaryNode(ShardRouting primaryShard) { + return clusterService.state().nodes().get(primaryShard.currentNodeId()); + } + // visible to tests protected boolean processLatestReceivedCheckpoint(IndexShard replicaShard, Thread thread) { final ReplicationCheckpoint latestPublishedCheckpoint = latestReceivedCheckpoint.get(replicaShard.shardId()); @@ -436,6 +508,9 @@ public void onReplicationDone(SegmentReplicationState state) { // Promote engine type for primary target if (indexShard.recoveryState().getPrimary() == true) { indexShard.resetToWriteableEngine(); + } else { + // Update the replica's checkpoint on primary's replication tracker. + updateVisibleCheckpoint(state.getReplicationId(), indexShard); } channel.sendResponse(TransportResponse.Empty.INSTANCE); } catch (InterruptedException | TimeoutException | IOException e) { diff --git a/server/src/main/java/org/opensearch/indices/replication/UpdateVisibleCheckpointRequest.java b/server/src/main/java/org/opensearch/indices/replication/UpdateVisibleCheckpointRequest.java new file mode 100644 index 0000000000000..2674adf711406 --- /dev/null +++ b/server/src/main/java/org/opensearch/indices/replication/UpdateVisibleCheckpointRequest.java @@ -0,0 +1,62 @@ +/* + * 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.indices.replication; + +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.common.io.stream.StreamOutput; +import org.opensearch.index.shard.ShardId; +import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; +import org.opensearch.indices.replication.common.SegmentReplicationTransportRequest; + +import java.io.IOException; + +/** + * Request object for updating the replica's checkpoint on primary for tracking. + * + * @opensearch.internal + */ +public class UpdateVisibleCheckpointRequest extends SegmentReplicationTransportRequest { + + private final ReplicationCheckpoint checkpoint; + private final ShardId primaryShardId; + + public UpdateVisibleCheckpointRequest(StreamInput in) throws IOException { + super(in); + checkpoint = new ReplicationCheckpoint(in); + primaryShardId = new ShardId(in); + } + + public UpdateVisibleCheckpointRequest( + long replicationId, + String targetAllocationId, + ShardId primaryShardId, + DiscoveryNode targetNode, + ReplicationCheckpoint checkpoint + ) { + super(replicationId, targetAllocationId, targetNode); + this.checkpoint = checkpoint; + this.primaryShardId = primaryShardId; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + checkpoint.writeTo(out); + primaryShardId.writeTo(out); + } + + public ReplicationCheckpoint getCheckpoint() { + return checkpoint; + } + + public ShardId getPrimaryShardId() { + return primaryShardId; + } +} diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index eb1fc2008df06..0660341dd92f8 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -1103,7 +1103,8 @@ protected Node( recoverySettings, transportService, new SegmentReplicationSourceFactory(transportService, recoverySettings, clusterService), - indicesService + indicesService, + clusterService ) ); b.bind(SegmentReplicationSourceService.class) diff --git a/server/src/test/java/org/opensearch/index/seqno/ReplicationTrackerTests.java b/server/src/test/java/org/opensearch/index/seqno/ReplicationTrackerTests.java index 7cfc95d7f5cff..d26d652451f35 100644 --- a/server/src/test/java/org/opensearch/index/seqno/ReplicationTrackerTests.java +++ b/server/src/test/java/org/opensearch/index/seqno/ReplicationTrackerTests.java @@ -1868,6 +1868,64 @@ public void testSegmentReplicationCheckpointTracking() { } } + public void testSegmentReplicationCheckpointTrackingInvalidAllocationIDs() { + Settings settings = Settings.builder().put(SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT).build(); + final long initialClusterStateVersion = randomNonNegativeLong(); + final int numberOfActiveAllocationsIds = randomIntBetween(2, 16); + final int numberOfInitializingIds = randomIntBetween(2, 16); + final Tuple, Set> activeAndInitializingAllocationIds = randomActiveAndInitializingAllocationIds( + numberOfActiveAllocationsIds, + numberOfInitializingIds + ); + final Set activeAllocationIds = activeAndInitializingAllocationIds.v1(); + final Set initializingIds = activeAndInitializingAllocationIds.v2(); + AllocationId primaryId = activeAllocationIds.iterator().next(); + IndexShardRoutingTable routingTable = routingTable(initializingIds, primaryId); + final ReplicationTracker tracker = newTracker(primaryId, settings); + + tracker.updateFromClusterManager(initialClusterStateVersion, ids(activeAllocationIds), routingTable); + tracker.activatePrimaryMode(NO_OPS_PERFORMED); + assertThat(tracker.getReplicationGroup().getInSyncAllocationIds(), equalTo(ids(activeAllocationIds))); + assertThat(tracker.getReplicationGroup().getRoutingTable(), equalTo(routingTable)); + assertTrue(activeAllocationIds.stream().allMatch(a -> tracker.getTrackedLocalCheckpointForShard(a.getId()).inSync)); + + // get insync ids, filter out the primary. + final Set inSyncAllocationIds = tracker.getReplicationGroup() + .getInSyncAllocationIds() + .stream() + .filter(id -> tracker.shardAllocationId.equals(id) == false) + .collect(Collectors.toSet()); + + final ReplicationCheckpoint initialCheckpoint = new ReplicationCheckpoint( + tracker.shardId(), + 0L, + 1, + 1, + 1L, + Codec.getDefault().getName() + ); + tracker.setLatestReplicationCheckpoint(initialCheckpoint); + + Set groupStats = tracker.getSegmentReplicationStats(); + assertEquals(inSyncAllocationIds.size(), groupStats.size()); + for (SegmentReplicationShardStats shardStat : groupStats) { + assertEquals(1, shardStat.getCheckpointsBehindCount()); + } + + // simulate replicas moved up to date. + final Map checkpoints = tracker.checkpoints; + for (String id : inSyncAllocationIds) { + final ReplicationTracker.CheckpointState checkpointState = checkpoints.get(id); + assertEquals(1, checkpointState.checkpointTimers.size()); + tracker.updateVisibleCheckpointForShard(id, initialCheckpoint); + assertEquals(0, checkpointState.checkpointTimers.size()); + } + + // Unknown allocation ID will be ignored. + tracker.updateVisibleCheckpointForShard("randomAllocationID", initialCheckpoint); + assertThrows(AssertionError.class, () -> tracker.updateVisibleCheckpointForShard(tracker.shardAllocationId, initialCheckpoint)); + } + public void testPrimaryContextHandoffWithRemoteTranslogEnabled() throws IOException { Settings settings = Settings.builder().put("index.remote_store.translog.enabled", "true").build(); final IndexSettings indexSettings = IndexSettingsModule.newIndexSettings("test", settings); diff --git a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java index aba04e9d30159..8c0e7be30ad19 100644 --- a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java @@ -19,6 +19,7 @@ import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.ShardRoutingHelper; +import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.collect.Tuple; import org.opensearch.common.concurrent.GatedCloseable; import org.opensearch.common.settings.ClusterSettings; @@ -295,7 +296,13 @@ public void testPublishCheckpointAfterRelocationHandOff() throws IOException { public void testRejectCheckpointOnShardRoutingPrimary() throws IOException { IndexShard primaryShard = newStartedShard(true); SegmentReplicationTargetService sut; - sut = prepareForReplication(primaryShard, null, mock(TransportService.class), mock(IndicesService.class)); + sut = prepareForReplication( + primaryShard, + null, + mock(TransportService.class), + mock(IndicesService.class), + mock(ClusterService.class) + ); SegmentReplicationTargetService spy = spy(sut); // Starting a new shard in PrimaryMode and shard routing primary. @@ -1011,6 +1018,7 @@ private SegmentReplicationTargetService newTargetService(SegmentReplicationSourc new RecoverySettings(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)), mock(TransportService.class), sourceFactory, + null, null ); } diff --git a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationSourceServiceTests.java b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationSourceServiceTests.java index 76481ebbecea3..1ceba05f16369 100644 --- a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationSourceServiceTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationSourceServiceTests.java @@ -14,6 +14,7 @@ import org.opensearch.cluster.ClusterChangedEvent; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; @@ -33,6 +34,7 @@ import org.opensearch.threadpool.TestThreadPool; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportException; +import org.opensearch.transport.TransportResponse; import org.opensearch.transport.TransportResponseHandler; import org.opensearch.transport.TransportService; @@ -57,12 +59,13 @@ public class SegmentReplicationSourceServiceTests extends OpenSearchTestCase { private DiscoveryNode localNode; private SegmentReplicationSourceService segmentReplicationSourceService; private OngoingSegmentReplications ongoingSegmentReplications; + private IndexShard mockIndexShard; @Override public void setUp() throws Exception { super.setUp(); // setup mocks - IndexShard mockIndexShard = CopyStateTests.createMockIndexShard(); + mockIndexShard = CopyStateTests.createMockIndexShard(); ShardId testShardId = mockIndexShard.shardId(); IndicesService mockIndicesService = mock(IndicesService.class); IndexService mockIndexService = mock(IndexService.class); @@ -145,6 +148,27 @@ public void onFailure(Exception e) { }); } + public void testUpdateVisibleCheckpoint() { + UpdateVisibleCheckpointRequest request = new UpdateVisibleCheckpointRequest( + 0L, + "", + mockIndexShard.shardId(), + localNode, + testCheckpoint + ); + executeUpdateVisibleCheckpoint(request, new ActionListener<>() { + @Override + public void onResponse(TransportResponse transportResponse) { + assertTrue(TransportResponse.Empty.INSTANCE.equals(transportResponse)); + } + + @Override + public void onFailure(Exception e) { + fail("unexpected exception: " + e); + } + }); + } + public void testCheckpointInfo() { executeGetCheckpointInfo(new ActionListener<>() { @Override @@ -226,4 +250,44 @@ public GetSegmentFilesResponse read(StreamInput in) throws IOException { } ); } + + private void executeUpdateVisibleCheckpoint(UpdateVisibleCheckpointRequest request, ActionListener listener) { + try (BytesStreamOutput out = new BytesStreamOutput()) { + request.writeTo(out); + try (StreamInput in = out.bytes().streamInput()) { + UpdateVisibleCheckpointRequest newRequest = new UpdateVisibleCheckpointRequest(in); + assertTrue(newRequest.getCheckpoint().equals(request.getCheckpoint())); + assertTrue(newRequest.getTargetAllocationId().equals(request.getTargetAllocationId())); + } + } catch (IOException e) { + fail("Failed to parse UpdateVisibleCheckpointRequest " + e); + } + + transportService.sendRequest( + localNode, + SegmentReplicationSourceService.Actions.UPDATE_VISIBLE_CHECKPOINT, + request, + new TransportResponseHandler<>() { + @Override + public void handleResponse(TransportResponse response) { + listener.onResponse(TransportResponse.Empty.INSTANCE); + } + + @Override + public void handleException(TransportException e) { + listener.onFailure(e); + } + + @Override + public String executor() { + return ThreadPool.Names.SAME; + } + + @Override + public CheckpointInfoResponse read(StreamInput in) throws IOException { + return new CheckpointInfoResponse(in); + } + } + ); + } } diff --git a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetServiceTests.java b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetServiceTests.java index 9c796ec05c22a..c632f2843cba2 100644 --- a/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetServiceTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/SegmentReplicationTargetServiceTests.java @@ -13,8 +13,12 @@ import org.opensearch.OpenSearchException; import org.opensearch.Version; import org.opensearch.action.ActionListener; +import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.cluster.node.DiscoveryNodes; +import org.opensearch.cluster.routing.RoutingTable; +import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.CancellableThreads; @@ -29,6 +33,7 @@ import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.indices.replication.common.ReplicationCollection; import org.opensearch.indices.replication.common.ReplicationFailedException; +import org.opensearch.indices.replication.common.ReplicationLuceneIndex; import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.threadpool.TestThreadPool; import org.opensearch.threadpool.ThreadPool; @@ -44,12 +49,14 @@ import java.util.concurrent.TimeUnit; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.atLeastOnce; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -75,6 +82,9 @@ public class SegmentReplicationTargetServiceTests extends IndexShardTestCase { private DiscoveryNode localNode; private IndicesService indicesService; + private ClusterService clusterService; + + private SegmentReplicationState state; private static long TRANSPORT_TIMEOUT = 30000;// 30sec @@ -115,8 +125,15 @@ public void setUp() throws Exception { transportService.acceptIncomingRequests(); indicesService = mock(IndicesService.class); - - sut = prepareForReplication(primaryShard, null, transportService, indicesService); + clusterService = mock(ClusterService.class); + ClusterState clusterState = mock(ClusterState.class); + RoutingTable mockRoutingTable = mock(RoutingTable.class); + when(clusterService.state()).thenReturn(clusterState); + when(clusterState.routingTable()).thenReturn(mockRoutingTable); + when(mockRoutingTable.shardRoutingTable(any())).thenReturn(primaryShard.getReplicationGroup().getRoutingTable()); + + when(clusterState.nodes()).thenReturn(DiscoveryNodes.builder().add(localNode).build()); + sut = prepareForReplication(primaryShard, replicaShard, transportService, indicesService, clusterService); initialCheckpoint = replicaShard.getLatestReplicationCheckpoint(); aheadCheckpoint = new ReplicationCheckpoint( initialCheckpoint.getShardId(), @@ -132,6 +149,14 @@ public void setUp() throws Exception { initialCheckpoint.getSegmentInfosVersion() + 1, primaryCodec ); + + state = new SegmentReplicationState( + replicaShard.routingEntry(), + new ReplicationLuceneIndex(), + 0L, + "", + new DiscoveryNode("local", buildNewFakeTransportAddress(), Version.CURRENT) + ); } @Override @@ -330,6 +355,42 @@ public void testAfterIndexShardStartedProcessesLatestReceivedCheckpoint() { verify(spy, times(1)).processLatestReceivedCheckpoint(eq(replicaShard), any()); } + public void testStartReplicationListenerSuccess() throws InterruptedException { + sut.updateLatestReceivedCheckpoint(aheadCheckpoint, replicaShard); + SegmentReplicationTargetService spy = spy(sut); + CountDownLatch latch = new CountDownLatch(1); + doAnswer(i -> { + ((SegmentReplicationTargetService.SegmentReplicationListener) i.getArgument(1)).onReplicationDone(state); + latch.countDown(); + return null; + }).when(spy).startReplication(any(), any()); + doNothing().when(spy).updateVisibleCheckpoint(eq(0L), any()); + spy.afterIndexShardStarted(replicaShard); + + latch.await(2, TimeUnit.SECONDS); + verify(spy, (atLeastOnce())).updateVisibleCheckpoint(eq(0L), eq(replicaShard)); + } + + public void testStartReplicationListenerFailure() throws InterruptedException { + sut.updateLatestReceivedCheckpoint(aheadCheckpoint, replicaShard); + SegmentReplicationTargetService spy = spy(sut); + CountDownLatch latch = new CountDownLatch(1); + doAnswer(i -> { + ((SegmentReplicationTargetService.SegmentReplicationListener) i.getArgument(1)).onReplicationFailure( + state, + new ReplicationFailedException(replicaShard, null), + false + ); + latch.countDown(); + return null; + }).when(spy).startReplication(any(), any()); + doNothing().when(spy).updateVisibleCheckpoint(eq(0L), any()); + spy.afterIndexShardStarted(replicaShard); + + latch.await(2, TimeUnit.SECONDS); + verify(spy, (never())).updateVisibleCheckpoint(eq(0L), eq(replicaShard)); + } + public void testDoNotProcessLatestCheckpointIfItIsbehind() { sut.updateLatestReceivedCheckpoint(replicaShard.getLatestReplicationCheckpoint(), replicaShard); assertFalse(sut.processLatestReceivedCheckpoint(replicaShard, null)); @@ -345,6 +406,7 @@ public void testOnNewCheckpointInvokedOnClosedShardDoesNothing() throws IOExcept public void testBeforeIndexShardClosed_DoesNothingForDocRepIndex() throws IOException { final SegmentReplicationSourceFactory sourceFactory = mock(SegmentReplicationSourceFactory.class); final IndicesService indicesService = mock(IndicesService.class); + final ClusterService clusterService = mock(ClusterService.class); final ReplicationCollection ongoingReplications = mock(ReplicationCollection.class); final SegmentReplicationTargetService targetService = new SegmentReplicationTargetService( threadPool, @@ -352,6 +414,7 @@ public void testBeforeIndexShardClosed_DoesNothingForDocRepIndex() throws IOExce mock(TransportService.class), sourceFactory, indicesService, + clusterService, ongoingReplications ); final Settings settings = Settings.builder().put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.DOCUMENT).build(); @@ -364,6 +427,7 @@ public void testBeforeIndexShardClosed_DoesNothingForDocRepIndex() throws IOExce public void testShardRoutingChanged_DoesNothingForDocRepIndex() throws IOException { final SegmentReplicationSourceFactory sourceFactory = mock(SegmentReplicationSourceFactory.class); final IndicesService indicesService = mock(IndicesService.class); + final ClusterService clusterService = mock(ClusterService.class); final ReplicationCollection ongoingReplications = mock(ReplicationCollection.class); final SegmentReplicationTargetService targetService = new SegmentReplicationTargetService( threadPool, @@ -371,6 +435,7 @@ public void testShardRoutingChanged_DoesNothingForDocRepIndex() throws IOExcepti mock(TransportService.class), sourceFactory, indicesService, + clusterService, ongoingReplications ); final Settings settings = Settings.builder().put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.DOCUMENT).build(); diff --git a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java index efa1db17d0461..0bb2b604e8f1a 100644 --- a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java @@ -1867,7 +1867,8 @@ public void onFailure(final Exception e) { recoverySettings, transportService, new SegmentReplicationSourceFactory(transportService, recoverySettings, clusterService), - indicesService + indicesService, + clusterService ), mock(SegmentReplicationSourceService.class), shardStateAction, diff --git a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java index d3d81083e72cc..22b6693ee09ec 100644 --- a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java @@ -1302,7 +1302,8 @@ public final SegmentReplicationTargetService prepareForReplication( IndexShard primaryShard, IndexShard target, TransportService transportService, - IndicesService indicesService + IndicesService indicesService, + ClusterService clusterService ) { final SegmentReplicationSourceFactory sourceFactory = mock(SegmentReplicationSourceFactory.class); final SegmentReplicationTargetService targetService = new SegmentReplicationTargetService( @@ -1310,7 +1311,8 @@ public final SegmentReplicationTargetService prepareForReplication( new RecoverySettings(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)), transportService, sourceFactory, - indicesService + indicesService, + clusterService ); final SegmentReplicationSource replicationSource = new TestReplicationSource() { @Override @@ -1379,7 +1381,8 @@ public final List replicateSegments(IndexShard primary primaryShard, replica, mock(TransportService.class), - mock(IndicesService.class) + mock(IndicesService.class), + mock(ClusterService.class) ); final SegmentReplicationTarget target = targetService.startReplication( replica, From cc0f344b1f34e56a203c212ab74f92fa8093e38e Mon Sep 17 00:00:00 2001 From: Sorabh Date: Wed, 14 Jun 2023 10:03:21 -0700 Subject: [PATCH 022/109] Rename QueryPhase actors like Suggest, Rescore to be processors rather than phase (#8025) Signed-off-by: Sorabh Hamirwasia --- CHANGELOG.md | 1 + .../opensearch/search/query/QueryPhase.java | 18 +++++++++--------- ...RescorePhase.java => RescoreProcessor.java} | 6 +++--- ...SuggestPhase.java => SuggestProcessor.java} | 6 +++--- 4 files changed, 16 insertions(+), 15 deletions(-) rename server/src/main/java/org/opensearch/search/rescore/{RescorePhase.java => RescoreProcessor.java} (93%) rename server/src/main/java/org/opensearch/search/suggest/{SuggestPhase.java => SuggestProcessor.java} (95%) diff --git a/CHANGELOG.md b/CHANGELOG.md index ec249afbc0bb5..05d89358b993d 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -66,6 +66,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Change `com.amazonaws.sdk.ec2MetadataServiceEndpointOverride` to `aws.ec2MetadataServiceEndpoint` ([7372](https://github.com/opensearch-project/OpenSearch/pull/7372/)) - Change `com.amazonaws.sdk.stsEndpointOverride` to `aws.stsEndpointOverride` ([7372](https://github.com/opensearch-project/OpenSearch/pull/7372/)) - Align range and default value for deletes_pct_allowed in merge policy ([#7730](https://github.com/opensearch-project/OpenSearch/pull/7730)) +- Rename QueryPhase actors like Suggest, Rescore to be processors rather than phase ([#8025](https://github.com/opensearch-project/OpenSearch/pull/8025)) ### Deprecated diff --git a/server/src/main/java/org/opensearch/search/query/QueryPhase.java b/server/src/main/java/org/opensearch/search/query/QueryPhase.java index 0add9123bc596..06be6683b5e4c 100644 --- a/server/src/main/java/org/opensearch/search/query/QueryPhase.java +++ b/server/src/main/java/org/opensearch/search/query/QueryPhase.java @@ -63,9 +63,9 @@ import org.opensearch.search.profile.ProfileShardResult; import org.opensearch.search.profile.SearchProfileShardResults; import org.opensearch.search.profile.query.InternalProfileCollector; -import org.opensearch.search.rescore.RescorePhase; +import org.opensearch.search.rescore.RescoreProcessor; import org.opensearch.search.sort.SortAndFormats; -import org.opensearch.search.suggest.SuggestPhase; +import org.opensearch.search.suggest.SuggestProcessor; import org.opensearch.tasks.TaskCancelledException; import org.opensearch.threadpool.ThreadPool; @@ -94,8 +94,8 @@ public class QueryPhase { public static final boolean SYS_PROP_REWRITE_SORT = Booleans.parseBoolean(System.getProperty("opensearch.search.rewrite_sort", "true")); public static final QueryPhaseSearcher DEFAULT_QUERY_PHASE_SEARCHER = new DefaultQueryPhaseSearcher(); private final QueryPhaseSearcher queryPhaseSearcher; - private final SuggestPhase suggestPhase; - private final RescorePhase rescorePhase; + private final SuggestProcessor suggestProcessor; + private final RescoreProcessor rescoreProcessor; public QueryPhase() { this(DEFAULT_QUERY_PHASE_SEARCHER); @@ -103,8 +103,8 @@ public QueryPhase() { public QueryPhase(QueryPhaseSearcher queryPhaseSearcher) { this.queryPhaseSearcher = Objects.requireNonNull(queryPhaseSearcher, "QueryPhaseSearcher is required"); - this.suggestPhase = new SuggestPhase(); - this.rescorePhase = new RescorePhase(); + this.suggestProcessor = new SuggestProcessor(); + this.rescoreProcessor = new RescoreProcessor(); } public void preProcess(SearchContext context) { @@ -130,7 +130,7 @@ public void preProcess(SearchContext context) { public void execute(SearchContext searchContext) throws QueryPhaseExecutionException { if (searchContext.hasOnlySuggest()) { - suggestPhase.execute(searchContext); + suggestProcessor.process(searchContext); searchContext.queryResult() .topDocs( new TopDocsAndMaxScore(new TopDocs(new TotalHits(0, TotalHits.Relation.EQUAL_TO), Lucene.EMPTY_SCORE_DOCS), Float.NaN), @@ -151,9 +151,9 @@ public void execute(SearchContext searchContext) throws QueryPhaseExecutionExcep boolean rescore = executeInternal(searchContext, queryPhaseSearcher); if (rescore) { // only if we do a regular search - rescorePhase.execute(searchContext); + rescoreProcessor.process(searchContext); } - suggestPhase.execute(searchContext); + suggestProcessor.process(searchContext); aggregationProcessor.postProcess(searchContext); if (searchContext.getProfilers() != null) { diff --git a/server/src/main/java/org/opensearch/search/rescore/RescorePhase.java b/server/src/main/java/org/opensearch/search/rescore/RescoreProcessor.java similarity index 93% rename from server/src/main/java/org/opensearch/search/rescore/RescorePhase.java rename to server/src/main/java/org/opensearch/search/rescore/RescoreProcessor.java index e8e3c1e07dab9..de886ba9f8419 100644 --- a/server/src/main/java/org/opensearch/search/rescore/RescorePhase.java +++ b/server/src/main/java/org/opensearch/search/rescore/RescoreProcessor.java @@ -41,13 +41,13 @@ import java.io.IOException; /** - * Rescore phase of a search request, used to run potentially expensive scoring models against the top matching documents. + * RescoreProcessor of a search request, used to run potentially expensive scoring models against the top matching documents. * * @opensearch.internal */ -public class RescorePhase { +public class RescoreProcessor { - public void execute(SearchContext context) { + public void process(SearchContext context) { TopDocs topDocs = context.queryResult().topDocs().topDocs; if (topDocs.scoreDocs.length == 0) { return; diff --git a/server/src/main/java/org/opensearch/search/suggest/SuggestPhase.java b/server/src/main/java/org/opensearch/search/suggest/SuggestProcessor.java similarity index 95% rename from server/src/main/java/org/opensearch/search/suggest/SuggestPhase.java rename to server/src/main/java/org/opensearch/search/suggest/SuggestProcessor.java index 6ecc8939cfe70..2860959835176 100644 --- a/server/src/main/java/org/opensearch/search/suggest/SuggestPhase.java +++ b/server/src/main/java/org/opensearch/search/suggest/SuggestProcessor.java @@ -45,13 +45,13 @@ import java.util.Map; /** - * Suggest phase of a search request, used to collect suggestions + * SuggestProcessor of a search request, used to collect suggestions * * @opensearch.internal */ -public class SuggestPhase { +public class SuggestProcessor { - public void execute(SearchContext context) { + public void process(SearchContext context) { final SuggestionSearchContext suggest = context.suggest(); if (suggest == null) { return; From a84583125908d9db8a1a13b1f1fdfba8b081789b Mon Sep 17 00:00:00 2001 From: Nishchay Malhotra <114057571+nishchay21@users.noreply.github.com> Date: Thu, 15 Jun 2023 04:18:49 +0530 Subject: [PATCH 023/109] Fixed capitalization in ClusterAwarenessHealthIT class name (#5825) * Refactoring ClusterAwarenessHealthIT class Signed-off-by: Nishchay Malhotra --------- Signed-off-by: Nishchay Malhotra --- ...sterAwarenessHealthIT.java => ClusterAwarenessHealthIT.java} | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) rename server/src/internalClusterTest/java/org/opensearch/cluster/{clusterAwarenessHealthIT.java => ClusterAwarenessHealthIT.java} (99%) diff --git a/server/src/internalClusterTest/java/org/opensearch/cluster/clusterAwarenessHealthIT.java b/server/src/internalClusterTest/java/org/opensearch/cluster/ClusterAwarenessHealthIT.java similarity index 99% rename from server/src/internalClusterTest/java/org/opensearch/cluster/clusterAwarenessHealthIT.java rename to server/src/internalClusterTest/java/org/opensearch/cluster/ClusterAwarenessHealthIT.java index ba759b3dd52d1..232342ae658a1 100644 --- a/server/src/internalClusterTest/java/org/opensearch/cluster/clusterAwarenessHealthIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/cluster/ClusterAwarenessHealthIT.java @@ -21,7 +21,7 @@ import static org.opensearch.test.NodeRoles.onlyRole; @OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) -public class clusterAwarenessHealthIT extends OpenSearchIntegTestCase { +public class ClusterAwarenessHealthIT extends OpenSearchIntegTestCase { public void testAwarenessAttributeHealthSucceeded() { createIndex("test"); From bb1f7e2a1e1add7b0054ade30b60637c44212c4a Mon Sep 17 00:00:00 2001 From: Sagar <99425694+sgup432@users.noreply.github.com> Date: Wed, 14 Jun 2023 17:00:58 -0700 Subject: [PATCH 024/109] [TaskCancellationMonitoringService] Changing version check to 2.9 (#8069) Signed-off-by: Sagar Upadhyaya --- .../opensearch/action/admin/cluster/node/stats/NodeStats.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodeStats.java b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodeStats.java index d03011774bb83..c183562e2e85a 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodeStats.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodeStats.java @@ -184,7 +184,7 @@ public NodeStats(StreamInput in) throws IOException { } else { fileCacheStats = null; } - if (in.getVersion().onOrAfter(Version.V_3_0_0)) { + if (in.getVersion().onOrAfter(Version.V_2_9_0)) { taskCancellationStats = in.readOptionalWriteable(TaskCancellationStats::new); } else { taskCancellationStats = null; @@ -408,7 +408,7 @@ public void writeTo(StreamOutput out) throws IOException { if (out.getVersion().onOrAfter(Version.V_2_7_0)) { out.writeOptionalWriteable(fileCacheStats); } - if (out.getVersion().onOrAfter(Version.V_3_0_0)) { + if (out.getVersion().onOrAfter(Version.V_2_9_0)) { out.writeOptionalWriteable(taskCancellationStats); } } From 1f4d0dfb27755e372657b0ac9c2001e2b6076bd5 Mon Sep 17 00:00:00 2001 From: Raghuvansh Raj Date: Thu, 15 Jun 2023 12:47:34 +0530 Subject: [PATCH 025/109] Add AWS v2 SDK and netty dependencies in repository-s3 plugin (#7216) Signed-off-by: Raghuvansh Raj --- plugins/repository-s3/build.gradle | 244 ++++++++++++++---- .../repository-s3/licenses/netty-LICENSE.txt | 202 +++++++++++++++ .../repository-s3/licenses/netty-NOTICE.txt | 116 +++++++++ .../netty-buffer-4.1.93.Final.jar.sha1 | 1 + .../netty-codec-4.1.93.Final.jar.sha1 | 1 + .../netty-codec-http-4.1.93.Final.jar.sha1 | 1 + .../netty-codec-http2-4.1.93.Final.jar.sha1 | 1 + .../netty-common-4.1.93.Final.jar.sha1 | 1 + .../netty-handler-4.1.93.Final.jar.sha1 | 1 + .../netty-nio-client-2.20.55.jar.sha1 | 1 + .../netty-resolver-4.1.93.Final.jar.sha1 | 1 + .../netty-transport-4.1.93.Final.jar.sha1 | 1 + ...nsport-classes-epoll-4.1.93.Final.jar.sha1 | 1 + ...t-native-unix-common-4.1.93.Final.jar.sha1 | 1 + 14 files changed, 523 insertions(+), 50 deletions(-) create mode 100644 plugins/repository-s3/licenses/netty-LICENSE.txt create mode 100644 plugins/repository-s3/licenses/netty-NOTICE.txt create mode 100644 plugins/repository-s3/licenses/netty-buffer-4.1.93.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-codec-4.1.93.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-codec-http-4.1.93.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-codec-http2-4.1.93.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-common-4.1.93.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-handler-4.1.93.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-nio-client-2.20.55.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-resolver-4.1.93.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-transport-4.1.93.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-transport-classes-epoll-4.1.93.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-transport-native-unix-common-4.1.93.Final.jar.sha1 diff --git a/plugins/repository-s3/build.gradle b/plugins/repository-s3/build.gradle index 572dc76da55b4..5bd9747ade9da 100644 --- a/plugins/repository-s3/build.gradle +++ b/plugins/repository-s3/build.gradle @@ -46,6 +46,7 @@ opensearchplugin { } dependencies { + // aws sdk v2 stack api "software.amazon.awssdk:sdk-core:${versions.aws}" api "software.amazon.awssdk:annotations:${versions.aws}" api "software.amazon.awssdk:aws-core:${versions.aws}" @@ -67,6 +68,8 @@ dependencies { api "software.amazon.awssdk:aws-json-protocol:${versions.aws}" api "software.amazon.awssdk:aws-query-protocol:${versions.aws}" api "software.amazon.awssdk:sts:${versions.aws}" + api "software.amazon.awssdk:netty-nio-client:${versions.aws}" + api "org.reactivestreams:reactive-streams:${versions.reactivestreams}" api "org.apache.httpcomponents:httpclient:${versions.httpclient}" api "org.apache.httpcomponents:httpcore:${versions.httpcore}" @@ -80,6 +83,18 @@ dependencies { api "joda-time:joda-time:${versions.joda}" api "org.slf4j:slf4j-api:${versions.slf4j}" + // network stack + api "io.netty:netty-buffer:${versions.netty}" + api "io.netty:netty-codec:${versions.netty}" + api "io.netty:netty-codec-http:${versions.netty}" + api "io.netty:netty-codec-http2:${versions.netty}" + api "io.netty:netty-common:${versions.netty}" + api "io.netty:netty-handler:${versions.netty}" + api "io.netty:netty-resolver:${versions.netty}" + api "io.netty:netty-transport:${versions.netty}" + api "io.netty:netty-transport-native-unix-common:${versions.netty}" + api "io.netty:netty-transport-classes-epoll:${versions.netty}" + // HACK: javax.xml.bind was removed from default modules in java 9, so we pull the api in here, // and allowlist this hack in JarHell api 'javax.xml.bind:jaxb-api:2.3.1' @@ -96,6 +111,7 @@ restResources { tasks.named("dependencyLicenses").configure { mapping from: /jackson-.*/, to: 'jackson' mapping from: /jaxb-.*/, to: 'jaxb' + mapping from: /netty-.*/, to: 'netty' } bundlePlugin { @@ -375,56 +391,184 @@ TaskProvider s3ThirdPartyTest = tasks.register("s3ThirdPartyTest", Test) { } tasks.named("check").configure { dependsOn(s3ThirdPartyTest) } -thirdPartyAudit.ignoreMissingClasses( - // classes are missing - 'javax.jms.Message', - 'javax.servlet.ServletContextEvent', - 'javax.servlet.ServletContextListener', - 'org.apache.avalon.framework.logger.Logger', - 'org.apache.log.Hierarchy', - 'org.apache.log.Logger', - 'org.slf4j.impl.StaticLoggerBinder', - 'org.slf4j.impl.StaticMDCBinder', - 'org.slf4j.impl.StaticMarkerBinder', - 'software.amazon.awssdk.arns.Arn', - 'software.amazon.awssdk.arns.ArnResource', - 'software.amazon.awssdk.crt.CRT', - 'software.amazon.awssdk.crt.auth.credentials.Credentials', - 'software.amazon.awssdk.crt.auth.credentials.CredentialsProvider', - 'software.amazon.awssdk.crt.auth.credentials.DelegateCredentialsProvider$DelegateCredentialsProviderBuilder', - 'software.amazon.awssdk.crt.http.HttpHeader', - 'software.amazon.awssdk.crt.http.HttpMonitoringOptions', - 'software.amazon.awssdk.crt.http.HttpProxyOptions', - 'software.amazon.awssdk.crt.http.HttpRequest', - 'software.amazon.awssdk.crt.http.HttpRequestBodyStream', - 'software.amazon.awssdk.crt.io.ClientBootstrap', - 'software.amazon.awssdk.crt.io.ExponentialBackoffRetryOptions', - 'software.amazon.awssdk.crt.io.StandardRetryOptions', - 'software.amazon.awssdk.crt.io.TlsCipherPreference', - 'software.amazon.awssdk.crt.io.TlsContext', - 'software.amazon.awssdk.crt.io.TlsContextOptions', - 'software.amazon.awssdk.crt.s3.ChecksumAlgorithm', - 'software.amazon.awssdk.crt.s3.ChecksumConfig', - 'software.amazon.awssdk.crt.s3.ChecksumConfig$ChecksumLocation', - 'software.amazon.awssdk.crt.s3.ResumeToken', - 'software.amazon.awssdk.crt.s3.S3Client', - 'software.amazon.awssdk.crt.s3.S3ClientOptions', - 'software.amazon.awssdk.crt.s3.S3FinishedResponseContext', - 'software.amazon.awssdk.crt.s3.S3MetaRequest', - 'software.amazon.awssdk.crt.s3.S3MetaRequestOptions', - 'software.amazon.awssdk.crt.s3.S3MetaRequestOptions$MetaRequestType', - 'software.amazon.awssdk.crt.s3.S3MetaRequestResponseHandler', - 'software.amazon.awssdk.crtcore.CrtConfigurationUtils', - 'software.amazon.awssdk.crtcore.CrtConnectionHealthConfiguration', - 'software.amazon.awssdk.crtcore.CrtConnectionHealthConfiguration$Builder', - 'software.amazon.awssdk.crtcore.CrtConnectionHealthConfiguration$DefaultBuilder', - 'software.amazon.awssdk.crtcore.CrtProxyConfiguration', - 'software.amazon.awssdk.crtcore.CrtProxyConfiguration$Builder', - 'software.amazon.awssdk.crtcore.CrtProxyConfiguration$DefaultBuilder', - 'software.amazon.eventstream.HeaderValue', - 'software.amazon.eventstream.Message', - 'software.amazon.eventstream.MessageDecoder' -) +thirdPartyAudit { + ignoreMissingClasses( + // classes are missing + 'javax.jms.Message', + 'javax.servlet.ServletContextEvent', + 'javax.servlet.ServletContextListener', + 'org.apache.avalon.framework.logger.Logger', + 'org.apache.log.Hierarchy', + 'org.apache.log.Logger', + + // from io.netty.handler.ssl.OpenSslEngine (netty) + 'io.netty.internal.tcnative.AsyncSSLPrivateKeyMethod', + 'io.netty.internal.tcnative.AsyncTask', + 'io.netty.internal.tcnative.Buffer', + 'io.netty.internal.tcnative.CertificateCompressionAlgo', + 'io.netty.internal.tcnative.Library', + 'io.netty.internal.tcnative.ResultCallback', + 'io.netty.internal.tcnative.SSL', + 'io.netty.internal.tcnative.SSLContext', + 'io.netty.internal.tcnative.SSLPrivateKeyMethod', + 'io.netty.internal.tcnative.SSLSession', + 'io.netty.internal.tcnative.SSLSessionCache', + 'io.netty.internal.tcnative.CertificateCallback', + 'io.netty.internal.tcnative.CertificateVerifier', + 'io.netty.internal.tcnative.SessionTicketKey', + 'io.netty.internal.tcnative.SniHostNameMatcher', + + 'com.aayushatharva.brotli4j.Brotli4jLoader', + 'com.aayushatharva.brotli4j.decoder.DecoderJNI$Status', + 'com.aayushatharva.brotli4j.decoder.DecoderJNI$Wrapper', + 'com.aayushatharva.brotli4j.encoder.BrotliEncoderChannel', + 'com.aayushatharva.brotli4j.encoder.Encoder', + 'com.aayushatharva.brotli4j.encoder.Encoder$Mode', + 'com.aayushatharva.brotli4j.encoder.Encoder$Parameters', + + 'com.google.protobuf.nano.CodedOutputByteBufferNano', + 'com.google.protobuf.nano.MessageNano', + + 'com.ning.compress.BufferRecycler', + 'com.ning.compress.lzf.ChunkDecoder', + 'com.ning.compress.lzf.ChunkEncoder', + 'com.ning.compress.lzf.LZFChunk', + 'com.ning.compress.lzf.LZFEncoder', + 'com.ning.compress.lzf.util.ChunkDecoderFactory', + 'com.ning.compress.lzf.util.ChunkEncoderFactory', + + 'io.netty.internal.tcnative.AsyncSSLPrivateKeyMethod', + 'io.netty.internal.tcnative.AsyncTask', + + 'io.netty.internal.tcnative.CertificateCallback', + 'io.netty.internal.tcnative.CertificateVerifier', + 'io.netty.internal.tcnative.ResultCallback', + 'io.netty.internal.tcnative.SessionTicketKey', + 'io.netty.internal.tcnative.SniHostNameMatcher', + 'io.netty.internal.tcnative.SSL', + 'io.netty.internal.tcnative.SSLSession', + 'io.netty.internal.tcnative.SSLSessionCache', + + 'lzma.sdk.lzma.Encoder', + 'net.jpountz.lz4.LZ4Compressor', + 'net.jpountz.lz4.LZ4Factory', + 'net.jpountz.lz4.LZ4FastDecompressor', + 'net.jpountz.xxhash.XXHash32', + 'net.jpountz.xxhash.XXHashFactory', + + // from io.netty.handler.ssl.util.BouncyCastleSelfSignedCertGenerator (netty) + 'org.bouncycastle.cert.X509v3CertificateBuilder', + 'org.bouncycastle.cert.jcajce.JcaX509CertificateConverter', + 'org.bouncycastle.operator.jcajce.JcaContentSignerBuilder', + 'org.bouncycastle.openssl.PEMEncryptedKeyPair', + 'org.bouncycastle.openssl.PEMParser', + 'org.bouncycastle.openssl.jcajce.JcaPEMKeyConverter', + 'org.bouncycastle.openssl.jcajce.JceOpenSSLPKCS8DecryptorProviderBuilder', + 'org.bouncycastle.openssl.jcajce.JcePEMDecryptorProviderBuilder', + 'org.bouncycastle.pkcs.PKCS8EncryptedPrivateKeyInfo', + + 'org.conscrypt.AllocatedBuffer', + 'org.conscrypt.BufferAllocator', + 'org.conscrypt.Conscrypt', + 'org.conscrypt.HandshakeListener', + + 'org.eclipse.jetty.alpn.ALPN$ClientProvider', + 'org.eclipse.jetty.alpn.ALPN$ServerProvider', + 'org.eclipse.jetty.alpn.ALPN', + + // from io.netty.handler.ssl.JettyNpnSslEngine (netty) + 'org.eclipse.jetty.npn.NextProtoNego$ClientProvider', + 'org.eclipse.jetty.npn.NextProtoNego$ServerProvider', + 'org.eclipse.jetty.npn.NextProtoNego', + + // from io.netty.handler.codec.marshalling.ChannelBufferByteInput (netty) + 'org.jboss.marshalling.ByteInput', + + // from io.netty.handler.codec.marshalling.ChannelBufferByteOutput (netty) + 'org.jboss.marshalling.ByteOutput', + + // from io.netty.handler.codec.marshalling.CompatibleMarshallingEncoder (netty) + 'org.jboss.marshalling.Marshaller', + + // from io.netty.handler.codec.marshalling.ContextBoundUnmarshallerProvider (netty) + 'org.jboss.marshalling.MarshallerFactory', + 'org.jboss.marshalling.MarshallingConfiguration', + 'org.jboss.marshalling.Unmarshaller', + + 'org.slf4j.impl.StaticLoggerBinder', + 'org.slf4j.impl.StaticMDCBinder', + 'org.slf4j.impl.StaticMarkerBinder', + 'reactor.blockhound.BlockHound$Builder', + 'reactor.blockhound.integration.BlockHoundIntegration', + + 'software.amazon.awssdk.arns.Arn', + 'software.amazon.awssdk.arns.ArnResource', + 'software.amazon.awssdk.crt.CRT', + 'software.amazon.awssdk.crt.auth.credentials.Credentials', + 'software.amazon.awssdk.crt.auth.credentials.CredentialsProvider', + 'software.amazon.awssdk.crt.auth.credentials.DelegateCredentialsProvider$DelegateCredentialsProviderBuilder', + 'software.amazon.awssdk.crt.http.HttpHeader', + 'software.amazon.awssdk.crt.http.HttpMonitoringOptions', + 'software.amazon.awssdk.crt.http.HttpProxyOptions', + 'software.amazon.awssdk.crt.http.HttpRequest', + 'software.amazon.awssdk.crt.http.HttpRequestBodyStream', + 'software.amazon.awssdk.crt.io.ClientBootstrap', + 'software.amazon.awssdk.crt.io.ExponentialBackoffRetryOptions', + 'software.amazon.awssdk.crt.io.StandardRetryOptions', + 'software.amazon.awssdk.crt.io.TlsCipherPreference', + 'software.amazon.awssdk.crt.io.TlsContext', + 'software.amazon.awssdk.crt.io.TlsContextOptions', + 'software.amazon.awssdk.crt.s3.ChecksumAlgorithm', + 'software.amazon.awssdk.crt.s3.ChecksumConfig', + 'software.amazon.awssdk.crt.s3.ChecksumConfig$ChecksumLocation', + 'software.amazon.awssdk.crt.s3.ResumeToken', + 'software.amazon.awssdk.crt.s3.S3Client', + 'software.amazon.awssdk.crt.s3.S3ClientOptions', + 'software.amazon.awssdk.crt.s3.S3FinishedResponseContext', + 'software.amazon.awssdk.crt.s3.S3MetaRequest', + 'software.amazon.awssdk.crt.s3.S3MetaRequestOptions', + 'software.amazon.awssdk.crt.s3.S3MetaRequestOptions$MetaRequestType', + 'software.amazon.awssdk.crt.s3.S3MetaRequestResponseHandler', + 'software.amazon.awssdk.crtcore.CrtConfigurationUtils', + 'software.amazon.awssdk.crtcore.CrtConnectionHealthConfiguration', + 'software.amazon.awssdk.crtcore.CrtConnectionHealthConfiguration$Builder', + 'software.amazon.awssdk.crtcore.CrtConnectionHealthConfiguration$DefaultBuilder', + 'software.amazon.awssdk.crtcore.CrtProxyConfiguration', + 'software.amazon.awssdk.crtcore.CrtProxyConfiguration$Builder', + 'software.amazon.awssdk.crtcore.CrtProxyConfiguration$DefaultBuilder', + 'software.amazon.eventstream.HeaderValue', + 'software.amazon.eventstream.Message', + 'software.amazon.eventstream.MessageDecoder' + ) + + ignoreViolations ( + 'io.netty.handler.ssl.util.OpenJdkSelfSignedCertGenerator', + 'io.netty.handler.ssl.util.OpenJdkSelfSignedCertGenerator$1', + 'io.netty.handler.ssl.util.OpenJdkSelfSignedCertGenerator$2', + 'io.netty.handler.ssl.util.OpenJdkSelfSignedCertGenerator$3', + 'io.netty.handler.ssl.util.OpenJdkSelfSignedCertGenerator$4', + 'io.netty.handler.ssl.util.OpenJdkSelfSignedCertGenerator$5', + + 'io.netty.util.internal.PlatformDependent0', + 'io.netty.util.internal.PlatformDependent0$1', + 'io.netty.util.internal.PlatformDependent0$2', + 'io.netty.util.internal.PlatformDependent0$3', + 'io.netty.util.internal.PlatformDependent0$4', + 'io.netty.util.internal.PlatformDependent0$6', + + 'io.netty.util.internal.shaded.org.jctools.queues.BaseLinkedQueueConsumerNodeRef', + 'io.netty.util.internal.shaded.org.jctools.queues.BaseLinkedQueueProducerNodeRef', + 'io.netty.util.internal.shaded.org.jctools.queues.BaseMpscLinkedArrayQueueColdProducerFields', + 'io.netty.util.internal.shaded.org.jctools.queues.BaseMpscLinkedArrayQueueConsumerFields', + 'io.netty.util.internal.shaded.org.jctools.queues.BaseMpscLinkedArrayQueueProducerFields', + 'io.netty.util.internal.shaded.org.jctools.queues.LinkedQueueNode', + 'io.netty.util.internal.shaded.org.jctools.queues.MpscArrayQueueConsumerIndexField', + 'io.netty.util.internal.shaded.org.jctools.queues.MpscArrayQueueProducerIndexField', + 'io.netty.util.internal.shaded.org.jctools.queues.MpscArrayQueueProducerLimitField', + 'io.netty.util.internal.shaded.org.jctools.util.UnsafeAccess', + 'io.netty.util.internal.shaded.org.jctools.util.UnsafeRefArrayAccess', + ) +} // jarhell with jdk (intentionally, because jaxb was removed from default modules in java 9) if (BuildParams.runtimeJavaVersion <= JavaVersion.VERSION_1_8) { diff --git a/plugins/repository-s3/licenses/netty-LICENSE.txt b/plugins/repository-s3/licenses/netty-LICENSE.txt new file mode 100644 index 0000000000000..d645695673349 --- /dev/null +++ b/plugins/repository-s3/licenses/netty-LICENSE.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. diff --git a/plugins/repository-s3/licenses/netty-NOTICE.txt b/plugins/repository-s3/licenses/netty-NOTICE.txt new file mode 100644 index 0000000000000..5bbf91a14de23 --- /dev/null +++ b/plugins/repository-s3/licenses/netty-NOTICE.txt @@ -0,0 +1,116 @@ + + The Netty Project + ================= + +Please visit the Netty web site for more information: + + * http://netty.io/ + +Copyright 2011 The Netty Project + +The Netty Project 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. + +Also, please refer to each LICENSE..txt file, which is located in +the 'license' directory of the distribution file, for the license terms of the +components that this product depends on. + +------------------------------------------------------------------------------- +This product contains the extensions to Java Collections Framework which has +been derived from the works by JSR-166 EG, Doug Lea, and Jason T. Greene: + + * LICENSE: + * license/LICENSE.jsr166y.txt (Public Domain) + * HOMEPAGE: + * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/ + * http://viewvc.jboss.org/cgi-bin/viewvc.cgi/jbosscache/experimental/jsr166/ + +This product contains a modified version of Robert Harder's Public Domain +Base64 Encoder and Decoder, which can be obtained at: + + * LICENSE: + * license/LICENSE.base64.txt (Public Domain) + * HOMEPAGE: + * http://iharder.sourceforge.net/current/java/base64/ + +This product contains a modified version of 'JZlib', a re-implementation of +zlib in pure Java, which can be obtained at: + + * LICENSE: + * license/LICENSE.jzlib.txt (BSD Style License) + * HOMEPAGE: + * http://www.jcraft.com/jzlib/ + +This product contains a modified version of 'Webbit', a Java event based +WebSocket and HTTP server: + + * LICENSE: + * license/LICENSE.webbit.txt (BSD License) + * HOMEPAGE: + * https://github.com/joewalnes/webbit + +This product optionally depends on 'Protocol Buffers', Google's data +interchange format, which can be obtained at: + + * LICENSE: + * license/LICENSE.protobuf.txt (New BSD License) + * HOMEPAGE: + * http://code.google.com/p/protobuf/ + +This product optionally depends on 'Bouncy Castle Crypto APIs' to generate +a temporary self-signed X.509 certificate when the JVM does not provide the +equivalent functionality. It can be obtained at: + + * LICENSE: + * license/LICENSE.bouncycastle.txt (MIT License) + * HOMEPAGE: + * http://www.bouncycastle.org/ + +This product optionally depends on 'SLF4J', a simple logging facade for Java, +which can be obtained at: + + * LICENSE: + * license/LICENSE.slf4j.txt (MIT License) + * HOMEPAGE: + * http://www.slf4j.org/ + +This product optionally depends on 'Apache Commons Logging', a logging +framework, which can be obtained at: + + * LICENSE: + * license/LICENSE.commons-logging.txt (Apache License 2.0) + * HOMEPAGE: + * http://commons.apache.org/logging/ + +This product optionally depends on 'Apache Log4J', a logging framework, +which can be obtained at: + + * LICENSE: + * license/LICENSE.log4j.txt (Apache License 2.0) + * HOMEPAGE: + * http://logging.apache.org/log4j/ + +This product optionally depends on 'JBoss Logging', a logging framework, +which can be obtained at: + + * LICENSE: + * license/LICENSE.jboss-logging.txt (GNU LGPL 2.1) + * HOMEPAGE: + * http://anonsvn.jboss.org/repos/common/common-logging-spi/ + +This product optionally depends on 'Apache Felix', an open source OSGi +framework implementation, which can be obtained at: + + * LICENSE: + * license/LICENSE.felix.txt (Apache License 2.0) + * HOMEPAGE: + * http://felix.apache.org/ diff --git a/plugins/repository-s3/licenses/netty-buffer-4.1.93.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-buffer-4.1.93.Final.jar.sha1 new file mode 100644 index 0000000000000..5c5a17a9466f1 --- /dev/null +++ b/plugins/repository-s3/licenses/netty-buffer-4.1.93.Final.jar.sha1 @@ -0,0 +1 @@ +87fac21f4ef95157866b07b48e3c707a2f13c581 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-codec-4.1.93.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-codec-4.1.93.Final.jar.sha1 new file mode 100644 index 0000000000000..2b12a111335a2 --- /dev/null +++ b/plugins/repository-s3/licenses/netty-codec-4.1.93.Final.jar.sha1 @@ -0,0 +1 @@ +503badb458b6586632be8d1f81aa4e5ab99a80fc \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-codec-http-4.1.93.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-codec-http-4.1.93.Final.jar.sha1 new file mode 100644 index 0000000000000..6719e882e40fe --- /dev/null +++ b/plugins/repository-s3/licenses/netty-codec-http-4.1.93.Final.jar.sha1 @@ -0,0 +1 @@ +36acf0c94d03eb6ecef78a749a32cbb7dc0c57b4 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-codec-http2-4.1.93.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-codec-http2-4.1.93.Final.jar.sha1 new file mode 100644 index 0000000000000..02423842d6244 --- /dev/null +++ b/plugins/repository-s3/licenses/netty-codec-http2-4.1.93.Final.jar.sha1 @@ -0,0 +1 @@ +0f1625b43bde13ec057da0d2fe381ded2547a70e \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-common-4.1.93.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-common-4.1.93.Final.jar.sha1 new file mode 100644 index 0000000000000..2324a54dc5735 --- /dev/null +++ b/plugins/repository-s3/licenses/netty-common-4.1.93.Final.jar.sha1 @@ -0,0 +1 @@ +1cfc49b91b0d3ddb30c9f7d8467e5d02ae8babdf \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-handler-4.1.93.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-handler-4.1.93.Final.jar.sha1 new file mode 100644 index 0000000000000..54e5b22047812 --- /dev/null +++ b/plugins/repository-s3/licenses/netty-handler-4.1.93.Final.jar.sha1 @@ -0,0 +1 @@ +10f7ed9d8e1bfcba416074c70e5388be96116bfc \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-nio-client-2.20.55.jar.sha1 b/plugins/repository-s3/licenses/netty-nio-client-2.20.55.jar.sha1 new file mode 100644 index 0000000000000..489f18e0bceaa --- /dev/null +++ b/plugins/repository-s3/licenses/netty-nio-client-2.20.55.jar.sha1 @@ -0,0 +1 @@ +4c269571ad2fb19851ebd7c7856aa2975fe0bab3 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-resolver-4.1.93.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-resolver-4.1.93.Final.jar.sha1 new file mode 100644 index 0000000000000..c795762c2d7f1 --- /dev/null +++ b/plugins/repository-s3/licenses/netty-resolver-4.1.93.Final.jar.sha1 @@ -0,0 +1 @@ +3860e99075f9e078364ed38f6d6fc8825b7a168a \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-transport-4.1.93.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-transport-4.1.93.Final.jar.sha1 new file mode 100644 index 0000000000000..687cade3c7b3a --- /dev/null +++ b/plugins/repository-s3/licenses/netty-transport-4.1.93.Final.jar.sha1 @@ -0,0 +1 @@ +1a0894915c8027ce83b4d6a811c4e765955efd15 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-transport-classes-epoll-4.1.93.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-transport-classes-epoll-4.1.93.Final.jar.sha1 new file mode 100644 index 0000000000000..81a180b455cdd --- /dev/null +++ b/plugins/repository-s3/licenses/netty-transport-classes-epoll-4.1.93.Final.jar.sha1 @@ -0,0 +1 @@ +79c2d95406bc2ef38c08c34efd16039919db80be \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-transport-native-unix-common-4.1.93.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-transport-native-unix-common-4.1.93.Final.jar.sha1 new file mode 100644 index 0000000000000..0f0acb2eccddf --- /dev/null +++ b/plugins/repository-s3/licenses/netty-transport-native-unix-common-4.1.93.Final.jar.sha1 @@ -0,0 +1 @@ +f4fb8b4c8da539091f43abcbb9f0389e48807eea \ No newline at end of file From 87e34a3b32d77640b3e948f4117a67fdcf7eda25 Mon Sep 17 00:00:00 2001 From: Rishab Nahata Date: Thu, 15 Jun 2023 13:20:44 +0530 Subject: [PATCH 026/109] Fix bug with delete shard routing weights on node restart (#8057) Signed-off-by: Rishab Nahata --- .../cluster/routing/WeightedRoutingIT.java | 47 +++++++++++++++++++ .../cluster/routing/WeightedRouting.java | 2 +- 2 files changed, 48 insertions(+), 1 deletion(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/cluster/routing/WeightedRoutingIT.java b/server/src/internalClusterTest/java/org/opensearch/cluster/routing/WeightedRoutingIT.java index 24ac2f2326931..a7597912c4110 100644 --- a/server/src/internalClusterTest/java/org/opensearch/cluster/routing/WeightedRoutingIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/cluster/routing/WeightedRoutingIT.java @@ -385,6 +385,53 @@ public void testWeightedRoutingMetadataOnOSProcessRestart() throws Exception { assertNotNull(internalCluster().clusterService().state().metadata().weightedRoutingMetadata()); } + public void testWeightedRoutingOnOSProcessRestartAfterWeightDelete() throws Exception { + Settings commonSettings = Settings.builder() + .put("cluster.routing.allocation.awareness.attributes", "zone") + .put("cluster.routing.allocation.awareness.force.zone.values", "a,b,c") + .build(); + + internalCluster().startNodes( + Settings.builder().put(commonSettings).put("node.attr.zone", "a").build(), + Settings.builder().put(commonSettings).put("node.attr.zone", "b").build(), + Settings.builder().put(commonSettings).put("node.attr.zone", "c").build() + ); + + logger.info("--> waiting for nodes to form a cluster"); + ClusterHealthResponse health = client().admin().cluster().prepareHealth().setWaitForNodes("3").execute().actionGet(); + assertThat(health.isTimedOut(), equalTo(false)); + + ensureGreen(); + + logger.info("--> setting shard routing weights for weighted round robin"); + Map weights = Map.of("a", 1.0, "b", 2.0, "c", 3.0); + WeightedRouting weightedRouting = new WeightedRouting("zone", weights); + // put api call to set weights + ClusterPutWeightedRoutingResponse response = client().admin() + .cluster() + .prepareWeightedRouting() + .setWeightedRouting(weightedRouting) + .setVersion(-1) + .get(); + assertEquals(response.isAcknowledged(), true); + + ensureStableCluster(3); + + // routing weights are set in cluster metadata + assertNotNull(internalCluster().clusterService().state().metadata().weightedRoutingMetadata()); + + ensureGreen(); + + // delete weighted routing metadata + ClusterDeleteWeightedRoutingResponse deleteResponse = client().admin().cluster().prepareDeleteWeightedRouting().setVersion(0).get(); + assertTrue(deleteResponse.isAcknowledged()); + + // Restart a random data node and check that OS process comes healthy + internalCluster().restartRandomDataNode(); + ensureGreen(); + assertNotNull(internalCluster().clusterService().state().metadata().weightedRoutingMetadata()); + } + public void testDeleteWeightedRouting_WeightsNotSet() { Settings commonSettings = Settings.builder() .put("cluster.routing.allocation.awareness.attributes", "zone") diff --git a/server/src/main/java/org/opensearch/cluster/routing/WeightedRouting.java b/server/src/main/java/org/opensearch/cluster/routing/WeightedRouting.java index bff8dd833f2de..8dcf9d8dc6bd7 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/WeightedRouting.java +++ b/server/src/main/java/org/opensearch/cluster/routing/WeightedRouting.java @@ -47,7 +47,7 @@ public WeightedRouting(StreamInput in) throws IOException { } public boolean isSet() { - return (!this.attributeName.isEmpty() && !this.weights.isEmpty()); + return this.attributeName != null && !this.attributeName.isEmpty() && this.weights != null && !this.weights.isEmpty(); } @Override From cb4361c00107e113c82f87103307c3626583e940 Mon Sep 17 00:00:00 2001 From: Ankit Kala Date: Thu, 15 Jun 2023 14:22:17 +0530 Subject: [PATCH 027/109] Establish seed node connections in async during node bootstrap (#8038) * Establish seed node connection setup in async Signed-off-by: Ankit Kala --- CHANGELOG.md | 1 + .../transport/RemoteClusterService.java | 32 +++++------------ .../transport/TransportService.java | 7 +++- .../transport/RemoteClusterClientTests.java | 2 +- .../transport/RemoteClusterServiceTests.java | 35 +++++++++++++------ 5 files changed, 42 insertions(+), 35 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 05d89358b993d..e532ab57a470c 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -102,6 +102,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Add TokenManager Interface ([#7452](https://github.com/opensearch-project/OpenSearch/pull/7452)) - Add Remote store as a segment replication source ([#7653](https://github.com/opensearch-project/OpenSearch/pull/7653)) - Add descending order search optimization through reverse segment read. ([#7967](https://github.com/opensearch-project/OpenSearch/pull/7967)) +- Make remote cluster connection setup in async ([#8038](https://github.com/opensearch-project/OpenSearch/pull/8038)) - Update components of segrep backpressure to support remote store. ([#8020](https://github.com/opensearch-project/OpenSearch/pull/8020)) ### Dependencies diff --git a/server/src/main/java/org/opensearch/transport/RemoteClusterService.java b/server/src/main/java/org/opensearch/transport/RemoteClusterService.java index 838367baf8a2e..60e166a4e300c 100644 --- a/server/src/main/java/org/opensearch/transport/RemoteClusterService.java +++ b/server/src/main/java/org/opensearch/transport/RemoteClusterService.java @@ -38,7 +38,6 @@ import org.opensearch.action.OriginalIndices; import org.opensearch.action.support.GroupedActionListener; import org.opensearch.action.support.IndicesOptions; -import org.opensearch.action.support.PlainActionFuture; import org.opensearch.client.Client; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.node.DiscoveryNodeRole; @@ -63,7 +62,6 @@ import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; import java.util.function.BiFunction; import java.util.function.Function; import java.util.function.Predicate; @@ -328,35 +326,23 @@ synchronized void updateRemoteCluster(String clusterAlias, Settings newSettings, } /** - * Connects to all remote clusters in a blocking fashion. This should be called on node startup to establish an initial connection + * Connects to all remote clusters in a non-blocking fashion. This should be called on node startup to establish an initial connection * to all configured seed nodes. */ - void initializeRemoteClusters() { - final TimeValue timeValue = REMOTE_INITIAL_CONNECTION_TIMEOUT_SETTING.get(settings); - final PlainActionFuture> future = new PlainActionFuture<>(); + void initializeRemoteClusters(ActionListener listener) { Set enabledClusters = RemoteClusterAware.getEnabledRemoteClusters(settings); - if (enabledClusters.isEmpty()) { + listener.onResponse(null); return; } - GroupedActionListener listener = new GroupedActionListener<>(future, enabledClusters.size()); - for (String clusterAlias : enabledClusters) { - updateRemoteCluster(clusterAlias, settings, listener); - } + GroupedActionListener groupListener = new GroupedActionListener<>( + ActionListener.wrap(r -> listener.onResponse(null), listener::onFailure), + enabledClusters.size() + ); - if (enabledClusters.isEmpty()) { - future.onResponse(null); - } - - try { - future.get(timeValue.millis(), TimeUnit.MILLISECONDS); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } catch (TimeoutException ex) { - logger.warn("failed to connect to remote clusters within {}", timeValue.toString()); - } catch (Exception e) { - throw new IllegalStateException("failed to connect to remote clusters", e); + for (String clusterAlias : enabledClusters) { + updateRemoteCluster(clusterAlias, settings, groupListener); } } diff --git a/server/src/main/java/org/opensearch/transport/TransportService.java b/server/src/main/java/org/opensearch/transport/TransportService.java index 0a31d2dbf6ec5..8cd46a42952ae 100644 --- a/server/src/main/java/org/opensearch/transport/TransportService.java +++ b/server/src/main/java/org/opensearch/transport/TransportService.java @@ -300,7 +300,12 @@ protected void doStart() { if (remoteClusterClient) { // here we start to connect to the remote clusters - remoteClusterService.initializeRemoteClusters(); + remoteClusterService.initializeRemoteClusters( + ActionListener.wrap( + r -> logger.info("Remote clusters initialized successfully."), + e -> logger.error("Remote clusters initialization failed partially", e) + ) + ); } } diff --git a/server/src/test/java/org/opensearch/transport/RemoteClusterClientTests.java b/server/src/test/java/org/opensearch/transport/RemoteClusterClientTests.java index 94c535a5e20f5..b89d652510850 100644 --- a/server/src/test/java/org/opensearch/transport/RemoteClusterClientTests.java +++ b/server/src/test/java/org/opensearch/transport/RemoteClusterClientTests.java @@ -86,7 +86,7 @@ public void testConnectAndExecuteRequest() throws Exception { service.acceptIncomingRequests(); logger.info("now accepting incoming requests on local transport"); RemoteClusterService remoteClusterService = service.getRemoteClusterService(); - assertTrue(remoteClusterService.isRemoteNodeConnected("test", remoteNode)); + assertBusy(() -> { assertTrue(remoteClusterService.isRemoteNodeConnected("test", remoteNode)); }, 10, TimeUnit.SECONDS); Client client = remoteClusterService.getRemoteClusterClient(threadPool, "test"); ClusterStateResponse clusterStateResponse = client.admin().cluster().prepareState().execute().get(); assertNotNull(clusterStateResponse); diff --git a/server/src/test/java/org/opensearch/transport/RemoteClusterServiceTests.java b/server/src/test/java/org/opensearch/transport/RemoteClusterServiceTests.java index a25820622b2cf..52bf7aa08fe67 100644 --- a/server/src/test/java/org/opensearch/transport/RemoteClusterServiceTests.java +++ b/server/src/test/java/org/opensearch/transport/RemoteClusterServiceTests.java @@ -59,6 +59,7 @@ import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiFunction; @@ -92,6 +93,20 @@ private MockTransportService startTransport( return RemoteClusterConnectionTests.startTransport(id, knownNodes, version, threadPool, settings); } + void initializeRemoteClusters(RemoteClusterService service) { + final PlainActionFuture future = new PlainActionFuture<>(); + service.initializeRemoteClusters(future); + try { + future.get(10, TimeUnit.SECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } catch (TimeoutException ex) { + logger.warn("Timed out connecting to remote clusters"); + } catch (Exception e) { + throw new IllegalStateException("failed to connect to remote clusters", e); + } + } + public void testSettingsAreRegistered() { assertTrue(ClusterSettings.BUILT_IN_CLUSTER_SETTINGS.contains(RemoteClusterService.REMOTE_CLUSTER_SKIP_UNAVAILABLE)); assertTrue(ClusterSettings.BUILT_IN_CLUSTER_SETTINGS.contains(RemoteClusterService.REMOTE_INITIAL_CONNECTION_TIMEOUT_SETTING)); @@ -157,7 +172,7 @@ public void testGroupClusterIndices() throws IOException { builder.putList("cluster.remote.cluster_2.seeds", cluster2Seed.getAddress().toString()); try (RemoteClusterService service = new RemoteClusterService(builder.build(), transportService)) { assertFalse(service.isCrossClusterSearchEnabled()); - service.initializeRemoteClusters(); + initializeRemoteClusters(service); assertTrue(service.isCrossClusterSearchEnabled()); assertTrue(service.isRemoteClusterRegistered("cluster_1")); assertTrue(service.isRemoteClusterRegistered("cluster_2")); @@ -228,7 +243,7 @@ public void testGroupIndices() throws IOException { builder.putList("cluster.remote.cluster_2.seeds", cluster2Seed.getAddress().toString()); try (RemoteClusterService service = new RemoteClusterService(builder.build(), transportService)) { assertFalse(service.isCrossClusterSearchEnabled()); - service.initializeRemoteClusters(); + initializeRemoteClusters(service); assertTrue(service.isCrossClusterSearchEnabled()); assertTrue(service.isRemoteClusterRegistered("cluster_1")); assertTrue(service.isRemoteClusterRegistered("cluster_2")); @@ -321,7 +336,7 @@ public void testIncrementallyAddClusters() throws IOException { builder.putList("cluster.remote.cluster_2.seeds", cluster2Seed.getAddress().toString()); try (RemoteClusterService service = new RemoteClusterService(Settings.EMPTY, transportService)) { assertFalse(service.isCrossClusterSearchEnabled()); - service.initializeRemoteClusters(); + initializeRemoteClusters(service); assertFalse(service.isCrossClusterSearchEnabled()); Settings cluster1Settings = createSettings( "cluster_1", @@ -384,7 +399,7 @@ public void testDefaultPingSchedule() throws IOException { transportService.acceptIncomingRequests(); try (RemoteClusterService service = new RemoteClusterService(settings, transportService)) { assertFalse(service.isCrossClusterSearchEnabled()); - service.initializeRemoteClusters(); + initializeRemoteClusters(service); assertTrue(service.isCrossClusterSearchEnabled()); service.validateAndUpdateRemoteCluster( "cluster_1", @@ -436,7 +451,7 @@ public void testCustomPingSchedule() throws IOException { TimeValue.timeValueSeconds(randomIntBetween(1, 10)); builder.put("cluster.remote.cluster_2.transport.ping_schedule", pingSchedule2); try (RemoteClusterService service = new RemoteClusterService(builder.build(), transportService)) { - service.initializeRemoteClusters(); + initializeRemoteClusters(service); assertTrue(service.isRemoteClusterRegistered("cluster_1")); RemoteClusterConnection remoteClusterConnection1 = service.getRemoteClusterConnection("cluster_1"); assertEquals(pingSchedule1, remoteClusterConnection1.getConnectionManager().getConnectionProfile().getPingInterval()); @@ -467,7 +482,7 @@ public void testChangeSettings() throws Exception { Settings.Builder builder = Settings.builder(); builder.putList("cluster.remote.cluster_1.seeds", cluster1Seed.getAddress().toString()); try (RemoteClusterService service = new RemoteClusterService(builder.build(), transportService)) { - service.initializeRemoteClusters(); + initializeRemoteClusters(service); RemoteClusterConnection remoteClusterConnection = service.getRemoteClusterConnection("cluster_1"); Settings.Builder settingsChange = Settings.builder(); TimeValue pingSchedule = TimeValue.timeValueSeconds(randomIntBetween(6, 8)); @@ -517,7 +532,7 @@ public void testRemoteNodeAttribute() throws IOException, InterruptedException { builder.putList("cluster.remote.cluster_2.seed", c2N1Node.getAddress().toString()); try (RemoteClusterService service = new RemoteClusterService(settings, transportService)) { assertFalse(service.isCrossClusterSearchEnabled()); - service.initializeRemoteClusters(); + initializeRemoteClusters(service); assertFalse(service.isCrossClusterSearchEnabled()); final CountDownLatch firstLatch = new CountDownLatch(1); @@ -580,7 +595,7 @@ public void testRemoteNodeRoles() throws IOException, InterruptedException { builder.putList("cluster.remote.cluster_2.seed", c2N1Node.getAddress().toString()); try (RemoteClusterService service = new RemoteClusterService(settings, transportService)) { assertFalse(service.isCrossClusterSearchEnabled()); - service.initializeRemoteClusters(); + initializeRemoteClusters(service); assertFalse(service.isCrossClusterSearchEnabled()); final CountDownLatch firstLatch = new CountDownLatch(1); @@ -648,7 +663,7 @@ public void testCollectNodes() throws InterruptedException, IOException { builder.putList("cluster.remote.cluster_2.seed", c2N1Node.getAddress().toString()); try (RemoteClusterService service = new RemoteClusterService(settings, transportService)) { assertFalse(service.isCrossClusterSearchEnabled()); - service.initializeRemoteClusters(); + initializeRemoteClusters(service); assertFalse(service.isCrossClusterSearchEnabled()); final CountDownLatch firstLatch = new CountDownLatch(1); @@ -896,7 +911,7 @@ public void testReconnectWhenStrategySettingsUpdated() throws Exception { builder.putList("cluster.remote.cluster_test.seeds", Collections.singletonList(node0.getAddress().toString())); try (RemoteClusterService service = new RemoteClusterService(builder.build(), transportService)) { assertFalse(service.isCrossClusterSearchEnabled()); - service.initializeRemoteClusters(); + initializeRemoteClusters(service); assertTrue(service.isCrossClusterSearchEnabled()); final RemoteClusterConnection firstRemoteClusterConnection = service.getRemoteClusterConnection("cluster_test"); From 60afeb7b05323f5626b395dc8ffd07eb5d773c15 Mon Sep 17 00:00:00 2001 From: Ankit Kala Date: Thu, 15 Jun 2023 17:54:28 +0530 Subject: [PATCH 028/109] Enforce 512 byte document ID limit in bulk updates (#8039) * Enforce doc id limit on UpdateRequests as well Signed-off-by: Ankit Kala * PR comments Signed-off-by: Ankit Kala * Address comments 2 Signed-off-by: Ankit Kala * Move changelog entry Signed-off-by: Andrew Ross --------- Signed-off-by: Ankit Kala Signed-off-by: Andrew Ross Co-authored-by: Andrew Ross --- CHANGELOG.md | 1 + .../action/bulk/BulkIntegrationIT.java | 35 ++++++++++++++ .../opensearch/action/DocWriteRequest.java | 20 ++++++++ .../opensearch/action/index/IndexRequest.java | 8 +--- .../action/update/UpdateRequest.java | 3 ++ .../action/bulk/BulkRequestTests.java | 46 +++++++++++++++++++ 6 files changed, 106 insertions(+), 7 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index e532ab57a470c..b964dc5844757 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -127,6 +127,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), ### Fixed - Fixing error: adding a new/forgotten parameter to the configuration for checking the config on startup in plugins/repository-s3 #7924 +- Enforce 512 byte document ID limit in bulk updates ([#8039](https://github.com/opensearch-project/OpenSearch/pull/8039)) ### Security diff --git a/server/src/internalClusterTest/java/org/opensearch/action/bulk/BulkIntegrationIT.java b/server/src/internalClusterTest/java/org/opensearch/action/bulk/BulkIntegrationIT.java index 2489fc4894742..ec8d02a8c57b7 100644 --- a/server/src/internalClusterTest/java/org/opensearch/action/bulk/BulkIntegrationIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/action/bulk/BulkIntegrationIT.java @@ -43,6 +43,7 @@ import org.opensearch.action.ingest.PutPipelineRequest; import org.opensearch.action.support.master.AcknowledgedResponse; import org.opensearch.action.support.replication.ReplicationRequest; +import org.opensearch.action.update.UpdateRequest; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.bytes.BytesReference; import org.opensearch.core.xcontent.XContentBuilder; @@ -218,4 +219,38 @@ public void testDeleteIndexWhileIndexing() throws Exception { assertFalse(thread.isAlive()); } + public void testDocIdTooLong() { + String index = "testing"; + createIndex(index); + String validId = String.join("", Collections.nCopies(512, "a")); + String invalidId = String.join("", Collections.nCopies(513, "a")); + + // Index Request + IndexRequest indexRequest = new IndexRequest(index).source(Collections.singletonMap("foo", "baz")); + // Valid id shouldn't throw any exception + assertFalse(client().prepareBulk().add(indexRequest.id(validId)).get().hasFailures()); + // Invalid id should throw the ActionRequestValidationException + validateDocIdLimit(() -> client().prepareBulk().add(indexRequest.id(invalidId)).get()); + + // Update Request + UpdateRequest updateRequest = new UpdateRequest(index, validId).doc("reason", "no source"); + // Valid id shouldn't throw any exception + assertFalse(client().prepareBulk().add(updateRequest).get().hasFailures()); + // Invalid id should throw the ActionRequestValidationException + validateDocIdLimit(() -> client().prepareBulk().add(updateRequest.id(invalidId)).get()); + } + + private void validateDocIdLimit(Runnable runner) { + try { + runner.run(); + fail("Request validation for docId didn't fail"); + } catch (ActionRequestValidationException e) { + assertEquals( + 1, + e.validationErrors().stream().filter(msg -> msg.contains("is too long, must be no longer than 512 bytes but was")).count() + ); + } catch (Exception e) { + fail("Request validation for docId failed with different exception: " + e); + } + } } diff --git a/server/src/main/java/org/opensearch/action/DocWriteRequest.java b/server/src/main/java/org/opensearch/action/DocWriteRequest.java index ed59b5e95a01f..2810452cd583d 100644 --- a/server/src/main/java/org/opensearch/action/DocWriteRequest.java +++ b/server/src/main/java/org/opensearch/action/DocWriteRequest.java @@ -32,6 +32,7 @@ package org.opensearch.action; import org.apache.lucene.util.Accountable; +import org.apache.lucene.util.UnicodeUtil; import org.opensearch.action.delete.DeleteRequest; import org.opensearch.action.index.IndexRequest; import org.opensearch.action.support.IndicesOptions; @@ -248,6 +249,25 @@ static DocWriteRequest readDocumentRequest(@Nullable ShardId shardId, StreamI return docWriteRequest; } + /** + * Validates whether the doc id length is under the limit. + * @param id DocId to verify + * @param validationException containing all the validation errors. + * @return validationException + */ + static ActionRequestValidationException validateDocIdLength(String id, ActionRequestValidationException validationException) { + if (id != null) { + int docIdLength = UnicodeUtil.calcUTF16toUTF8Length(id, 0, id.length()); + if (docIdLength > 512) { + return addValidationError( + "id [" + id + "] is too long, must be no longer than 512 bytes but was: " + docIdLength, + validationException + ); + } + } + return validationException; + } + /** write a document write (index/delete/update) request*/ static void writeDocumentRequest(StreamOutput out, DocWriteRequest request) throws IOException { if (request instanceof IndexRequest) { 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 2182b8d1ffa4c..1cce14ef447f5 100644 --- a/server/src/main/java/org/opensearch/action/index/IndexRequest.java +++ b/server/src/main/java/org/opensearch/action/index/IndexRequest.java @@ -65,7 +65,6 @@ import org.opensearch.index.shard.ShardId; import java.io.IOException; -import java.nio.charset.StandardCharsets; import java.util.Locale; import java.util.Map; import java.util.Objects; @@ -228,12 +227,7 @@ public ActionRequestValidationException validate() { validationException = DocWriteRequest.validateSeqNoBasedCASParams(this, validationException); - if (id != null && id.getBytes(StandardCharsets.UTF_8).length > 512) { - validationException = addValidationError( - "id [" + id + "] is too long, must be no longer than 512 bytes but was: " + id.getBytes(StandardCharsets.UTF_8).length, - validationException - ); - } + validationException = DocWriteRequest.validateDocIdLength(id, validationException); if (pipeline != null && pipeline.isEmpty()) { validationException = addValidationError("pipeline cannot be an empty string", validationException); 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 56913b1ec5915..315ec47e30296 100644 --- a/server/src/main/java/org/opensearch/action/update/UpdateRequest.java +++ b/server/src/main/java/org/opensearch/action/update/UpdateRequest.java @@ -231,6 +231,9 @@ public ActionRequestValidationException validate() { if (doc == null && docAsUpsert) { validationException = addValidationError("doc must be specified if doc_as_upsert is enabled", validationException); } + + validationException = DocWriteRequest.validateDocIdLength(id, validationException); + return validationException; } diff --git a/server/src/test/java/org/opensearch/action/bulk/BulkRequestTests.java b/server/src/test/java/org/opensearch/action/bulk/BulkRequestTests.java index 94e866f959e95..1be82e045ec4c 100644 --- a/server/src/test/java/org/opensearch/action/bulk/BulkRequestTests.java +++ b/server/src/test/java/org/opensearch/action/bulk/BulkRequestTests.java @@ -53,6 +53,7 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -272,6 +273,51 @@ public void testBulkRequestWithRefresh() throws Exception { ); } + public void testBulkRequestInvalidDocIDDuringCreate() { + String validDocID = String.join("", Collections.nCopies(512, "a")); + String invalidDocID = String.join("", Collections.nCopies(513, "a")); + + // doc id length under limit + IndexRequest indexRequest = new IndexRequest("index").id(validDocID).source(Requests.INDEX_CONTENT_TYPE, "field", "value"); + BulkRequest bulkRequest = new BulkRequest(); + bulkRequest.add(indexRequest); + assertNull(bulkRequest.validate()); + + // doc id length over limit + indexRequest.id(invalidDocID); + ActionRequestValidationException validate = bulkRequest.validate(); + assertThat(validate, notNullValue()); + assertEquals( + 1, + validate.validationErrors() + .stream() + .filter(msg -> msg.contains("is too long, must be no longer than 512 bytes but was: ")) + .count() + ); + } + + public void testBulkRequestInvalidDocIDDuringUpdate() { + String validDocID = String.join("", Collections.nCopies(512, "a")); + String invalidDocID = String.join("", Collections.nCopies(513, "a")); + // doc id length under limit + UpdateRequest updateRequest = new UpdateRequest("index", validDocID).doc("reason", "no source"); + BulkRequest bulkRequest = new BulkRequest(); + bulkRequest.add(updateRequest); + assertNull(bulkRequest.validate()); + + // doc id length over limit + updateRequest.id(invalidDocID); + ActionRequestValidationException validate = bulkRequest.validate(); + assertThat(validate, notNullValue()); + assertEquals( + 1, + validate.validationErrors() + .stream() + .filter(msg -> msg.contains("is too long, must be no longer than 512 bytes but was: ")) + .count() + ); + } + // issue 15120 public void testBulkNoSource() throws Exception { BulkRequest bulkRequest = new BulkRequest(); From de51a496a2e727d74aef76d7fb2714b95a582169 Mon Sep 17 00:00:00 2001 From: Varun Bansal Date: Thu, 15 Jun 2023 22:40:48 +0530 Subject: [PATCH 029/109] [Remote Store] Add latest local and remote refresh timestamps in the stats api response (#7922) * Add latest local and remote refresh timestamps in the stats api response Signed-off-by: bansvaru * add a separate field for clock time to allow backpressure to use nano precision fields Signed-off-by: bansvaru * fix spotloss failures Signed-off-by: bansvaru * correct update validation for remote refresh clock time Signed-off-by: bansvaru * remove unused wall clock time lag Signed-off-by: bansvaru * revert unintended changes to backtracker precision time fields Signed-off-by: bansvaru * fix indentation Signed-off-by: bansvaru --------- Signed-off-by: bansvaru --- .../remotestore/stats/RemoteStoreStats.java | 13 +++++- .../remote/RemoteRefreshSegmentTracker.java | 41 +++++++++++++++++++ .../shard/RemoteStoreRefreshListener.java | 12 ++++-- .../stats/RemoteStoreStatsTestHelper.java | 7 +++- 4 files changed, 67 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/remotestore/stats/RemoteStoreStats.java b/server/src/main/java/org/opensearch/action/admin/cluster/remotestore/stats/RemoteStoreStats.java index a6414e3e0c37e..117ad8ac2bf59 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/remotestore/stats/RemoteStoreStats.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/remotestore/stats/RemoteStoreStats.java @@ -42,7 +42,8 @@ public RemoteRefreshSegmentTracker.Stats getStats() { public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject() .field(Fields.SHARD_ID, remoteSegmentUploadShardStats.shardId) - + .field(Fields.LOCAL_REFRESH_TIMESTAMP, remoteSegmentUploadShardStats.localRefreshClockTimeMs) + .field(Fields.REMOTE_REFRESH_TIMESTAMP, remoteSegmentUploadShardStats.remoteRefreshClockTimeMs) .field(Fields.REFRESH_TIME_LAG_IN_MILLIS, remoteSegmentUploadShardStats.refreshTimeLagMs) .field(Fields.REFRESH_LAG, remoteSegmentUploadShardStats.localRefreshNumber - remoteSegmentUploadShardStats.remoteRefreshNumber) .field(Fields.BYTES_LAG, remoteSegmentUploadShardStats.bytesLag) @@ -104,6 +105,16 @@ static final class Fields { */ static final String REFRESH_TIME_LAG_IN_MILLIS = "refresh_time_lag_in_millis"; + /** + * Last successful local refresh timestamp in milliseconds + */ + static final String LOCAL_REFRESH_TIMESTAMP = "local_refresh_timestamp_in_millis"; + + /** + * Last successful remote refresh timestamp in milliseconds + */ + static final String REMOTE_REFRESH_TIMESTAMP = "remote_refresh_timestamp_in_millis"; + /** * Total write rejections due to remote store backpressure kick in */ diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentTracker.java b/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentTracker.java index 265566cd375fd..919d34258a507 100644 --- a/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentTracker.java +++ b/server/src/main/java/org/opensearch/index/remote/RemoteRefreshSegmentTracker.java @@ -47,6 +47,11 @@ public class RemoteRefreshSegmentTracker { */ private volatile long localRefreshTimeMs; + /** + * The refresh time(clock) of the most recent refresh. + */ + private volatile long localRefreshClockTimeMs; + /** * Sequence number of the most recent remote refresh. */ @@ -57,6 +62,11 @@ public class RemoteRefreshSegmentTracker { */ private volatile long remoteRefreshTimeMs; + /** + * The refresh time(clock) of most recent remote refresh. + */ + private volatile long remoteRefreshClockTimeMs; + /** * Keeps the seq no lag computed so that we do not compute it for every request. */ @@ -167,9 +177,12 @@ public RemoteRefreshSegmentTracker( ) { this.shardId = shardId; // Both the local refresh time and remote refresh time are set with current time to give consistent view of time lag when it arises. + long currentClockTimeMs = System.currentTimeMillis(); long currentTimeMs = System.nanoTime() / 1_000_000L; localRefreshTimeMs = currentTimeMs; remoteRefreshTimeMs = currentTimeMs; + localRefreshClockTimeMs = currentClockTimeMs; + remoteRefreshClockTimeMs = currentClockTimeMs; uploadBytesMovingAverageReference = new AtomicReference<>(new MovingAverage(uploadBytesMovingAverageWindowSize)); uploadBytesPerSecMovingAverageReference = new AtomicReference<>(new MovingAverage(uploadBytesPerSecMovingAverageWindowSize)); uploadTimeMsMovingAverageReference = new AtomicReference<>(new MovingAverage(uploadTimeMsMovingAverageWindowSize)); @@ -199,6 +212,10 @@ public long getLocalRefreshTimeMs() { return localRefreshTimeMs; } + public long getLocalRefreshClockTimeMs() { + return localRefreshClockTimeMs; + } + public void updateLocalRefreshTimeMs(long localRefreshTimeMs) { assert localRefreshTimeMs >= this.localRefreshTimeMs : "newLocalRefreshTimeMs=" + localRefreshTimeMs @@ -209,6 +226,10 @@ public void updateLocalRefreshTimeMs(long localRefreshTimeMs) { computeTimeMsLag(); } + public void updateLocalRefreshClockTimeMs(long localRefreshClockTimeMs) { + this.localRefreshClockTimeMs = localRefreshClockTimeMs; + } + long getRemoteRefreshSeqNo() { return remoteRefreshSeqNo; } @@ -227,6 +248,10 @@ long getRemoteRefreshTimeMs() { return remoteRefreshTimeMs; } + long getRemoteRefreshClockTimeMs() { + return remoteRefreshClockTimeMs; + } + public void updateRemoteRefreshTimeMs(long remoteRefreshTimeMs) { assert remoteRefreshTimeMs >= this.remoteRefreshTimeMs : "newRemoteRefreshTimeMs=" + remoteRefreshTimeMs @@ -237,6 +262,10 @@ public void updateRemoteRefreshTimeMs(long remoteRefreshTimeMs) { computeTimeMsLag(); } + public void updateRemoteRefreshClockTimeMs(long remoteRefreshClockTimeMs) { + this.remoteRefreshClockTimeMs = remoteRefreshClockTimeMs; + } + private void computeRefreshSeqNoLag() { refreshSeqNoLag = localRefreshSeqNo - remoteRefreshSeqNo; } @@ -446,6 +475,8 @@ void updateUploadTimeMsMovingAverageWindowSize(int updatedSize) { public RemoteRefreshSegmentTracker.Stats stats() { return new RemoteRefreshSegmentTracker.Stats( shardId, + localRefreshClockTimeMs, + remoteRefreshClockTimeMs, timeMsLag, localRefreshSeqNo, remoteRefreshSeqNo, @@ -473,6 +504,8 @@ public RemoteRefreshSegmentTracker.Stats stats() { public static class Stats implements Writeable { public final ShardId shardId; + public final long localRefreshClockTimeMs; + public final long remoteRefreshClockTimeMs; public final long refreshTimeLagMs; public final long localRefreshNumber; public final long remoteRefreshNumber; @@ -492,6 +525,8 @@ public static class Stats implements Writeable { public Stats( ShardId shardId, + long localRefreshClockTimeMs, + long remoteRefreshClockTimeMs, long refreshTimeLagMs, long localRefreshNumber, long remoteRefreshNumber, @@ -510,6 +545,8 @@ public Stats( long bytesLag ) { this.shardId = shardId; + this.localRefreshClockTimeMs = localRefreshClockTimeMs; + this.remoteRefreshClockTimeMs = remoteRefreshClockTimeMs; this.refreshTimeLagMs = refreshTimeLagMs; this.localRefreshNumber = localRefreshNumber; this.remoteRefreshNumber = remoteRefreshNumber; @@ -531,6 +568,8 @@ public Stats( public Stats(StreamInput in) throws IOException { try { this.shardId = new ShardId(in); + this.localRefreshClockTimeMs = in.readLong(); + this.remoteRefreshClockTimeMs = in.readLong(); this.refreshTimeLagMs = in.readLong(); this.localRefreshNumber = in.readLong(); this.remoteRefreshNumber = in.readLong(); @@ -555,6 +594,8 @@ public Stats(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { shardId.writeTo(out); + out.writeLong(localRefreshClockTimeMs); + out.writeLong(remoteRefreshClockTimeMs); out.writeLong(refreshTimeLagMs); out.writeLong(localRefreshNumber); out.writeLong(remoteRefreshNumber); diff --git a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java index b24a78971b71f..d7f7373e83bd0 100644 --- a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java +++ b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java @@ -179,7 +179,8 @@ private synchronized void syncSegments(boolean isRetry) { return; } beforeSegmentsSync(isRetry); - long refreshTimeMs = segmentTracker.getLocalRefreshTimeMs(), refreshSeqNo = segmentTracker.getLocalRefreshSeqNo(); + long refreshTimeMs = segmentTracker.getLocalRefreshTimeMs(), refreshClockTimeMs = segmentTracker.getLocalRefreshClockTimeMs(); + long refreshSeqNo = segmentTracker.getLocalRefreshSeqNo(); long bytesBeforeUpload = segmentTracker.getUploadBytesSucceeded(), startTimeInNS = System.nanoTime(); boolean shouldRetry = true; try { @@ -231,7 +232,7 @@ private synchronized void syncSegments(boolean isRetry) { // Start metadata file upload uploadMetadata(localSegmentsPostRefresh, segmentInfos); clearStaleFilesFromLocalSegmentChecksumMap(localSegmentsPostRefresh); - onSuccessfulSegmentsSync(refreshTimeMs, refreshSeqNo, lastRefreshedCheckpoint, checkpoint); + onSuccessfulSegmentsSync(refreshTimeMs, refreshClockTimeMs, refreshSeqNo, lastRefreshedCheckpoint, checkpoint); // At this point since we have uploaded new segments, segment infos and segment metadata file, // along with marking minSeqNoToKeep, upload has succeeded completely. shouldRetry = false; @@ -277,6 +278,7 @@ private void beforeSegmentsSync(boolean isRetry) { private void onSuccessfulSegmentsSync( long refreshTimeMs, + long refreshClockTimeMs, long refreshSeqNo, long lastRefreshedCheckpoint, ReplicationCheckpoint checkpoint @@ -284,7 +286,7 @@ private void onSuccessfulSegmentsSync( // Update latest uploaded segment files name in segment tracker segmentTracker.setLatestUploadedFiles(latestFileNameSizeOnLocalMap.keySet()); // Update the remote refresh time and refresh seq no - updateRemoteRefreshTimeAndSeqNo(refreshTimeMs, refreshSeqNo); + updateRemoteRefreshTimeAndSeqNo(refreshTimeMs, refreshClockTimeMs, refreshSeqNo); // Reset the backoffDelayIterator for the future failures resetBackOffDelayIterator(); // Cancel the scheduled cancellable retry if possible and set it to null @@ -384,6 +386,7 @@ private void deleteStaleCommits() { * Updates the last refresh time and refresh seq no which is seen by local store. */ private void updateLocalRefreshTimeAndSeqNo() { + segmentTracker.updateLocalRefreshClockTimeMs(System.currentTimeMillis()); segmentTracker.updateLocalRefreshTimeMs(System.nanoTime() / 1_000_000L); segmentTracker.updateLocalRefreshSeqNo(segmentTracker.getLocalRefreshSeqNo() + 1); } @@ -391,7 +394,8 @@ private void updateLocalRefreshTimeAndSeqNo() { /** * Updates the last refresh time and refresh seq no which is seen by remote store. */ - private void updateRemoteRefreshTimeAndSeqNo(long refreshTimeMs, long refreshSeqNo) { + private void updateRemoteRefreshTimeAndSeqNo(long refreshTimeMs, long refreshClockTimeMs, long refreshSeqNo) { + segmentTracker.updateRemoteRefreshClockTimeMs(refreshClockTimeMs); segmentTracker.updateRemoteRefreshTimeMs(refreshTimeMs); segmentTracker.updateRemoteRefreshSeqNo(refreshSeqNo); } diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/remotestore/stats/RemoteStoreStatsTestHelper.java b/server/src/test/java/org/opensearch/action/admin/cluster/remotestore/stats/RemoteStoreStatsTestHelper.java index 041a979a687d5..096dfc9145d57 100644 --- a/server/src/test/java/org/opensearch/action/admin/cluster/remotestore/stats/RemoteStoreStatsTestHelper.java +++ b/server/src/test/java/org/opensearch/action/admin/cluster/remotestore/stats/RemoteStoreStatsTestHelper.java @@ -20,11 +20,16 @@ */ public class RemoteStoreStatsTestHelper { static RemoteRefreshSegmentTracker.Stats createPressureTrackerStats(ShardId shardId) { - return new RemoteRefreshSegmentTracker.Stats(shardId, 100, 3, 2, 10, 5, 5, 10, 5, 5, 3, 2, 5, 2, 3, 4, 9); + return new RemoteRefreshSegmentTracker.Stats(shardId, 101, 102, 100, 3, 2, 10, 5, 5, 10, 5, 5, 3, 2, 5, 2, 3, 4, 9); } static void compareStatsResponse(Map statsObject, RemoteRefreshSegmentTracker.Stats pressureTrackerStats) { assertEquals(statsObject.get(RemoteStoreStats.Fields.SHARD_ID), pressureTrackerStats.shardId.toString()); + assertEquals(statsObject.get(RemoteStoreStats.Fields.LOCAL_REFRESH_TIMESTAMP), (int) pressureTrackerStats.localRefreshClockTimeMs); + assertEquals( + statsObject.get(RemoteStoreStats.Fields.REMOTE_REFRESH_TIMESTAMP), + (int) pressureTrackerStats.remoteRefreshClockTimeMs + ); assertEquals(statsObject.get(RemoteStoreStats.Fields.REFRESH_TIME_LAG_IN_MILLIS), (int) pressureTrackerStats.refreshTimeLagMs); assertEquals( statsObject.get(RemoteStoreStats.Fields.REFRESH_LAG), From 606255fa4c251d570f3ed83e02f9d97e144a36af Mon Sep 17 00:00:00 2001 From: Harish Bhakuni Date: Thu, 15 Jun 2023 14:20:12 -0700 Subject: [PATCH 030/109] Create Snapshot Changes for Shallow Copy Snapshot has been backported to 2.x, change version checks from 3.0 to 2.9. (#8092) Signed-off-by: Harish Bhakuni Co-authored-by: Harish Bhakuni --- .../main/java/org/opensearch/cluster/SnapshotsInProgress.java | 4 ++-- .../src/main/java/org/opensearch/snapshots/SnapshotInfo.java | 4 ++-- .../snapshots/SnapshotsInProgressSerializationTests.java | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/opensearch/cluster/SnapshotsInProgress.java b/server/src/main/java/org/opensearch/cluster/SnapshotsInProgress.java index c554c40506b03..d827dc6409778 100644 --- a/server/src/main/java/org/opensearch/cluster/SnapshotsInProgress.java +++ b/server/src/main/java/org/opensearch/cluster/SnapshotsInProgress.java @@ -300,7 +300,7 @@ private Entry(StreamInput in) throws IOException { dataStreams = in.readStringList(); source = in.readOptionalWriteable(SnapshotId::new); clones = in.readMap(RepositoryShardId::new, ShardSnapshotStatus::readFrom); - if (in.getVersion().onOrAfter(Version.V_3_0_0)) { + if (in.getVersion().onOrAfter(Version.V_2_9_0)) { remoteStoreIndexShallowCopy = in.readBoolean(); } else { remoteStoreIndexShallowCopy = false; @@ -736,7 +736,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeStringCollection(dataStreams); out.writeOptionalWriteable(source); out.writeMap(clones, (o, v) -> v.writeTo(o), (o, v) -> v.writeTo(o)); - if (out.getVersion().onOrAfter(Version.V_3_0_0)) { + if (out.getVersion().onOrAfter(Version.V_2_9_0)) { out.writeBoolean(remoteStoreIndexShallowCopy); } } diff --git a/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java b/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java index a3744a38307fd..1619f36738f7b 100644 --- a/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java +++ b/server/src/main/java/org/opensearch/snapshots/SnapshotInfo.java @@ -420,7 +420,7 @@ public SnapshotInfo(final StreamInput in) throws IOException { includeGlobalState = in.readOptionalBoolean(); userMetadata = in.readMap(); dataStreams = in.readStringList(); - if (in.getVersion().onOrAfter(Version.V_3_0_0)) { + if (in.getVersion().onOrAfter(Version.V_2_9_0)) { remoteStoreIndexShallowCopy = in.readOptionalBoolean(); } } @@ -866,7 +866,7 @@ public void writeTo(final StreamOutput out) throws IOException { out.writeOptionalBoolean(includeGlobalState); out.writeMap(userMetadata); out.writeStringCollection(dataStreams); - if (out.getVersion().onOrAfter(Version.V_3_0_0)) { + if (out.getVersion().onOrAfter(Version.V_2_9_0)) { out.writeOptionalBoolean(remoteStoreIndexShallowCopy); } } diff --git a/server/src/test/java/org/opensearch/snapshots/SnapshotsInProgressSerializationTests.java b/server/src/test/java/org/opensearch/snapshots/SnapshotsInProgressSerializationTests.java index 91f7021ef565e..bf491cacc88d1 100644 --- a/server/src/test/java/org/opensearch/snapshots/SnapshotsInProgressSerializationTests.java +++ b/server/src/test/java/org/opensearch/snapshots/SnapshotsInProgressSerializationTests.java @@ -215,7 +215,7 @@ public void testSerDeRemoteStoreIndexShallowCopy() throws IOException { } } try (StreamInput in = out.bytes().streamInput()) { - in.setVersion(Version.V_3_0_0); + in.setVersion(Version.V_2_9_0); actualSnapshotsInProgress = new SnapshotsInProgress(in); assert in.available() == 0; for (Entry curr_entry : actualSnapshotsInProgress.entries()) { From 0b775e79f637c7225ae908e368f1a62ce9ece011 Mon Sep 17 00:00:00 2001 From: Chris White Date: Thu, 15 Jun 2023 18:23:51 -0700 Subject: [PATCH 031/109] Added insecureString options to allow to easier migration (#5496) Signed-off-by: Chris White --- CHANGELOG.md | 1 + .../common/settings/SecureSetting.java | 43 ++++++- .../common/settings/InsecureSettingTests.java | 114 ++++++++++++++++++ 3 files changed, 152 insertions(+), 6 deletions(-) create mode 100644 server/src/test/java/org/opensearch/common/settings/InsecureSettingTests.java diff --git a/CHANGELOG.md b/CHANGELOG.md index b964dc5844757..b6f2b534eb379 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -120,6 +120,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Compress and cache cluster state during validate join request ([#7321](https://github.com/opensearch-project/OpenSearch/pull/7321)) - [Snapshot Interop] Add Changes in Create Snapshot Flow for remote store interoperability. ([#7118](https://github.com/opensearch-project/OpenSearch/pull/7118)) - Add new query profile collector fields with concurrent search execution ([#7898](https://github.com/opensearch-project/OpenSearch/pull/7898)) +- Allow insecure string settings to warn-log usage and advise to migration of a newer secure variant ([#5496](https://github.com/opensearch-project/OpenSearch/pull/5496)) ### Deprecated diff --git a/server/src/main/java/org/opensearch/common/settings/SecureSetting.java b/server/src/main/java/org/opensearch/common/settings/SecureSetting.java index 13abbcdc706e0..8e304a506399a 100644 --- a/server/src/main/java/org/opensearch/common/settings/SecureSetting.java +++ b/server/src/main/java/org/opensearch/common/settings/SecureSetting.java @@ -40,6 +40,9 @@ import java.util.EnumSet; import java.util.Set; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + /** * A secure setting. * @@ -161,7 +164,17 @@ public static Setting secureString(String name, Setting insecureString(String name) { - return new InsecureStringSetting(name); + return insecureString(name, null, false); + } + + /** + * A setting which contains a sensitive string, but usage is logged when found outside secure settings, regardless + * of the opensearch.allow_insecure_settings value. Typically. this is used when migrating old legacy settings + * to secure variants while preserving existing functionality. + * @see #insecureString(String) + */ + public static Setting insecureString(String name, String secureName, boolean allowWithWarning) { + return new InsecureStringSetting(name, secureName, allowWithWarning); } /** @@ -206,22 +219,40 @@ SecureString getFallback(Settings settings) { * @opensearch.internal */ private static class InsecureStringSetting extends Setting { + private static final Logger LOG = LogManager.getLogger(InsecureStringSetting.class); private final String name; + private final String secureName; + private final boolean allowWithWarning; + + private boolean warningLogged; - private InsecureStringSetting(String name) { + private InsecureStringSetting(String name, String secureName, boolean allowWithWarning) { super(name, "", SecureString::new, Property.Deprecated, Property.Filtered, Property.NodeScope); this.name = name; + this.secureName = secureName; + this.allowWithWarning = allowWithWarning; } @Override public SecureString get(Settings settings) { - if (ALLOW_INSECURE_SETTINGS == false && exists(settings)) { - throw new IllegalArgumentException( - "Setting [" + name + "] is insecure, " + "but property [allow_insecure_settings] is not set" - ); + if (exists(settings)) { + logUsage(); + + if (ALLOW_INSECURE_SETTINGS == false && this.allowWithWarning == false) { + throw new IllegalArgumentException( + "Setting [" + name + "] is insecure, " + "but property [allow_insecure_settings] is not set" + ); + } } return super.get(settings); } + + private synchronized void logUsage() { + if (!this.warningLogged) { + LOG.warn("Setting [{}] is insecure, but a secure variant [{}] is advised to be used instead", this.name, this.secureName); + this.warningLogged = true; + } + } } /** diff --git a/server/src/test/java/org/opensearch/common/settings/InsecureSettingTests.java b/server/src/test/java/org/opensearch/common/settings/InsecureSettingTests.java new file mode 100644 index 0000000000000..cca8deda2964d --- /dev/null +++ b/server/src/test/java/org/opensearch/common/settings/InsecureSettingTests.java @@ -0,0 +1,114 @@ +/* + * 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.common.settings; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.core.LogEvent; +import org.apache.logging.log4j.core.appender.AbstractAppender; +import org.apache.logging.log4j.core.config.Property; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; + +import org.opensearch.common.logging.Loggers; +import org.opensearch.test.OpenSearchTestCase; + +public class InsecureSettingTests extends OpenSearchTestCase { + private List rootLogMsgs = new ArrayList<>(); + private AbstractAppender rootAppender; + + protected void assertSettingWarning() { + assertWarnings( + "[setting.name] setting was deprecated in OpenSearch and will be removed in a future release! See the breaking changes documentation for the next major version." + ); + + Assert.assertTrue( + this.rootLogMsgs.stream() + .anyMatch( + msg -> msg.equals( + "Setting [setting.name] is insecure, but a secure variant [setting.name_secure] is advised to be used instead" + ) + ) + ); + } + + @Before + public void addInsecureSettingsAppender() { + this.rootLogMsgs.clear(); + rootAppender = new AbstractAppender("root", null, null, true, Property.EMPTY_ARRAY) { + @Override + public void append(LogEvent event) { + String message = event.getMessage().getFormattedMessage(); + InsecureSettingTests.this.rootLogMsgs.add(message); + } + }; + Loggers.addAppender(LogManager.getRootLogger(), rootAppender); + rootAppender.start(); + } + + @After + public void removeInsecureSettingsAppender() { + Loggers.removeAppender(LogManager.getRootLogger(), rootAppender); + } + + public void testShouldRaiseExceptionByDefault() { + final var setting = SecureSetting.insecureString("setting.name"); + final var settings = Settings.builder().put(setting.getKey(), "value").build(); + + final var exception = Assert.assertThrows(IllegalArgumentException.class, () -> setting.get(settings)); + Assert.assertEquals( + "Setting [setting.name] is insecure, but property [allow_insecure_settings] is not set", + exception.getMessage() + ); + } + + public void testShouldLogWarn() { + final var setting = SecureSetting.insecureString("setting.name", "setting.name_secure", true); + final var settings = Settings.builder().put(setting.getKey(), "value").build(); + + Assert.assertEquals("value", setting.get(settings).toString()); + assertSettingWarning(); + } + + public void testShouldLogWarnOnce() { + final var setting = SecureSetting.insecureString("setting.name", "setting.name_secure", true); + final var settings = Settings.builder().put(setting.getKey(), "value").build(); + + Assert.assertEquals("value", setting.get(settings).toString()); + Assert.assertEquals("value", setting.get(settings).toString()); + assertSettingWarning(); + + // check that warning was only logged once + Assert.assertEquals(1, this.rootLogMsgs.stream().filter(msg -> msg.contains("but a secure variant")).count()); + + } + + public void testShouldRaiseExceptionIfConfigured() { + final var setting = SecureSetting.insecureString("setting.name", "setting.name_secure", false); + final var settings = Settings.builder().put(setting.getKey(), "value").build(); + + final var exception = Assert.assertThrows(IllegalArgumentException.class, () -> setting.get(settings)); + Assert.assertEquals( + "Setting [setting.name] is insecure, but property [allow_insecure_settings] is not set", + exception.getMessage() + ); + } + + public void testShouldFallbackToInsecure() { + final var insecureSetting = SecureSetting.insecureString("setting.name", "setting.name_secure", true); + final var secureSetting = SecureSetting.secureString("setting.name_secure", insecureSetting); + final var settings = Settings.builder().put(insecureSetting.getKey(), "value").build(); + + Assert.assertEquals("value", secureSetting.get(settings).toString()); + assertSettingWarning(); + } +} From 8eea7b986453271cd227a7b98ee09e70d5e28634 Mon Sep 17 00:00:00 2001 From: Nick Knize Date: Fri, 16 Jun 2023 09:38:04 -0500 Subject: [PATCH 032/109] [Refactor] Serverless common classes to libraries (#8065) This commit refactors common classes from the server module common package to the common and core library so they can be used across serverless and cloud native implementations without depending on the server module. Signed-off-by: Nicholas Walter Knize --- .../org/opensearch/common/CheckedBiConsumer.java | 0 .../org/opensearch/common/CheckedBiFunction.java | 0 .../java/org/opensearch/common/CheckedSupplier.java | 0 .../org/opensearch/common/CheckedTriFunction.java | 0 .../main/java/org/opensearch/common/Classes.java | 0 .../main/java/org/opensearch/common/Explicit.java | 0 .../common/ExponentiallyWeightedMovingAverage.java | 0 .../java/org/opensearch/common/LocalTimeOffset.java | 0 .../org/opensearch/common/MacAddressProvider.java | 0 .../java/org/opensearch/common/NamedRegistry.java | 0 .../opensearch/common/RandomBasedUUIDGenerator.java | 0 .../org/opensearch/common/SecureRandomHolder.java | 0 .../main/java/org/opensearch/common/StopWatch.java | 13 +++++++++++-- .../org/opensearch/common/SuppressLoggerChecks.java | 0 .../opensearch/common/TimeBasedUUIDGenerator.java | 0 .../java/org/opensearch/common/TriConsumer.java | 0 .../java/org/opensearch/common/TriFunction.java | 0 .../java/org/opensearch/common/UUIDGenerator.java | 0 .../src/main/java/org/opensearch/common/UUIDs.java | 0 .../org/opensearch/common/ValidationException.java | 0 .../org/opensearch}/common/lease/Releasable.java | 4 ++-- .../org/opensearch}/common/lease/Releasables.java | 2 +- .../org/opensearch}/common/lease/package-info.java | 2 +- .../matrix/stats/MatrixStatsAggregator.java | 2 +- .../bucket/geogrid/GeoGridAggregator.java | 2 +- .../metrics/AbstractGeoBoundsAggregator.java | 2 +- .../join/aggregations/ParentJoinAggregator.java | 4 ++-- .../netty4/Netty4MessageChannelHandler.java | 2 +- .../transport/netty4/Netty4Transport.java | 2 +- .../transport/nio/TcpReadWriteHandler.java | 4 ++-- .../org/opensearch/index/IndexingPressureIT.java | 2 +- .../index/SegmentReplicationPressureIT.java | 2 +- .../opensearch/index/ShardIndexingPressureIT.java | 2 +- .../index/ShardIndexingPressureSettingsIT.java | 2 +- .../replication/SegmentReplicationBaseIT.java | 2 +- .../indices/replication/SegmentReplicationIT.java | 2 +- .../indices/state/ReopenWhileClosingIT.java | 2 +- .../TransportVerifyShardBeforeCloseAction.java | 2 +- .../TransportVerifyShardIndexBlockAction.java | 2 +- .../query/TransportValidateQueryAction.java | 2 +- .../opensearch/action/bulk/TransportBulkAction.java | 2 +- .../action/bulk/TransportShardBulkAction.java | 2 +- .../action/explain/TransportExplainAction.java | 2 +- .../action/search/AbstractSearchAsyncAction.java | 4 ++-- .../action/search/CanMatchPreFilterSearchPhase.java | 2 +- .../action/search/QueryPhaseResultConsumer.java | 4 ++-- .../action/search/SearchPhaseContext.java | 2 +- .../opensearch/action/support/TransportAction.java | 4 ++-- .../replication/PendingReplicationActions.java | 2 +- .../replication/TransportReplicationAction.java | 4 ++-- .../support/replication/TransportWriteAction.java | 2 +- .../src/main/java/org/opensearch/client/Client.java | 2 +- .../cluster/coordination/Coordinator.java | 2 +- .../coordination/ElectionSchedulerFactory.java | 2 +- .../cluster/coordination/LeaderChecker.java | 2 +- .../cluster/coordination/PreVoteCollector.java | 2 +- .../cluster/service/ClusterApplierService.java | 12 ++++++------ .../common/bytes/ReleasableBytesReference.java | 2 +- .../common/component/LifecycleComponent.java | 2 +- .../common/compress/DeflateCompressor.java | 2 +- .../common/concurrent/RefCountedReleasable.java | 2 +- .../io/stream/ReleasableBytesStreamOutput.java | 4 ++-- .../common/lucene/index/FreqTermsEnum.java | 4 ++-- .../org/opensearch/common/recycler/Recycler.java | 2 +- .../opensearch/common/util/AbstractBigArray.java | 2 +- .../org/opensearch/common/util/AbstractHash.java | 2 +- .../common/util/AbstractPagedHashMap.java | 2 +- .../java/org/opensearch/common/util/BigArray.java | 2 +- .../java/org/opensearch/common/util/BigArrays.java | 4 ++-- .../java/org/opensearch/common/util/BitArray.java | 4 ++-- .../org/opensearch/common/util/BytesRefHash.java | 4 ++-- .../java/org/opensearch/common/util/LongHash.java | 2 +- .../org/opensearch/common/util/LongLongHash.java | 2 +- .../common/util/LongObjectPagedHashMap.java | 2 +- .../common/util/concurrent/KeyedLock.java | 2 +- .../common/util/concurrent/ReleasableLock.java | 2 +- .../java/org/opensearch/env/NodeEnvironment.java | 2 +- .../org/opensearch/gateway/AsyncShardFetch.java | 2 +- .../org/opensearch/gateway/GatewayAllocator.java | 2 +- .../gateway/PersistedClusterStateService.java | 4 ++-- .../org/opensearch/http/DefaultRestChannel.java | 4 ++-- .../java/org/opensearch/index/IndexingPressure.java | 2 +- .../opensearch/index/IndexingPressureService.java | 2 +- .../org/opensearch/index/ShardIndexingPressure.java | 2 +- .../java/org/opensearch/index/engine/Engine.java | 4 ++-- .../org/opensearch/index/engine/InternalEngine.java | 2 +- .../org/opensearch/index/engine/LiveVersionMap.java | 2 +- .../opensearch/index/engine/SoftDeletesPolicy.java | 2 +- .../opensearch/index/fielddata/LeafFieldData.java | 2 +- .../index/seqno/RetentionLeaseActions.java | 2 +- .../java/org/opensearch/index/shard/IndexShard.java | 4 ++-- .../index/shard/IndexShardOperationPermits.java | 2 +- .../opensearch/index/shard/RefreshListeners.java | 2 +- .../opensearch/index/translog/RemoteFsTranslog.java | 4 ++-- .../org/opensearch/index/translog/Translog.java | 4 ++-- .../index/translog/TranslogDeletionPolicy.java | 2 +- .../opensearch/index/translog/TranslogWriter.java | 2 +- .../java/org/opensearch/indices/IndicesService.java | 2 +- .../opensearch/indices/RunUnderPrimaryPermit.java | 2 +- .../fielddata/cache/IndicesFieldDataCache.java | 2 +- .../LocalStorePeerRecoverySourceHandler.java | 2 +- .../indices/recovery/MultiFileWriter.java | 2 +- .../indices/recovery/RecoverySourceHandler.java | 4 ++-- .../RemoteStorePeerRecoverySourceHandler.java | 2 +- .../replication/SegmentFileTransferHandler.java | 2 +- server/src/main/java/org/opensearch/node/Node.java | 2 +- .../repositories/blobstore/BlobStoreRepository.java | 2 +- .../main/java/org/opensearch/rest/RestResponse.java | 2 +- .../org/opensearch/search/DefaultSearchContext.java | 2 +- .../java/org/opensearch/search/SearchService.java | 4 ++-- .../opensearch/search/aggregations/Aggregator.java | 2 +- .../aggregations/bucket/BucketsAggregator.java | 2 +- .../bucket/composite/BinaryValuesSource.java | 2 +- .../bucket/composite/CompositeAggregator.java | 2 +- .../composite/CompositeValuesCollectorQueue.java | 4 ++-- .../bucket/composite/DoubleValuesSource.java | 2 +- .../bucket/composite/GlobalOrdinalValuesSource.java | 2 +- .../bucket/composite/LongValuesSource.java | 2 +- .../composite/SingleDimensionValuesSource.java | 2 +- .../bucket/composite/UnsignedLongValuesSource.java | 2 +- .../histogram/AbstractHistogramAggregator.java | 2 +- .../histogram/AutoDateHistogramAggregator.java | 2 +- .../bucket/histogram/DateHistogramAggregator.java | 2 +- .../histogram/DateRangeHistogramAggregator.java | 2 +- .../histogram/VariableWidthHistogramAggregator.java | 4 ++-- .../bucket/sampler/BestDocsDeferringCollector.java | 4 ++-- .../sampler/DiversifiedMapSamplerAggregator.java | 2 +- .../bucket/sampler/SamplerAggregator.java | 2 +- .../bucket/terms/BytesKeyedBucketOrds.java | 4 ++-- .../terms/GlobalOrdinalsStringTermsAggregator.java | 4 ++-- .../bucket/terms/LongKeyedBucketOrds.java | 2 +- .../bucket/terms/LongRareTermsAggregator.java | 2 +- .../bucket/terms/MapStringTermsAggregator.java | 4 ++-- .../bucket/terms/MultiTermsAggregator.java | 2 +- .../bucket/terms/NumericTermsAggregator.java | 4 ++-- .../bucket/terms/SignificanceLookup.java | 4 ++-- .../terms/SignificantTextAggregatorFactory.java | 2 +- .../bucket/terms/StringRareTermsAggregator.java | 2 +- .../metrics/AbstractHDRPercentilesAggregator.java | 2 +- .../metrics/AbstractHyperLogLogPlusPlus.java | 2 +- .../AbstractTDigestPercentilesAggregator.java | 2 +- .../search/aggregations/metrics/AvgAggregator.java | 2 +- .../aggregations/metrics/CardinalityAggregator.java | 4 ++-- .../metrics/ExtendedStatsAggregator.java | 2 +- .../aggregations/metrics/GeoCentroidAggregator.java | 2 +- .../aggregations/metrics/HyperLogLogPlusPlus.java | 4 ++-- .../metrics/HyperLogLogPlusPlusSparse.java | 4 ++-- .../search/aggregations/metrics/MaxAggregator.java | 2 +- .../metrics/MedianAbsoluteDeviationAggregator.java | 2 +- .../search/aggregations/metrics/MinAggregator.java | 2 +- .../metrics/ScriptedMetricAggregator.java | 2 +- .../aggregations/metrics/StatsAggregator.java | 2 +- .../search/aggregations/metrics/SumAggregator.java | 2 +- .../aggregations/metrics/TopHitsAggregator.java | 2 +- .../aggregations/metrics/ValueCountAggregator.java | 2 +- .../aggregations/metrics/WeightedAvgAggregator.java | 2 +- .../search/internal/ContextIndexSearcher.java | 2 +- .../search/internal/PitReaderContext.java | 4 ++-- .../opensearch/search/internal/ReaderContext.java | 4 ++-- .../opensearch/search/internal/SearchContext.java | 4 ++-- .../org/opensearch/search/sort/BucketedSort.java | 4 ++-- .../main/java/org/opensearch/tasks/TaskManager.java | 4 ++-- .../org/opensearch/transport/InboundAggregator.java | 4 ++-- .../org/opensearch/transport/InboundDecoder.java | 2 +- .../org/opensearch/transport/InboundMessage.java | 4 ++-- .../org/opensearch/transport/InboundPipeline.java | 4 ++-- .../org/opensearch/transport/OutboundHandler.java | 4 ++-- .../transport/RequestHandlerRegistry.java | 4 ++-- .../opensearch/transport/TaskTransportChannel.java | 2 +- .../opensearch/transport/TcpTransportChannel.java | 2 +- .../org/opensearch/transport/TransportService.java | 2 +- .../cluster/node/tasks/TaskManagerTestCase.java | 2 +- .../TransportResyncReplicationActionTests.java | 2 +- .../action/search/MockSearchPhaseContext.java | 2 +- .../TransportReplicationActionTests.java | 2 +- ...nsportReplicationAllPermitsAcquisitionTests.java | 2 +- ...ransportWriteActionForIndexingPressureTests.java | 2 +- .../replication/TransportWriteActionTests.java | 2 +- .../coordination/ElectionSchedulerFactoryTests.java | 2 +- .../cluster/coordination/PreVoteCollectorTests.java | 2 +- .../cluster/service/TaskExecutorTests.java | 2 +- .../org/opensearch/common/ReleasablesTests.java | 4 ++-- .../opensearch/common/util/BinarySearcherTests.java | 2 +- .../common/util/concurrent/KeyedLockTests.java | 2 +- .../common/util/concurrent/ReleasableLockTests.java | 2 +- .../opensearch/http/DefaultRestChannelTests.java | 2 +- .../index/IndexingPressureServiceTests.java | 2 +- .../org/opensearch/index/IndexingPressureTests.java | 2 +- ...ardIndexingPressureConcurrentExecutionTests.java | 2 +- .../index/ShardIndexingPressureTests.java | 2 +- .../index/engine/LiveVersionMapTests.java | 2 +- .../index/engine/SoftDeletesPolicyTests.java | 2 +- .../replication/RecoveryDuringReplicationTests.java | 2 +- .../index/seqno/RetentionLeaseActionsTests.java | 2 +- .../shard/IndexShardOperationPermitsTests.java | 2 +- .../org/opensearch/index/shard/IndexShardTests.java | 4 ++-- .../index/shard/RefreshListenersTests.java | 2 +- .../shard/RemoteStoreRefreshListenerTests.java | 2 +- .../shard/SegmentReplicationIndexShardTests.java | 2 +- .../index/translog/TranslogDeletionPolicyTests.java | 2 +- .../LocalStorePeerRecoverySourceHandlerTests.java | 2 +- .../metrics/InternalCardinalityTests.java | 2 +- .../search/sort/BucketedSortTestCase.java | 2 +- .../java/org/opensearch/tasks/TaskManagerTests.java | 4 ++-- .../opensearch/transport/InboundPipelineTests.java | 2 +- .../transport/TransportDecompressorTests.java | 2 +- .../coordination/AbstractCoordinatorTestCase.java | 2 +- .../OpenSearchIndexLevelReplicationTestCase.java | 4 ++-- .../opensearch/index/shard/IndexShardTestCase.java | 2 +- .../search/aggregations/AggregatorTestCase.java | 4 ++-- .../org/opensearch/test/InternalTestCluster.java | 2 +- .../opensearch/transport/nio/MockNioTransport.java | 4 ++-- 212 files changed, 255 insertions(+), 246 deletions(-) rename {server => libs/common}/src/main/java/org/opensearch/common/CheckedBiConsumer.java (100%) rename {server => libs/common}/src/main/java/org/opensearch/common/CheckedBiFunction.java (100%) rename {server => libs/common}/src/main/java/org/opensearch/common/CheckedSupplier.java (100%) rename {server => libs/common}/src/main/java/org/opensearch/common/CheckedTriFunction.java (100%) rename {server => libs/common}/src/main/java/org/opensearch/common/Classes.java (100%) rename {server => libs/common}/src/main/java/org/opensearch/common/Explicit.java (100%) rename {server => libs/common}/src/main/java/org/opensearch/common/ExponentiallyWeightedMovingAverage.java (100%) rename {server => libs/common}/src/main/java/org/opensearch/common/LocalTimeOffset.java (100%) rename {server => libs/common}/src/main/java/org/opensearch/common/MacAddressProvider.java (100%) rename {server => libs/common}/src/main/java/org/opensearch/common/NamedRegistry.java (100%) rename {server => libs/common}/src/main/java/org/opensearch/common/RandomBasedUUIDGenerator.java (100%) rename {server => libs/common}/src/main/java/org/opensearch/common/SecureRandomHolder.java (100%) rename {server => libs/common}/src/main/java/org/opensearch/common/StopWatch.java (96%) rename {server => libs/common}/src/main/java/org/opensearch/common/SuppressLoggerChecks.java (100%) rename {server => libs/common}/src/main/java/org/opensearch/common/TimeBasedUUIDGenerator.java (100%) rename {server => libs/common}/src/main/java/org/opensearch/common/TriConsumer.java (100%) rename {server => libs/common}/src/main/java/org/opensearch/common/TriFunction.java (100%) rename {server => libs/common}/src/main/java/org/opensearch/common/UUIDGenerator.java (100%) rename {server => libs/common}/src/main/java/org/opensearch/common/UUIDs.java (100%) rename {server => libs/common}/src/main/java/org/opensearch/common/ValidationException.java (100%) rename libs/{core/src/main/java/org/opensearch/core => common/src/main/java/org/opensearch}/common/lease/Releasable.java (89%) rename libs/{core/src/main/java/org/opensearch/core => common/src/main/java/org/opensearch}/common/lease/Releasables.java (99%) rename libs/{core/src/main/java/org/opensearch/core => common/src/main/java/org/opensearch}/common/lease/package-info.java (85%) diff --git a/server/src/main/java/org/opensearch/common/CheckedBiConsumer.java b/libs/common/src/main/java/org/opensearch/common/CheckedBiConsumer.java similarity index 100% rename from server/src/main/java/org/opensearch/common/CheckedBiConsumer.java rename to libs/common/src/main/java/org/opensearch/common/CheckedBiConsumer.java diff --git a/server/src/main/java/org/opensearch/common/CheckedBiFunction.java b/libs/common/src/main/java/org/opensearch/common/CheckedBiFunction.java similarity index 100% rename from server/src/main/java/org/opensearch/common/CheckedBiFunction.java rename to libs/common/src/main/java/org/opensearch/common/CheckedBiFunction.java diff --git a/server/src/main/java/org/opensearch/common/CheckedSupplier.java b/libs/common/src/main/java/org/opensearch/common/CheckedSupplier.java similarity index 100% rename from server/src/main/java/org/opensearch/common/CheckedSupplier.java rename to libs/common/src/main/java/org/opensearch/common/CheckedSupplier.java diff --git a/server/src/main/java/org/opensearch/common/CheckedTriFunction.java b/libs/common/src/main/java/org/opensearch/common/CheckedTriFunction.java similarity index 100% rename from server/src/main/java/org/opensearch/common/CheckedTriFunction.java rename to libs/common/src/main/java/org/opensearch/common/CheckedTriFunction.java diff --git a/server/src/main/java/org/opensearch/common/Classes.java b/libs/common/src/main/java/org/opensearch/common/Classes.java similarity index 100% rename from server/src/main/java/org/opensearch/common/Classes.java rename to libs/common/src/main/java/org/opensearch/common/Classes.java diff --git a/server/src/main/java/org/opensearch/common/Explicit.java b/libs/common/src/main/java/org/opensearch/common/Explicit.java similarity index 100% rename from server/src/main/java/org/opensearch/common/Explicit.java rename to libs/common/src/main/java/org/opensearch/common/Explicit.java diff --git a/server/src/main/java/org/opensearch/common/ExponentiallyWeightedMovingAverage.java b/libs/common/src/main/java/org/opensearch/common/ExponentiallyWeightedMovingAverage.java similarity index 100% rename from server/src/main/java/org/opensearch/common/ExponentiallyWeightedMovingAverage.java rename to libs/common/src/main/java/org/opensearch/common/ExponentiallyWeightedMovingAverage.java diff --git a/server/src/main/java/org/opensearch/common/LocalTimeOffset.java b/libs/common/src/main/java/org/opensearch/common/LocalTimeOffset.java similarity index 100% rename from server/src/main/java/org/opensearch/common/LocalTimeOffset.java rename to libs/common/src/main/java/org/opensearch/common/LocalTimeOffset.java diff --git a/server/src/main/java/org/opensearch/common/MacAddressProvider.java b/libs/common/src/main/java/org/opensearch/common/MacAddressProvider.java similarity index 100% rename from server/src/main/java/org/opensearch/common/MacAddressProvider.java rename to libs/common/src/main/java/org/opensearch/common/MacAddressProvider.java diff --git a/server/src/main/java/org/opensearch/common/NamedRegistry.java b/libs/common/src/main/java/org/opensearch/common/NamedRegistry.java similarity index 100% rename from server/src/main/java/org/opensearch/common/NamedRegistry.java rename to libs/common/src/main/java/org/opensearch/common/NamedRegistry.java diff --git a/server/src/main/java/org/opensearch/common/RandomBasedUUIDGenerator.java b/libs/common/src/main/java/org/opensearch/common/RandomBasedUUIDGenerator.java similarity index 100% rename from server/src/main/java/org/opensearch/common/RandomBasedUUIDGenerator.java rename to libs/common/src/main/java/org/opensearch/common/RandomBasedUUIDGenerator.java diff --git a/server/src/main/java/org/opensearch/common/SecureRandomHolder.java b/libs/common/src/main/java/org/opensearch/common/SecureRandomHolder.java similarity index 100% rename from server/src/main/java/org/opensearch/common/SecureRandomHolder.java rename to libs/common/src/main/java/org/opensearch/common/SecureRandomHolder.java diff --git a/server/src/main/java/org/opensearch/common/StopWatch.java b/libs/common/src/main/java/org/opensearch/common/StopWatch.java similarity index 96% rename from server/src/main/java/org/opensearch/common/StopWatch.java rename to libs/common/src/main/java/org/opensearch/common/StopWatch.java index 5bdf083f0247b..dea1b2b695ace 100644 --- a/server/src/main/java/org/opensearch/common/StopWatch.java +++ b/libs/common/src/main/java/org/opensearch/common/StopWatch.java @@ -33,7 +33,6 @@ package org.opensearch.common; import org.opensearch.common.unit.TimeValue; -import org.opensearch.core.common.lease.Releasable; import java.text.NumberFormat; import java.util.LinkedList; @@ -155,7 +154,7 @@ public StopWatch stop() throws IllegalStateException { return this; } - public Releasable timing(String taskName) { + public TimingHandle timing(String taskName) { start(taskName); return this::stop; } @@ -268,4 +267,14 @@ public TimeValue getTime() { } } + /** + * Stops the watch and auto calls close in try-with-resources usage + * + * @opensearch.internal + */ + public interface TimingHandle extends AutoCloseable { + @Override + void close(); + } + } diff --git a/server/src/main/java/org/opensearch/common/SuppressLoggerChecks.java b/libs/common/src/main/java/org/opensearch/common/SuppressLoggerChecks.java similarity index 100% rename from server/src/main/java/org/opensearch/common/SuppressLoggerChecks.java rename to libs/common/src/main/java/org/opensearch/common/SuppressLoggerChecks.java diff --git a/server/src/main/java/org/opensearch/common/TimeBasedUUIDGenerator.java b/libs/common/src/main/java/org/opensearch/common/TimeBasedUUIDGenerator.java similarity index 100% rename from server/src/main/java/org/opensearch/common/TimeBasedUUIDGenerator.java rename to libs/common/src/main/java/org/opensearch/common/TimeBasedUUIDGenerator.java diff --git a/server/src/main/java/org/opensearch/common/TriConsumer.java b/libs/common/src/main/java/org/opensearch/common/TriConsumer.java similarity index 100% rename from server/src/main/java/org/opensearch/common/TriConsumer.java rename to libs/common/src/main/java/org/opensearch/common/TriConsumer.java diff --git a/server/src/main/java/org/opensearch/common/TriFunction.java b/libs/common/src/main/java/org/opensearch/common/TriFunction.java similarity index 100% rename from server/src/main/java/org/opensearch/common/TriFunction.java rename to libs/common/src/main/java/org/opensearch/common/TriFunction.java diff --git a/server/src/main/java/org/opensearch/common/UUIDGenerator.java b/libs/common/src/main/java/org/opensearch/common/UUIDGenerator.java similarity index 100% rename from server/src/main/java/org/opensearch/common/UUIDGenerator.java rename to libs/common/src/main/java/org/opensearch/common/UUIDGenerator.java diff --git a/server/src/main/java/org/opensearch/common/UUIDs.java b/libs/common/src/main/java/org/opensearch/common/UUIDs.java similarity index 100% rename from server/src/main/java/org/opensearch/common/UUIDs.java rename to libs/common/src/main/java/org/opensearch/common/UUIDs.java diff --git a/server/src/main/java/org/opensearch/common/ValidationException.java b/libs/common/src/main/java/org/opensearch/common/ValidationException.java similarity index 100% rename from server/src/main/java/org/opensearch/common/ValidationException.java rename to libs/common/src/main/java/org/opensearch/common/ValidationException.java diff --git a/libs/core/src/main/java/org/opensearch/core/common/lease/Releasable.java b/libs/common/src/main/java/org/opensearch/common/lease/Releasable.java similarity index 89% rename from libs/core/src/main/java/org/opensearch/core/common/lease/Releasable.java rename to libs/common/src/main/java/org/opensearch/common/lease/Releasable.java index 38d153f13b932..30bea6185febc 100644 --- a/libs/core/src/main/java/org/opensearch/core/common/lease/Releasable.java +++ b/libs/common/src/main/java/org/opensearch/common/lease/Releasable.java @@ -30,12 +30,12 @@ * GitHub history for details. */ -package org.opensearch.core.common.lease; +package org.opensearch.common.lease; import java.io.Closeable; /** - * Specialization of {@link AutoCloseable} that may only throw an {@code BaseOpenSearchException}. + * Specialization of {@link AutoCloseable} for calls that might not throw a checked exception. * * @opensearch.internal */ diff --git a/libs/core/src/main/java/org/opensearch/core/common/lease/Releasables.java b/libs/common/src/main/java/org/opensearch/common/lease/Releasables.java similarity index 99% rename from libs/core/src/main/java/org/opensearch/core/common/lease/Releasables.java rename to libs/common/src/main/java/org/opensearch/common/lease/Releasables.java index 2d02b82924bac..31bfa9a2dd7ab 100644 --- a/libs/core/src/main/java/org/opensearch/core/common/lease/Releasables.java +++ b/libs/common/src/main/java/org/opensearch/common/lease/Releasables.java @@ -30,7 +30,7 @@ * GitHub history for details. */ -package org.opensearch.core.common.lease; +package org.opensearch.common.lease; import org.opensearch.common.Nullable; import org.opensearch.common.util.io.IOUtils; diff --git a/libs/core/src/main/java/org/opensearch/core/common/lease/package-info.java b/libs/common/src/main/java/org/opensearch/common/lease/package-info.java similarity index 85% rename from libs/core/src/main/java/org/opensearch/core/common/lease/package-info.java rename to libs/common/src/main/java/org/opensearch/common/lease/package-info.java index ffb6a7e106bb5..f7097486a9c64 100644 --- a/libs/core/src/main/java/org/opensearch/core/common/lease/package-info.java +++ b/libs/common/src/main/java/org/opensearch/common/lease/package-info.java @@ -7,4 +7,4 @@ */ /** Base Releasables package. */ -package org.opensearch.core.common.lease; +package org.opensearch.common.lease; diff --git a/modules/aggs-matrix-stats/src/main/java/org/opensearch/search/aggregations/matrix/stats/MatrixStatsAggregator.java b/modules/aggs-matrix-stats/src/main/java/org/opensearch/search/aggregations/matrix/stats/MatrixStatsAggregator.java index 045b79ed5bcd1..eaf8a3bc6fe43 100644 --- a/modules/aggs-matrix-stats/src/main/java/org/opensearch/search/aggregations/matrix/stats/MatrixStatsAggregator.java +++ b/modules/aggs-matrix-stats/src/main/java/org/opensearch/search/aggregations/matrix/stats/MatrixStatsAggregator.java @@ -35,7 +35,7 @@ import org.apache.lucene.search.ScoreMode; import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.ObjectArray; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.fielddata.NumericDoubleValues; import org.opensearch.search.MultiValueMode; import org.opensearch.search.aggregations.Aggregator; diff --git a/modules/geo/src/main/java/org/opensearch/geo/search/aggregations/bucket/geogrid/GeoGridAggregator.java b/modules/geo/src/main/java/org/opensearch/geo/search/aggregations/bucket/geogrid/GeoGridAggregator.java index 1fa441999a6a2..db07ac8f947e5 100644 --- a/modules/geo/src/main/java/org/opensearch/geo/search/aggregations/bucket/geogrid/GeoGridAggregator.java +++ b/modules/geo/src/main/java/org/opensearch/geo/search/aggregations/bucket/geogrid/GeoGridAggregator.java @@ -34,7 +34,7 @@ import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.SortedNumericDocValues; import org.apache.lucene.search.ScoreMode; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.search.aggregations.Aggregator; import org.opensearch.search.aggregations.AggregatorFactories; import org.opensearch.search.aggregations.CardinalityUpperBound; diff --git a/modules/geo/src/main/java/org/opensearch/geo/search/aggregations/metrics/AbstractGeoBoundsAggregator.java b/modules/geo/src/main/java/org/opensearch/geo/search/aggregations/metrics/AbstractGeoBoundsAggregator.java index 8d467df52626c..2f013b76d5a67 100644 --- a/modules/geo/src/main/java/org/opensearch/geo/search/aggregations/metrics/AbstractGeoBoundsAggregator.java +++ b/modules/geo/src/main/java/org/opensearch/geo/search/aggregations/metrics/AbstractGeoBoundsAggregator.java @@ -10,7 +10,7 @@ import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.DoubleArray; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.search.aggregations.Aggregator; import org.opensearch.search.aggregations.InternalAggregation; import org.opensearch.search.aggregations.metrics.MetricsAggregator; diff --git a/modules/parent-join/src/main/java/org/opensearch/join/aggregations/ParentJoinAggregator.java b/modules/parent-join/src/main/java/org/opensearch/join/aggregations/ParentJoinAggregator.java index 5f9f1e66587f0..0b0ff2b0a382e 100644 --- a/modules/parent-join/src/main/java/org/opensearch/join/aggregations/ParentJoinAggregator.java +++ b/modules/parent-join/src/main/java/org/opensearch/join/aggregations/ParentJoinAggregator.java @@ -44,8 +44,8 @@ import org.opensearch.common.lucene.Lucene; import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.BitArray; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.search.aggregations.Aggregator; import org.opensearch.search.aggregations.AggregatorFactories; import org.opensearch.search.aggregations.CardinalityUpperBound; diff --git a/modules/transport-netty4/src/main/java/org/opensearch/transport/netty4/Netty4MessageChannelHandler.java b/modules/transport-netty4/src/main/java/org/opensearch/transport/netty4/Netty4MessageChannelHandler.java index 255f486fd1cbb..5e01823bf808c 100644 --- a/modules/transport-netty4/src/main/java/org/opensearch/transport/netty4/Netty4MessageChannelHandler.java +++ b/modules/transport-netty4/src/main/java/org/opensearch/transport/netty4/Netty4MessageChannelHandler.java @@ -43,7 +43,7 @@ import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.bytes.ReleasableBytesReference; import org.opensearch.common.util.PageCacheRecycler; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.InboundPipeline; import org.opensearch.transport.Transport; diff --git a/modules/transport-netty4/src/main/java/org/opensearch/transport/netty4/Netty4Transport.java b/modules/transport-netty4/src/main/java/org/opensearch/transport/netty4/Netty4Transport.java index 3f0cf7a757acc..5bf515548eb54 100644 --- a/modules/transport-netty4/src/main/java/org/opensearch/transport/netty4/Netty4Transport.java +++ b/modules/transport-netty4/src/main/java/org/opensearch/transport/netty4/Netty4Transport.java @@ -62,7 +62,7 @@ import org.opensearch.common.util.PageCacheRecycler; import org.opensearch.common.util.concurrent.OpenSearchExecutors; import org.opensearch.common.util.net.NetUtils; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.indices.breaker.CircuitBreakerService; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.Netty4NioSocketChannel; diff --git a/plugins/transport-nio/src/main/java/org/opensearch/transport/nio/TcpReadWriteHandler.java b/plugins/transport-nio/src/main/java/org/opensearch/transport/nio/TcpReadWriteHandler.java index 5464018c08f71..a769ab815e0ea 100644 --- a/plugins/transport-nio/src/main/java/org/opensearch/transport/nio/TcpReadWriteHandler.java +++ b/plugins/transport-nio/src/main/java/org/opensearch/transport/nio/TcpReadWriteHandler.java @@ -38,8 +38,8 @@ import org.opensearch.common.bytes.ReleasableBytesReference; import org.opensearch.common.util.PageCacheRecycler; import org.opensearch.common.util.io.IOUtils; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.nio.BytesWriteHandler; import org.opensearch.nio.InboundChannelBuffer; import org.opensearch.nio.Page; diff --git a/server/src/internalClusterTest/java/org/opensearch/index/IndexingPressureIT.java b/server/src/internalClusterTest/java/org/opensearch/index/IndexingPressureIT.java index b69b860f3b69c..b3040f832f5fd 100644 --- a/server/src/internalClusterTest/java/org/opensearch/index/IndexingPressureIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/index/IndexingPressureIT.java @@ -45,7 +45,7 @@ import org.opensearch.common.UUIDs; import org.opensearch.common.collect.Tuple; import org.opensearch.common.settings.Settings; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; import org.opensearch.plugins.Plugin; import org.opensearch.test.OpenSearchIntegTestCase; diff --git a/server/src/internalClusterTest/java/org/opensearch/index/SegmentReplicationPressureIT.java b/server/src/internalClusterTest/java/org/opensearch/index/SegmentReplicationPressureIT.java index 7b8c6a7feb1fd..f8ed719e0f292 100644 --- a/server/src/internalClusterTest/java/org/opensearch/index/SegmentReplicationPressureIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/index/SegmentReplicationPressureIT.java @@ -14,7 +14,7 @@ import org.opensearch.common.UUIDs; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.IndexShardState; diff --git a/server/src/internalClusterTest/java/org/opensearch/index/ShardIndexingPressureIT.java b/server/src/internalClusterTest/java/org/opensearch/index/ShardIndexingPressureIT.java index f37daabed5abf..e8cc8543d36e9 100644 --- a/server/src/internalClusterTest/java/org/opensearch/index/ShardIndexingPressureIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/index/ShardIndexingPressureIT.java @@ -21,7 +21,7 @@ import org.opensearch.common.UUIDs; import org.opensearch.common.collect.Tuple; import org.opensearch.common.settings.Settings; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; import org.opensearch.index.shard.ShardId; import org.opensearch.indices.IndicesService; diff --git a/server/src/internalClusterTest/java/org/opensearch/index/ShardIndexingPressureSettingsIT.java b/server/src/internalClusterTest/java/org/opensearch/index/ShardIndexingPressureSettingsIT.java index 28ce5f30a52a8..e0fc1e93df686 100644 --- a/server/src/internalClusterTest/java/org/opensearch/index/ShardIndexingPressureSettingsIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/index/ShardIndexingPressureSettingsIT.java @@ -23,7 +23,7 @@ import org.opensearch.common.UUIDs; import org.opensearch.common.collect.Tuple; import org.opensearch.common.settings.Settings; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; import org.opensearch.index.shard.ShardId; import org.opensearch.indices.IndicesService; diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationBaseIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationBaseIT.java index 385f54dd4fd5e..52fe85b51cebd 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationBaseIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationBaseIT.java @@ -18,7 +18,7 @@ import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.common.Nullable; import org.opensearch.common.settings.Settings; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.Index; import org.opensearch.index.IndexModule; import org.opensearch.index.IndexService; diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationIT.java index 153a7802a8f0e..0a593e6149ddd 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationIT.java @@ -46,7 +46,7 @@ import org.opensearch.common.lucene.index.OpenSearchDirectoryReader; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.common.util.FeatureFlags; import org.opensearch.index.IndexModule; import org.opensearch.index.SegmentReplicationPerGroupStats; 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 05a50535b59da..92eefefab7867 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/state/ReopenWhileClosingIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/state/ReopenWhileClosingIT.java @@ -41,7 +41,7 @@ import org.opensearch.common.Glob; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.concurrent.RunOnce; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.core.common.Strings; import org.opensearch.plugins.Plugin; import org.opensearch.test.OpenSearchIntegTestCase; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java b/server/src/main/java/org/opensearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java index 165169e36bf8d..8e2691f2d8e19 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java @@ -48,7 +48,7 @@ import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.settings.Settings; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.ShardId; import org.opensearch.indices.IndicesService; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/readonly/TransportVerifyShardIndexBlockAction.java b/server/src/main/java/org/opensearch/action/admin/indices/readonly/TransportVerifyShardIndexBlockAction.java index af5c26702c8b8..b295977933b25 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/readonly/TransportVerifyShardIndexBlockAction.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/readonly/TransportVerifyShardIndexBlockAction.java @@ -48,7 +48,7 @@ import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.settings.Settings; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.ShardId; import org.opensearch.indices.IndicesService; diff --git a/server/src/main/java/org/opensearch/action/admin/indices/validate/query/TransportValidateQueryAction.java b/server/src/main/java/org/opensearch/action/admin/indices/validate/query/TransportValidateQueryAction.java index ab886d476e7f4..431c5468d1850 100644 --- a/server/src/main/java/org/opensearch/action/admin/indices/validate/query/TransportValidateQueryAction.java +++ b/server/src/main/java/org/opensearch/action/admin/indices/validate/query/TransportValidateQueryAction.java @@ -50,7 +50,7 @@ import org.opensearch.common.Randomness; import org.opensearch.common.inject.Inject; import org.opensearch.common.io.stream.StreamInput; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.IndexNotFoundException; import org.opensearch.index.query.ParsedQuery; import org.opensearch.index.query.QueryShardException; 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 51f347d6c1532..fe9d0f21e01b2 100644 --- a/server/src/main/java/org/opensearch/action/bulk/TransportBulkAction.java +++ b/server/src/main/java/org/opensearch/action/bulk/TransportBulkAction.java @@ -71,7 +71,7 @@ import org.opensearch.common.inject.Inject; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.AtomicArray; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.Index; import org.opensearch.index.IndexNotFoundException; import org.opensearch.index.IndexingPressureService; 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 f9b91ac70355e..ff9226e5e6317 100644 --- a/server/src/main/java/org/opensearch/action/bulk/TransportShardBulkAction.java +++ b/server/src/main/java/org/opensearch/action/bulk/TransportShardBulkAction.java @@ -79,7 +79,7 @@ import org.opensearch.common.util.concurrent.AbstractRunnable; import org.opensearch.common.xcontent.XContentHelper; import org.opensearch.common.xcontent.XContentType; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.index.IndexingPressureService; import org.opensearch.index.SegmentReplicationPressureService; diff --git a/server/src/main/java/org/opensearch/action/explain/TransportExplainAction.java b/server/src/main/java/org/opensearch/action/explain/TransportExplainAction.java index a54ea389d1c20..5752860af4f16 100644 --- a/server/src/main/java/org/opensearch/action/explain/TransportExplainAction.java +++ b/server/src/main/java/org/opensearch/action/explain/TransportExplainAction.java @@ -45,7 +45,7 @@ import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.inject.Inject; import org.opensearch.common.io.stream.Writeable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.IndexService; import org.opensearch.index.engine.Engine; import org.opensearch.index.get.GetResult; 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 c2dd5f639db75..89869582910e5 100644 --- a/server/src/main/java/org/opensearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/opensearch/action/search/AbstractSearchAsyncAction.java @@ -48,8 +48,8 @@ import org.opensearch.common.SetOnce; import org.opensearch.common.util.concurrent.AbstractRunnable; import org.opensearch.common.util.concurrent.AtomicArray; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; 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/CanMatchPreFilterSearchPhase.java b/server/src/main/java/org/opensearch/action/search/CanMatchPreFilterSearchPhase.java index 9694695e4fbbb..ec4d45a0a7124 100644 --- a/server/src/main/java/org/opensearch/action/search/CanMatchPreFilterSearchPhase.java +++ b/server/src/main/java/org/opensearch/action/search/CanMatchPreFilterSearchPhase.java @@ -36,7 +36,7 @@ import org.opensearch.action.ActionListener; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.routing.GroupShardsIterator; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.search.SearchService.CanMatchResponse; import org.opensearch.search.SearchShardTarget; import org.opensearch.search.builder.SearchSourceBuilder; diff --git a/server/src/main/java/org/opensearch/action/search/QueryPhaseResultConsumer.java b/server/src/main/java/org/opensearch/action/search/QueryPhaseResultConsumer.java index 6698f33ea6ab7..1aa18a695d31f 100644 --- a/server/src/main/java/org/opensearch/action/search/QueryPhaseResultConsumer.java +++ b/server/src/main/java/org/opensearch/action/search/QueryPhaseResultConsumer.java @@ -40,8 +40,8 @@ import org.opensearch.common.io.stream.NamedWriteableRegistry; import org.opensearch.common.lucene.search.TopDocsAndMaxScore; import org.opensearch.common.util.concurrent.AbstractRunnable; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.search.SearchPhaseResult; import org.opensearch.search.SearchShardTarget; import org.opensearch.search.aggregations.InternalAggregation.ReduceContextBuilder; 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 04d0dab088d35..04b481249520b 100644 --- a/server/src/main/java/org/opensearch/action/search/SearchPhaseContext.java +++ b/server/src/main/java/org/opensearch/action/search/SearchPhaseContext.java @@ -35,7 +35,7 @@ import org.opensearch.action.OriginalIndices; import org.opensearch.common.Nullable; import org.opensearch.common.util.concurrent.AtomicArray; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.search.SearchPhaseResult; import org.opensearch.search.SearchShardTarget; import org.opensearch.search.internal.InternalSearchResponse; diff --git a/server/src/main/java/org/opensearch/action/support/TransportAction.java b/server/src/main/java/org/opensearch/action/support/TransportAction.java index 7c2b68d972fc1..e2f5ecda101b8 100644 --- a/server/src/main/java/org/opensearch/action/support/TransportAction.java +++ b/server/src/main/java/org/opensearch/action/support/TransportAction.java @@ -39,8 +39,8 @@ import org.opensearch.action.ActionRequestValidationException; import org.opensearch.action.ActionResponse; import org.opensearch.common.util.concurrent.ThreadContext; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.tasks.Task; import org.opensearch.tasks.TaskCancelledException; import org.opensearch.tasks.TaskId; 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 744962a735d74..c3b7ec9a03967 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 @@ -34,7 +34,7 @@ import org.opensearch.action.support.RetryableAction; import org.opensearch.common.util.concurrent.ConcurrentCollections; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.shard.PrimaryShardClosedException; import org.opensearch.index.shard.IndexShardClosedException; import org.opensearch.index.shard.ReplicationGroup; diff --git a/server/src/main/java/org/opensearch/action/support/replication/TransportReplicationAction.java b/server/src/main/java/org/opensearch/action/support/replication/TransportReplicationAction.java index ce492dc70287f..3feb5b23e5540 100644 --- a/server/src/main/java/org/opensearch/action/support/replication/TransportReplicationAction.java +++ b/server/src/main/java/org/opensearch/action/support/replication/TransportReplicationAction.java @@ -67,8 +67,8 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.AbstractRunnable; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.IndexNotFoundException; import org.opensearch.index.IndexService; import org.opensearch.index.seqno.SequenceNumbers; diff --git a/server/src/main/java/org/opensearch/action/support/replication/TransportWriteAction.java b/server/src/main/java/org/opensearch/action/support/replication/TransportWriteAction.java index 1bcac6b7510d6..44e2f34b17377 100644 --- a/server/src/main/java/org/opensearch/action/support/replication/TransportWriteAction.java +++ b/server/src/main/java/org/opensearch/action/support/replication/TransportWriteAction.java @@ -48,7 +48,7 @@ import org.opensearch.common.Nullable; import org.opensearch.common.io.stream.Writeable; import org.opensearch.common.settings.Settings; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.IndexingPressureService; import org.opensearch.index.engine.Engine; import org.opensearch.index.mapper.MapperParsingException; diff --git a/server/src/main/java/org/opensearch/client/Client.java b/server/src/main/java/org/opensearch/client/Client.java index 3257c3efff131..6d9b1f7adb64e 100644 --- a/server/src/main/java/org/opensearch/client/Client.java +++ b/server/src/main/java/org/opensearch/client/Client.java @@ -87,7 +87,7 @@ import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Setting.Property; import org.opensearch.common.settings.Settings; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import java.util.Map; diff --git a/server/src/main/java/org/opensearch/cluster/coordination/Coordinator.java b/server/src/main/java/org/opensearch/cluster/coordination/Coordinator.java index b569a09e2bb9d..546fe2d4643d3 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/Coordinator.java @@ -74,7 +74,7 @@ import org.opensearch.common.xcontent.XContentHelper; import org.opensearch.common.xcontent.XContentType; import org.opensearch.common.xcontent.json.JsonXContent; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.discovery.Discovery; import org.opensearch.discovery.DiscoveryModule; import org.opensearch.discovery.DiscoveryStats; diff --git a/server/src/main/java/org/opensearch/cluster/coordination/ElectionSchedulerFactory.java b/server/src/main/java/org/opensearch/cluster/coordination/ElectionSchedulerFactory.java index ee53860837e7d..fa7927001effc 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/ElectionSchedulerFactory.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/ElectionSchedulerFactory.java @@ -41,7 +41,7 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.AbstractRunnable; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.threadpool.ThreadPool; import org.opensearch.threadpool.ThreadPool.Names; diff --git a/server/src/main/java/org/opensearch/cluster/coordination/LeaderChecker.java b/server/src/main/java/org/opensearch/cluster/coordination/LeaderChecker.java index a3a2760af8463..06b722069e27a 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/LeaderChecker.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/LeaderChecker.java @@ -44,7 +44,7 @@ import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.monitor.NodeHealthService; import org.opensearch.monitor.StatusInfo; import org.opensearch.threadpool.ThreadPool.Names; diff --git a/server/src/main/java/org/opensearch/cluster/coordination/PreVoteCollector.java b/server/src/main/java/org/opensearch/cluster/coordination/PreVoteCollector.java index 3cc9cd119c1ab..2cf7ebcc850a5 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/PreVoteCollector.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/PreVoteCollector.java @@ -41,7 +41,7 @@ import org.opensearch.common.Nullable; import org.opensearch.common.collect.Tuple; import org.opensearch.common.io.stream.StreamInput; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.monitor.NodeHealthService; import org.opensearch.monitor.StatusInfo; import org.opensearch.threadpool.ThreadPool.Names; diff --git a/server/src/main/java/org/opensearch/cluster/service/ClusterApplierService.java b/server/src/main/java/org/opensearch/cluster/service/ClusterApplierService.java index 3dd9c8c8b790d..7f1c9f01f7e6f 100644 --- a/server/src/main/java/org/opensearch/cluster/service/ClusterApplierService.java +++ b/server/src/main/java/org/opensearch/cluster/service/ClusterApplierService.java @@ -50,13 +50,13 @@ import org.opensearch.common.Nullable; import org.opensearch.common.Priority; import org.opensearch.common.StopWatch; +import org.opensearch.common.StopWatch.TimingHandle; import org.opensearch.common.component.AbstractLifecycleComponent; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.OpenSearchExecutors; -import org.opensearch.core.common.lease.Releasable; import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; import org.opensearch.common.util.concurrent.PrioritizedOpenSearchThreadPoolExecutor; import org.opensearch.common.util.concurrent.ThreadContext; @@ -444,7 +444,7 @@ private void runTask(UpdateTask task) { final StopWatch stopWatch = new StopWatch(); final ClusterState newClusterState; try { - try (Releasable ignored = stopWatch.timing("running task [" + task.source + ']')) { + try (TimingHandle ignored = stopWatch.timing("running task [" + task.source + ']')) { newClusterState = task.apply(previousClusterState); } } catch (Exception e) { @@ -544,7 +544,7 @@ private void applyChanges(UpdateTask task, ClusterState previousClusterState, Cl } logger.trace("connecting to nodes of cluster state with version {}", newClusterState.version()); - try (Releasable ignored = stopWatch.timing("connecting to new nodes")) { + try (TimingHandle ignored = stopWatch.timing("connecting to new nodes")) { connectToNodesAndWait(newClusterState); } @@ -552,7 +552,7 @@ private void applyChanges(UpdateTask task, ClusterState previousClusterState, Cl if (clusterChangedEvent.state().blocks().disableStatePersistence() == false && clusterChangedEvent.metadataChanged()) { logger.debug("applying settings from cluster state with version {}", newClusterState.version()); final Settings incomingSettings = clusterChangedEvent.state().metadata().settings(); - try (Releasable ignored = stopWatch.timing("applying settings")) { + try (TimingHandle ignored = stopWatch.timing("applying settings")) { clusterSettings.applySettings(incomingSettings); } } @@ -602,7 +602,7 @@ private static void callClusterStateAppliers( ) { for (ClusterStateApplier applier : clusterStateAppliers) { logger.trace("calling [{}] with change to version [{}]", applier, clusterChangedEvent.state().version()); - try (Releasable ignored = stopWatch.timing("running applier [" + applier + "]")) { + try (TimingHandle ignored = stopWatch.timing("running applier [" + applier + "]")) { applier.applyClusterState(clusterChangedEvent); } } @@ -621,7 +621,7 @@ private void callClusterStateListener( for (ClusterStateListener listener : listeners) { try { logger.trace("calling [{}] with change to version [{}]", listener, clusterChangedEvent.state().version()); - try (Releasable ignored = stopWatch.timing("notifying listener [" + listener + "]")) { + try (TimingHandle ignored = stopWatch.timing("notifying listener [" + listener + "]")) { listener.clusterChanged(clusterChangedEvent); } } catch (Exception ex) { diff --git a/server/src/main/java/org/opensearch/common/bytes/ReleasableBytesReference.java b/server/src/main/java/org/opensearch/common/bytes/ReleasableBytesReference.java index 7c5ef272d637f..5345c46943518 100644 --- a/server/src/main/java/org/opensearch/common/bytes/ReleasableBytesReference.java +++ b/server/src/main/java/org/opensearch/common/bytes/ReleasableBytesReference.java @@ -36,7 +36,7 @@ import org.apache.lucene.util.BytesRefIterator; import org.opensearch.common.concurrent.RefCountedReleasable; import org.opensearch.common.io.stream.StreamInput; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.core.xcontent.XContentBuilder; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/common/component/LifecycleComponent.java b/server/src/main/java/org/opensearch/common/component/LifecycleComponent.java index fc31e8ae3a35c..984d55df1bdfa 100644 --- a/server/src/main/java/org/opensearch/common/component/LifecycleComponent.java +++ b/server/src/main/java/org/opensearch/common/component/LifecycleComponent.java @@ -32,7 +32,7 @@ package org.opensearch.common.component; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; /** * Base interface for a lifecycle component. diff --git a/server/src/main/java/org/opensearch/common/compress/DeflateCompressor.java b/server/src/main/java/org/opensearch/common/compress/DeflateCompressor.java index 07d34f07d4693..476ad818ab5ac 100644 --- a/server/src/main/java/org/opensearch/common/compress/DeflateCompressor.java +++ b/server/src/main/java/org/opensearch/common/compress/DeflateCompressor.java @@ -35,7 +35,7 @@ import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.core.Assertions; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import java.io.BufferedInputStream; import java.io.BufferedOutputStream; diff --git a/server/src/main/java/org/opensearch/common/concurrent/RefCountedReleasable.java b/server/src/main/java/org/opensearch/common/concurrent/RefCountedReleasable.java index b614c5e08ef22..77d406a9bb4a3 100644 --- a/server/src/main/java/org/opensearch/common/concurrent/RefCountedReleasable.java +++ b/server/src/main/java/org/opensearch/common/concurrent/RefCountedReleasable.java @@ -14,7 +14,7 @@ package org.opensearch.common.concurrent; import org.opensearch.common.util.concurrent.AbstractRefCounted; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; /** * Decorator class that wraps an object reference as a {@link AbstractRefCounted} instance. diff --git a/server/src/main/java/org/opensearch/common/io/stream/ReleasableBytesStreamOutput.java b/server/src/main/java/org/opensearch/common/io/stream/ReleasableBytesStreamOutput.java index d716e099cec5c..752ae353cb394 100644 --- a/server/src/main/java/org/opensearch/common/io/stream/ReleasableBytesStreamOutput.java +++ b/server/src/main/java/org/opensearch/common/io/stream/ReleasableBytesStreamOutput.java @@ -35,8 +35,8 @@ import org.opensearch.common.bytes.ReleasableBytesReference; import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.PageCacheRecycler; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; /** * An bytes stream output that allows providing a {@link BigArrays} instance diff --git a/server/src/main/java/org/opensearch/common/lucene/index/FreqTermsEnum.java b/server/src/main/java/org/opensearch/common/lucene/index/FreqTermsEnum.java index 2ebbf8c69ad5a..906ca6a098c2e 100644 --- a/server/src/main/java/org/opensearch/common/lucene/index/FreqTermsEnum.java +++ b/server/src/main/java/org/opensearch/common/lucene/index/FreqTermsEnum.java @@ -41,8 +41,8 @@ import org.opensearch.common.util.BytesRefHash; import org.opensearch.common.util.IntArray; import org.opensearch.common.util.LongArray; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/common/recycler/Recycler.java b/server/src/main/java/org/opensearch/common/recycler/Recycler.java index af09c0f0e2eab..0b0c98772a77c 100644 --- a/server/src/main/java/org/opensearch/common/recycler/Recycler.java +++ b/server/src/main/java/org/opensearch/common/recycler/Recycler.java @@ -32,7 +32,7 @@ package org.opensearch.common.recycler; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; /** * A recycled object, note, implementations should support calling obtain and then recycle diff --git a/server/src/main/java/org/opensearch/common/util/AbstractBigArray.java b/server/src/main/java/org/opensearch/common/util/AbstractBigArray.java index 0ee1f4b48b37e..e8fd3990befa8 100644 --- a/server/src/main/java/org/opensearch/common/util/AbstractBigArray.java +++ b/server/src/main/java/org/opensearch/common/util/AbstractBigArray.java @@ -34,7 +34,7 @@ import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.RamUsageEstimator; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.common.recycler.Recycler; import java.lang.reflect.Array; diff --git a/server/src/main/java/org/opensearch/common/util/AbstractHash.java b/server/src/main/java/org/opensearch/common/util/AbstractHash.java index d77e63d1a659c..ce12a4f5d1bf2 100644 --- a/server/src/main/java/org/opensearch/common/util/AbstractHash.java +++ b/server/src/main/java/org/opensearch/common/util/AbstractHash.java @@ -32,7 +32,7 @@ package org.opensearch.common.util; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; /** * Base implementation for {@link BytesRefHash} and {@link LongHash}, or any class that diff --git a/server/src/main/java/org/opensearch/common/util/AbstractPagedHashMap.java b/server/src/main/java/org/opensearch/common/util/AbstractPagedHashMap.java index e61bb8b45e711..1ff3038297008 100644 --- a/server/src/main/java/org/opensearch/common/util/AbstractPagedHashMap.java +++ b/server/src/main/java/org/opensearch/common/util/AbstractPagedHashMap.java @@ -33,7 +33,7 @@ package org.opensearch.common.util; import com.carrotsearch.hppc.BitMixer; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; /** * Base implementation for a hash table that is paged, recycles arrays and grows in-place. diff --git a/server/src/main/java/org/opensearch/common/util/BigArray.java b/server/src/main/java/org/opensearch/common/util/BigArray.java index 35467c6fa1ad3..f6cac6374bbeb 100644 --- a/server/src/main/java/org/opensearch/common/util/BigArray.java +++ b/server/src/main/java/org/opensearch/common/util/BigArray.java @@ -33,7 +33,7 @@ package org.opensearch.common.util; import org.apache.lucene.util.Accountable; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; /** * Base abstraction of an array. diff --git a/server/src/main/java/org/opensearch/common/util/BigArrays.java b/server/src/main/java/org/opensearch/common/util/BigArrays.java index 5351494d0abc9..475b1d4c81ad2 100644 --- a/server/src/main/java/org/opensearch/common/util/BigArrays.java +++ b/server/src/main/java/org/opensearch/common/util/BigArrays.java @@ -39,8 +39,8 @@ import org.opensearch.common.breaker.CircuitBreaker; import org.opensearch.common.breaker.CircuitBreakingException; import org.opensearch.common.recycler.Recycler; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.indices.breaker.CircuitBreakerService; import java.util.Arrays; diff --git a/server/src/main/java/org/opensearch/common/util/BitArray.java b/server/src/main/java/org/opensearch/common/util/BitArray.java index 4d701a9cbe2c8..b75a5d07e70d7 100644 --- a/server/src/main/java/org/opensearch/common/util/BitArray.java +++ b/server/src/main/java/org/opensearch/common/util/BitArray.java @@ -32,8 +32,8 @@ package org.opensearch.common.util; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; /** * A bit array that is implemented using a growing {@link LongArray} diff --git a/server/src/main/java/org/opensearch/common/util/BytesRefHash.java b/server/src/main/java/org/opensearch/common/util/BytesRefHash.java index 03275adb278ad..eb44b30a7e36c 100644 --- a/server/src/main/java/org/opensearch/common/util/BytesRefHash.java +++ b/server/src/main/java/org/opensearch/common/util/BytesRefHash.java @@ -34,8 +34,8 @@ import com.carrotsearch.hppc.BitMixer; import org.apache.lucene.util.BytesRef; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; /** * Specialized hash table implementation similar to Lucene's BytesRefHash that maps diff --git a/server/src/main/java/org/opensearch/common/util/LongHash.java b/server/src/main/java/org/opensearch/common/util/LongHash.java index 8ec0f1565c4d5..71272e79c65b3 100644 --- a/server/src/main/java/org/opensearch/common/util/LongHash.java +++ b/server/src/main/java/org/opensearch/common/util/LongHash.java @@ -32,7 +32,7 @@ package org.opensearch.common.util; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; /** * Specialized hash table implementation similar to BytesRefHash that maps diff --git a/server/src/main/java/org/opensearch/common/util/LongLongHash.java b/server/src/main/java/org/opensearch/common/util/LongLongHash.java index 0dd13b4208c7a..1a720eae82a1d 100644 --- a/server/src/main/java/org/opensearch/common/util/LongLongHash.java +++ b/server/src/main/java/org/opensearch/common/util/LongLongHash.java @@ -34,7 +34,7 @@ import com.carrotsearch.hppc.BitMixer; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; /** * Specialized hash table implementation similar to BytesRefHash that maps diff --git a/server/src/main/java/org/opensearch/common/util/LongObjectPagedHashMap.java b/server/src/main/java/org/opensearch/common/util/LongObjectPagedHashMap.java index 116b74baa24d1..9d654e96779bf 100644 --- a/server/src/main/java/org/opensearch/common/util/LongObjectPagedHashMap.java +++ b/server/src/main/java/org/opensearch/common/util/LongObjectPagedHashMap.java @@ -32,7 +32,7 @@ package org.opensearch.common.util; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import java.util.Iterator; import java.util.NoSuchElementException; diff --git a/server/src/main/java/org/opensearch/common/util/concurrent/KeyedLock.java b/server/src/main/java/org/opensearch/common/util/concurrent/KeyedLock.java index bb863654469ee..9d4235f8d8a1a 100644 --- a/server/src/main/java/org/opensearch/common/util/concurrent/KeyedLock.java +++ b/server/src/main/java/org/opensearch/common/util/concurrent/KeyedLock.java @@ -32,7 +32,7 @@ package org.opensearch.common.util.concurrent; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicBoolean; diff --git a/server/src/main/java/org/opensearch/common/util/concurrent/ReleasableLock.java b/server/src/main/java/org/opensearch/common/util/concurrent/ReleasableLock.java index 8e3c4cbc64da6..cd87b58e87180 100644 --- a/server/src/main/java/org/opensearch/common/util/concurrent/ReleasableLock.java +++ b/server/src/main/java/org/opensearch/common/util/concurrent/ReleasableLock.java @@ -34,7 +34,7 @@ import org.opensearch.common.unit.TimeValue; import org.opensearch.core.Assertions; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.engine.EngineException; import java.util.concurrent.locks.Lock; diff --git a/server/src/main/java/org/opensearch/env/NodeEnvironment.java b/server/src/main/java/org/opensearch/env/NodeEnvironment.java index 32d81e8e1e482..f4a7066fa709b 100644 --- a/server/src/main/java/org/opensearch/env/NodeEnvironment.java +++ b/server/src/main/java/org/opensearch/env/NodeEnvironment.java @@ -61,7 +61,7 @@ import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.io.IOUtils; import org.opensearch.core.util.FileSystemUtils; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.gateway.MetadataStateFormat; import org.opensearch.gateway.PersistedClusterStateService; diff --git a/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java b/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java index 0b8756e499fd0..35272d9f54dc6 100644 --- a/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java +++ b/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java @@ -43,7 +43,7 @@ import org.opensearch.cluster.node.DiscoveryNodes; import org.opensearch.cluster.routing.allocation.RoutingAllocation; import org.opensearch.common.Nullable; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; import org.opensearch.index.shard.ShardId; import org.opensearch.transport.ReceiveTimeoutTransportException; diff --git a/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java b/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java index fe8504158abc2..1a4681766e489 100644 --- a/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java +++ b/server/src/main/java/org/opensearch/gateway/GatewayAllocator.java @@ -51,7 +51,7 @@ import org.opensearch.common.inject.Inject; import org.opensearch.common.util.concurrent.ConcurrentCollections; import org.opensearch.common.util.set.Sets; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.shard.ShardId; import org.opensearch.indices.store.TransportNodesListShardStoreMetadata; diff --git a/server/src/main/java/org/opensearch/gateway/PersistedClusterStateService.java b/server/src/main/java/org/opensearch/gateway/PersistedClusterStateService.java index 217bac3b4b99d..2515cd631481e 100644 --- a/server/src/main/java/org/opensearch/gateway/PersistedClusterStateService.java +++ b/server/src/main/java/org/opensearch/gateway/PersistedClusterStateService.java @@ -81,8 +81,8 @@ import org.opensearch.common.xcontent.XContentFactory; import org.opensearch.common.xcontent.XContentType; import org.opensearch.common.util.io.IOUtils; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.core.xcontent.ToXContent; import org.opensearch.core.xcontent.XContentBuilder; diff --git a/server/src/main/java/org/opensearch/http/DefaultRestChannel.java b/server/src/main/java/org/opensearch/http/DefaultRestChannel.java index 8f94bf4c7717d..bee7ae9d31d27 100644 --- a/server/src/main/java/org/opensearch/http/DefaultRestChannel.java +++ b/server/src/main/java/org/opensearch/http/DefaultRestChannel.java @@ -41,8 +41,8 @@ import org.opensearch.common.network.CloseableChannel; import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.concurrent.ThreadContext; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.rest.AbstractRestChannel; import org.opensearch.rest.RestChannel; import org.opensearch.rest.RestRequest; diff --git a/server/src/main/java/org/opensearch/index/IndexingPressure.java b/server/src/main/java/org/opensearch/index/IndexingPressure.java index 4dfea7f50fde8..33be340feb335 100644 --- a/server/src/main/java/org/opensearch/index/IndexingPressure.java +++ b/server/src/main/java/org/opensearch/index/IndexingPressure.java @@ -38,7 +38,7 @@ import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.ByteSizeValue; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; import org.opensearch.index.stats.IndexingPressureStats; diff --git a/server/src/main/java/org/opensearch/index/IndexingPressureService.java b/server/src/main/java/org/opensearch/index/IndexingPressureService.java index 1c683189cd34e..c28c18eb5517f 100644 --- a/server/src/main/java/org/opensearch/index/IndexingPressureService.java +++ b/server/src/main/java/org/opensearch/index/IndexingPressureService.java @@ -8,7 +8,7 @@ import org.opensearch.action.admin.indices.stats.CommonStatsFlags; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.settings.Settings; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.shard.ShardId; import org.opensearch.index.stats.IndexingPressureStats; import org.opensearch.index.stats.ShardIndexingPressureStats; diff --git a/server/src/main/java/org/opensearch/index/ShardIndexingPressure.java b/server/src/main/java/org/opensearch/index/ShardIndexingPressure.java index d14910e14eafb..53e0f7cea79b2 100644 --- a/server/src/main/java/org/opensearch/index/ShardIndexingPressure.java +++ b/server/src/main/java/org/opensearch/index/ShardIndexingPressure.java @@ -11,7 +11,7 @@ import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; import org.opensearch.index.ShardIndexingPressureTracker.CommonOperationTracker; import org.opensearch.index.ShardIndexingPressureTracker.OperationTracker; diff --git a/server/src/main/java/org/opensearch/index/engine/Engine.java b/server/src/main/java/org/opensearch/index/engine/Engine.java index 51cdef861f445..9bb99d0ef0d99 100644 --- a/server/src/main/java/org/opensearch/index/engine/Engine.java +++ b/server/src/main/java/org/opensearch/index/engine/Engine.java @@ -72,8 +72,8 @@ import org.opensearch.common.unit.ByteSizeValue; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.ReleasableLock; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.VersionType; import org.opensearch.index.mapper.IdFieldMapper; import org.opensearch.index.mapper.Mapping; 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 b96bade177be2..a249209bd863a 100644 --- a/server/src/main/java/org/opensearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/opensearch/index/engine/InternalEngine.java @@ -86,7 +86,7 @@ import org.opensearch.common.util.concurrent.KeyedLock; import org.opensearch.common.util.concurrent.ReleasableLock; import org.opensearch.common.util.io.IOUtils; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.IndexSettings; import org.opensearch.index.VersionType; import org.opensearch.index.fieldvisitor.IdOnlyFieldVisitor; diff --git a/server/src/main/java/org/opensearch/index/engine/LiveVersionMap.java b/server/src/main/java/org/opensearch/index/engine/LiveVersionMap.java index 1ea4d7143b284..07fcc546d2fb2 100644 --- a/server/src/main/java/org/opensearch/index/engine/LiveVersionMap.java +++ b/server/src/main/java/org/opensearch/index/engine/LiveVersionMap.java @@ -38,7 +38,7 @@ import org.apache.lucene.util.RamUsageEstimator; import org.opensearch.common.util.concurrent.ConcurrentCollections; import org.opensearch.common.util.concurrent.KeyedLock; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import java.io.IOException; import java.util.Collection; diff --git a/server/src/main/java/org/opensearch/index/engine/SoftDeletesPolicy.java b/server/src/main/java/org/opensearch/index/engine/SoftDeletesPolicy.java index 81da845a04230..eef73f1d49a8e 100644 --- a/server/src/main/java/org/opensearch/index/engine/SoftDeletesPolicy.java +++ b/server/src/main/java/org/opensearch/index/engine/SoftDeletesPolicy.java @@ -34,7 +34,7 @@ import org.apache.lucene.document.LongPoint; import org.apache.lucene.search.Query; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.mapper.SeqNoFieldMapper; import org.opensearch.index.seqno.RetentionLease; import org.opensearch.index.seqno.RetentionLeases; diff --git a/server/src/main/java/org/opensearch/index/fielddata/LeafFieldData.java b/server/src/main/java/org/opensearch/index/fielddata/LeafFieldData.java index 88aa44fe02ed3..2982363937710 100644 --- a/server/src/main/java/org/opensearch/index/fielddata/LeafFieldData.java +++ b/server/src/main/java/org/opensearch/index/fielddata/LeafFieldData.java @@ -33,7 +33,7 @@ package org.opensearch.index.fielddata; import org.apache.lucene.util.Accountable; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.mapper.DocValueFetcher; import org.opensearch.search.DocValueFormat; diff --git a/server/src/main/java/org/opensearch/index/seqno/RetentionLeaseActions.java b/server/src/main/java/org/opensearch/index/seqno/RetentionLeaseActions.java index 6476b1f0dc1c4..2ce0f1f524a43 100644 --- a/server/src/main/java/org/opensearch/index/seqno/RetentionLeaseActions.java +++ b/server/src/main/java/org/opensearch/index/seqno/RetentionLeaseActions.java @@ -47,7 +47,7 @@ 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.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.IndexService; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.ShardId; diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index 946488bf441bc..e368ee82b1084 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -101,8 +101,8 @@ import org.opensearch.common.util.set.Sets; import org.opensearch.common.xcontent.XContentHelper; import org.opensearch.common.util.io.IOUtils; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.gateway.WriteStateException; import org.opensearch.index.Index; import org.opensearch.index.IndexModule; diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShardOperationPermits.java b/server/src/main/java/org/opensearch/index/shard/IndexShardOperationPermits.java index 8279a2f6399c5..389de85a04483 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShardOperationPermits.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShardOperationPermits.java @@ -43,7 +43,7 @@ import org.opensearch.common.util.concurrent.RunOnce; import org.opensearch.common.util.concurrent.ThreadContext.StoredContext; import org.opensearch.common.util.io.IOUtils; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.threadpool.ThreadPool; import java.io.Closeable; diff --git a/server/src/main/java/org/opensearch/index/shard/RefreshListeners.java b/server/src/main/java/org/opensearch/index/shard/RefreshListeners.java index c1934596cd5c0..07c791e1dce14 100644 --- a/server/src/main/java/org/opensearch/index/shard/RefreshListeners.java +++ b/server/src/main/java/org/opensearch/index/shard/RefreshListeners.java @@ -38,7 +38,7 @@ import org.opensearch.common.metrics.MeanMetric; import org.opensearch.common.util.concurrent.RunOnce; import org.opensearch.common.util.concurrent.ThreadContext; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.translog.Translog; import java.io.Closeable; diff --git a/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java b/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java index 939402058e048..6ebb1bf7d2252 100644 --- a/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java +++ b/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java @@ -14,8 +14,8 @@ import org.opensearch.common.util.concurrent.ReleasableLock; import org.opensearch.common.util.io.IOUtils; import org.opensearch.core.util.FileSystemUtils; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.shard.ShardId; import org.opensearch.index.translog.transfer.BlobStoreTransferService; import org.opensearch.index.translog.transfer.FileTransferTracker; 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 5816fdfaff754..a9acf76f4834d 100644 --- a/server/src/main/java/org/opensearch/index/translog/Translog.java +++ b/server/src/main/java/org/opensearch/index/translog/Translog.java @@ -46,8 +46,8 @@ 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.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.core.common.Strings; import org.opensearch.index.IndexSettings; import org.opensearch.index.VersionType; diff --git a/server/src/main/java/org/opensearch/index/translog/TranslogDeletionPolicy.java b/server/src/main/java/org/opensearch/index/translog/TranslogDeletionPolicy.java index 608b860580f79..fde6d6bbc0632 100644 --- a/server/src/main/java/org/opensearch/index/translog/TranslogDeletionPolicy.java +++ b/server/src/main/java/org/opensearch/index/translog/TranslogDeletionPolicy.java @@ -34,7 +34,7 @@ import org.apache.lucene.util.Counter; import org.opensearch.core.Assertions; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.seqno.SequenceNumbers; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/index/translog/TranslogWriter.java b/server/src/main/java/org/opensearch/index/translog/TranslogWriter.java index 5584c08410a91..423e9dd960ed7 100644 --- a/server/src/main/java/org/opensearch/index/translog/TranslogWriter.java +++ b/server/src/main/java/org/opensearch/index/translog/TranslogWriter.java @@ -49,7 +49,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.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.core.Assertions; import org.opensearch.index.seqno.SequenceNumbers; import org.opensearch.index.shard.ShardId; diff --git a/server/src/main/java/org/opensearch/indices/IndicesService.java b/server/src/main/java/org/opensearch/indices/IndicesService.java index f7c351026e644..60e90a21b6309 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesService.java +++ b/server/src/main/java/org/opensearch/indices/IndicesService.java @@ -85,7 +85,7 @@ import org.opensearch.common.xcontent.XContentFactory; import org.opensearch.common.xcontent.XContentType; import org.opensearch.common.util.io.IOUtils; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; import org.opensearch.core.util.FileSystemUtils; import org.opensearch.core.xcontent.NamedXContentRegistry; diff --git a/server/src/main/java/org/opensearch/indices/RunUnderPrimaryPermit.java b/server/src/main/java/org/opensearch/indices/RunUnderPrimaryPermit.java index da410f6f629cc..b7995accda1da 100644 --- a/server/src/main/java/org/opensearch/indices/RunUnderPrimaryPermit.java +++ b/server/src/main/java/org/opensearch/indices/RunUnderPrimaryPermit.java @@ -12,7 +12,7 @@ import org.opensearch.action.ActionListener; import org.opensearch.common.util.CancellableThreads; import org.opensearch.common.util.concurrent.FutureUtils; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.IndexShardRelocatedException; import org.opensearch.threadpool.ThreadPool; diff --git a/server/src/main/java/org/opensearch/indices/fielddata/cache/IndicesFieldDataCache.java b/server/src/main/java/org/opensearch/indices/fielddata/cache/IndicesFieldDataCache.java index f37e1b3351c06..45f78155589b3 100644 --- a/server/src/main/java/org/opensearch/indices/fielddata/cache/IndicesFieldDataCache.java +++ b/server/src/main/java/org/opensearch/indices/fielddata/cache/IndicesFieldDataCache.java @@ -52,7 +52,7 @@ import org.opensearch.common.settings.Setting.Property; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.ByteSizeValue; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.Index; import org.opensearch.index.fielddata.LeafFieldData; import org.opensearch.index.fielddata.IndexFieldData; diff --git a/server/src/main/java/org/opensearch/indices/recovery/LocalStorePeerRecoverySourceHandler.java b/server/src/main/java/org/opensearch/indices/recovery/LocalStorePeerRecoverySourceHandler.java index 7c8e1c743426c..d7890d6e3e5e0 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/LocalStorePeerRecoverySourceHandler.java +++ b/server/src/main/java/org/opensearch/indices/recovery/LocalStorePeerRecoverySourceHandler.java @@ -18,7 +18,7 @@ import org.opensearch.common.SetOnce; import org.opensearch.common.concurrent.GatedCloseable; import org.opensearch.common.unit.TimeValue; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.engine.RecoveryEngineException; import org.opensearch.index.seqno.ReplicationTracker; import org.opensearch.index.seqno.RetentionLease; 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 e7c374f431242..4f9db27ffc9db 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/MultiFileWriter.java +++ b/server/src/main/java/org/opensearch/indices/recovery/MultiFileWriter.java @@ -40,7 +40,7 @@ import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.util.concurrent.AbstractRefCounted; import org.opensearch.common.util.concurrent.ConcurrentCollections; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.core.common.Strings; import org.opensearch.index.store.Store; import org.opensearch.index.store.StoreFileMetadata; diff --git a/server/src/main/java/org/opensearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/opensearch/indices/recovery/RecoverySourceHandler.java index 797a5b11cefe6..5e278f06cfb8f 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/opensearch/indices/recovery/RecoverySourceHandler.java @@ -56,8 +56,8 @@ import org.opensearch.common.util.concurrent.ListenableFuture; import org.opensearch.common.util.concurrent.OpenSearchExecutors; import org.opensearch.common.util.io.IOUtils; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.engine.RecoveryEngineException; import org.opensearch.index.seqno.RetentionLease; import org.opensearch.index.seqno.RetentionLeaseNotFoundException; diff --git a/server/src/main/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandler.java b/server/src/main/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandler.java index 7c51812ec94a2..6af4ce7461633 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandler.java +++ b/server/src/main/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandler.java @@ -13,7 +13,7 @@ import org.opensearch.action.StepListener; import org.opensearch.common.concurrent.GatedCloseable; import org.opensearch.common.unit.TimeValue; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.engine.RecoveryEngineException; import org.opensearch.index.seqno.SequenceNumbers; import org.opensearch.index.shard.IndexShard; diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentFileTransferHandler.java b/server/src/main/java/org/opensearch/indices/replication/SegmentFileTransferHandler.java index c303a53c827f8..44029f0c89cb0 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentFileTransferHandler.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentFileTransferHandler.java @@ -22,7 +22,7 @@ import org.opensearch.common.lucene.store.InputStreamIndexInput; import org.opensearch.common.util.CancellableThreads; import org.opensearch.common.util.io.IOUtils; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.store.Store; import org.opensearch.index.store.StoreFileMetadata; diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index 0660341dd92f8..c51eff57de68c 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -134,7 +134,7 @@ import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.PageCacheRecycler; import org.opensearch.common.util.io.IOUtils; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.discovery.Discovery; import org.opensearch.discovery.DiscoveryModule; diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java index 13550117a24a4..c0d6f49a5ce0d 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java @@ -97,7 +97,7 @@ import org.opensearch.common.xcontent.LoggingDeprecationHandler; import org.opensearch.common.xcontent.XContentFactory; import org.opensearch.common.xcontent.XContentType; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.core.util.BytesRefUtils; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.core.xcontent.XContentParser; diff --git a/server/src/main/java/org/opensearch/rest/RestResponse.java b/server/src/main/java/org/opensearch/rest/RestResponse.java index afa7180d2abe6..fd08f8cf96fd8 100644 --- a/server/src/main/java/org/opensearch/rest/RestResponse.java +++ b/server/src/main/java/org/opensearch/rest/RestResponse.java @@ -34,7 +34,7 @@ import org.opensearch.OpenSearchException; import org.opensearch.common.bytes.BytesReference; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import java.util.ArrayList; import java.util.Collections; diff --git a/server/src/main/java/org/opensearch/search/DefaultSearchContext.java b/server/src/main/java/org/opensearch/search/DefaultSearchContext.java index c50d0280aec2d..e83bfe8486904 100644 --- a/server/src/main/java/org/opensearch/search/DefaultSearchContext.java +++ b/server/src/main/java/org/opensearch/search/DefaultSearchContext.java @@ -49,7 +49,7 @@ import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.FeatureFlags; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; import org.opensearch.index.IndexService; import org.opensearch.index.IndexSettings; diff --git a/server/src/main/java/org/opensearch/search/SearchService.java b/server/src/main/java/org/opensearch/search/SearchService.java index 23d35cb823342..7d67c6c3b45f4 100644 --- a/server/src/main/java/org/opensearch/search/SearchService.java +++ b/server/src/main/java/org/opensearch/search/SearchService.java @@ -67,8 +67,8 @@ import org.opensearch.common.util.concurrent.ConcurrentCollections; import org.opensearch.common.util.concurrent.ConcurrentMapLong; import org.opensearch.common.util.io.IOUtils; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; import org.opensearch.index.Index; import org.opensearch.index.IndexNotFoundException; diff --git a/server/src/main/java/org/opensearch/search/aggregations/Aggregator.java b/server/src/main/java/org/opensearch/search/aggregations/Aggregator.java index 72ebc0d2cf087..4d9b973e4550f 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/Aggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/Aggregator.java @@ -37,7 +37,7 @@ 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.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.core.xcontent.DeprecationHandler; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.core.xcontent.XContentParser; diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/BucketsAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/BucketsAggregator.java index 89db461338eab..5c335dfffeaa7 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/BucketsAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/BucketsAggregator.java @@ -34,7 +34,7 @@ import org.apache.lucene.index.LeafReaderContext; import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.LongArray; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.search.aggregations.AggregationExecutionException; import org.opensearch.search.aggregations.Aggregator; import org.opensearch.search.aggregations.AggregatorBase; diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/BinaryValuesSource.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/BinaryValuesSource.java index 1f2609f1ea871..917105f4b7011 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/BinaryValuesSource.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/BinaryValuesSource.java @@ -41,7 +41,7 @@ import org.opensearch.common.CheckedFunction; import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.ObjectArray; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.fielddata.SortedBinaryDocValues; import org.opensearch.index.mapper.MappedFieldType; import org.opensearch.index.mapper.StringFieldType; diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/CompositeAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/CompositeAggregator.java index c7e27f672ef4a..8b487fc499602 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/CompositeAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/CompositeAggregator.java @@ -57,7 +57,7 @@ import org.apache.lucene.search.comparators.LongComparator; import org.apache.lucene.util.Bits; import org.apache.lucene.util.RoaringDocIdSet; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.IndexSortConfig; import org.opensearch.search.DocValueFormat; import org.opensearch.search.aggregations.Aggregator; diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/CompositeValuesCollectorQueue.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/CompositeValuesCollectorQueue.java index 43f5ddd203953..11fc267e7d745 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/CompositeValuesCollectorQueue.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/CompositeValuesCollectorQueue.java @@ -37,8 +37,8 @@ import org.apache.lucene.util.PriorityQueue; import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.LongArray; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.search.aggregations.LeafBucketCollector; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/DoubleValuesSource.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/DoubleValuesSource.java index 0af2266509dd3..10eebe204d601 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/DoubleValuesSource.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/DoubleValuesSource.java @@ -39,7 +39,7 @@ import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.BitArray; import org.opensearch.common.util.DoubleArray; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.fielddata.SortedNumericDoubleValues; import org.opensearch.index.mapper.MappedFieldType; import org.opensearch.search.DocValueFormat; diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/GlobalOrdinalValuesSource.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/GlobalOrdinalValuesSource.java index 57038372672d8..6331496196b2a 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/GlobalOrdinalValuesSource.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/GlobalOrdinalValuesSource.java @@ -41,7 +41,7 @@ import org.opensearch.common.CheckedFunction; import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.LongArray; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.mapper.MappedFieldType; import org.opensearch.index.mapper.StringFieldType; import org.opensearch.search.DocValueFormat; diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/LongValuesSource.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/LongValuesSource.java index 5b3db1b48d653..8b162ffaf8604 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/LongValuesSource.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/LongValuesSource.java @@ -48,7 +48,7 @@ import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.BitArray; import org.opensearch.common.util.LongArray; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.mapper.DateFieldMapper; import org.opensearch.index.mapper.MappedFieldType; import org.opensearch.index.mapper.NumberFieldMapper; diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/SingleDimensionValuesSource.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/SingleDimensionValuesSource.java index 30fedac53691f..9f48b54350e25 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/SingleDimensionValuesSource.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/SingleDimensionValuesSource.java @@ -37,7 +37,7 @@ import org.apache.lucene.search.Query; import org.opensearch.common.Nullable; import org.opensearch.common.util.BigArrays; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.mapper.MappedFieldType; import org.opensearch.search.DocValueFormat; import org.opensearch.search.aggregations.LeafBucketCollector; diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/UnsignedLongValuesSource.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/UnsignedLongValuesSource.java index 1c5d9aacc1ad0..a45f1a9a5840b 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/UnsignedLongValuesSource.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/composite/UnsignedLongValuesSource.java @@ -18,7 +18,7 @@ import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.BitArray; import org.opensearch.common.util.LongArray; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.mapper.MappedFieldType; import org.opensearch.index.mapper.NumberFieldMapper; import org.opensearch.search.DocValueFormat; diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/AbstractHistogramAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/AbstractHistogramAggregator.java index fdbed59ad2ab6..d3a4a51e5b6f2 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/AbstractHistogramAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/AbstractHistogramAggregator.java @@ -33,7 +33,7 @@ package org.opensearch.search.aggregations.bucket.histogram; import org.apache.lucene.util.CollectionUtil; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.search.DocValueFormat; import org.opensearch.search.aggregations.Aggregator; import org.opensearch.search.aggregations.AggregatorFactories; diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java index b92d16cfb540a..b6b0c2cec5b68 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/AutoDateHistogramAggregator.java @@ -40,7 +40,7 @@ import org.opensearch.common.util.ByteArray; import org.opensearch.common.util.IntArray; import org.opensearch.common.util.LongArray; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.search.DocValueFormat; import org.opensearch.search.aggregations.Aggregator; import org.opensearch.search.aggregations.AggregatorFactories; diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java index 7c548bf331f55..040621ce8ec34 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/DateHistogramAggregator.java @@ -37,7 +37,7 @@ import org.apache.lucene.util.CollectionUtil; import org.opensearch.common.Nullable; import org.opensearch.common.Rounding; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.search.DocValueFormat; import org.opensearch.search.aggregations.Aggregator; import org.opensearch.search.aggregations.AggregatorFactories; diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/DateRangeHistogramAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/DateRangeHistogramAggregator.java index 46b1abfd9645c..f70a5bb537300 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/DateRangeHistogramAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/DateRangeHistogramAggregator.java @@ -37,7 +37,7 @@ import org.apache.lucene.util.CollectionUtil; import org.opensearch.common.Nullable; import org.opensearch.common.Rounding; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.fielddata.SortedBinaryDocValues; import org.opensearch.index.mapper.RangeFieldMapper; import org.opensearch.index.mapper.RangeType; diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/VariableWidthHistogramAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/VariableWidthHistogramAggregator.java index 70e496b582008..803c90be467b1 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/VariableWidthHistogramAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/histogram/VariableWidthHistogramAggregator.java @@ -39,8 +39,8 @@ import org.opensearch.common.Nullable; import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.DoubleArray; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.fielddata.SortedNumericDoubleValues; import org.opensearch.search.DocValueFormat; import org.opensearch.search.aggregations.Aggregator; diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/sampler/BestDocsDeferringCollector.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/sampler/BestDocsDeferringCollector.java index d4eca59b6b621..f0a3a9999ad94 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/sampler/BestDocsDeferringCollector.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/sampler/BestDocsDeferringCollector.java @@ -44,8 +44,8 @@ import org.opensearch.OpenSearchException; import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.ObjectArray; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.search.aggregations.BucketCollector; import org.opensearch.search.aggregations.LeafBucketCollector; import org.opensearch.search.aggregations.MultiBucketCollector; diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/sampler/DiversifiedMapSamplerAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/sampler/DiversifiedMapSamplerAggregator.java index 8b163f1d7fd50..92f847a4a6b5d 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/sampler/DiversifiedMapSamplerAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/sampler/DiversifiedMapSamplerAggregator.java @@ -40,7 +40,7 @@ import org.apache.lucene.util.BytesRef; import org.opensearch.OpenSearchException; import org.opensearch.common.util.BytesRefHash; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.fielddata.AbstractNumericDocValues; import org.opensearch.index.fielddata.SortedBinaryDocValues; import org.opensearch.search.aggregations.Aggregator; diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/sampler/SamplerAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/sampler/SamplerAggregator.java index a1dac28486ccd..533948c7483e8 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/sampler/SamplerAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/sampler/SamplerAggregator.java @@ -37,7 +37,7 @@ import org.apache.lucene.util.RamUsageEstimator; import org.opensearch.common.xcontent.LoggingDeprecationHandler; import org.opensearch.core.ParseField; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.search.aggregations.AggregationExecutionException; import org.opensearch.search.aggregations.Aggregator; import org.opensearch.search.aggregations.AggregatorFactories; diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/BytesKeyedBucketOrds.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/BytesKeyedBucketOrds.java index a6941bfae73ab..f5595977004e0 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/BytesKeyedBucketOrds.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/BytesKeyedBucketOrds.java @@ -35,8 +35,8 @@ import org.apache.lucene.util.BytesRef; import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.BytesRefHash; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.search.aggregations.CardinalityUpperBound; /** diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/GlobalOrdinalsStringTermsAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/GlobalOrdinalsStringTermsAggregator.java index e82194587edd7..996ff2fea8b1a 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/GlobalOrdinalsStringTermsAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/GlobalOrdinalsStringTermsAggregator.java @@ -43,8 +43,8 @@ import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.util.LongArray; import org.opensearch.common.util.LongHash; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.search.DocValueFormat; import org.opensearch.search.aggregations.AggregationExecutionException; diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/LongKeyedBucketOrds.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/LongKeyedBucketOrds.java index 7ad03b2153a2e..6b511587cc271 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/LongKeyedBucketOrds.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/LongKeyedBucketOrds.java @@ -35,7 +35,7 @@ import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.LongHash; import org.opensearch.common.util.LongLongHash; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.search.aggregations.CardinalityUpperBound; /** diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/LongRareTermsAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/LongRareTermsAggregator.java index 22c4d372dc0db..0e7956744091f 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/LongRareTermsAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/LongRareTermsAggregator.java @@ -35,7 +35,7 @@ import org.apache.lucene.index.SortedNumericDocValues; import org.opensearch.common.util.LongHash; import org.opensearch.common.util.SetBackedScalingCuckooFilter; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.search.DocValueFormat; import org.opensearch.search.aggregations.Aggregator; import org.opensearch.search.aggregations.AggregatorFactories; diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/MapStringTermsAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/MapStringTermsAggregator.java index 8991b1c55d7b8..bcdf1f4480a31 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/MapStringTermsAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/MapStringTermsAggregator.java @@ -37,8 +37,8 @@ import org.apache.lucene.util.BytesRefBuilder; import org.apache.lucene.util.PriorityQueue; import org.opensearch.common.util.LongArray; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.fielddata.SortedBinaryDocValues; import org.opensearch.search.DocValueFormat; import org.opensearch.search.aggregations.Aggregator; diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/MultiTermsAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/MultiTermsAggregator.java index 64e635e020f7a..c810ba8f38624 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/MultiTermsAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/MultiTermsAggregator.java @@ -21,7 +21,7 @@ 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.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.fielddata.SortedBinaryDocValues; import org.opensearch.index.fielddata.SortedNumericDoubleValues; import org.opensearch.search.DocValueFormat; diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/NumericTermsAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/NumericTermsAggregator.java index 9cbe454a7b8e4..a0265135fe9d3 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/NumericTermsAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/NumericTermsAggregator.java @@ -39,8 +39,8 @@ import org.apache.lucene.util.PriorityQueue; import org.opensearch.common.Numbers; import org.opensearch.common.util.LongArray; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.fielddata.FieldData; import org.opensearch.search.DocValueFormat; import org.opensearch.search.aggregations.Aggregator; diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/SignificanceLookup.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/SignificanceLookup.java index e484383b4a9b3..aa1409a7bec78 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/SignificanceLookup.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/SignificanceLookup.java @@ -47,8 +47,8 @@ import org.opensearch.common.util.BytesRefHash; import org.opensearch.common.util.LongArray; import org.opensearch.common.util.LongHash; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.mapper.MappedFieldType; import org.opensearch.index.query.QueryBuilder; import org.opensearch.index.query.QueryShardContext; diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/SignificantTextAggregatorFactory.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/SignificantTextAggregatorFactory.java index f318c045e46a8..e5cc3f9dbaabd 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/SignificantTextAggregatorFactory.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/SignificantTextAggregatorFactory.java @@ -43,7 +43,7 @@ import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.BytesRefHash; import org.opensearch.common.util.ObjectArray; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.mapper.MappedFieldType; import org.opensearch.index.query.QueryBuilder; import org.opensearch.index.query.QueryShardContext; diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/StringRareTermsAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/StringRareTermsAggregator.java index c57f2f6d7f2ce..b12c397b00fe2 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/StringRareTermsAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/StringRareTermsAggregator.java @@ -36,7 +36,7 @@ import org.apache.lucene.util.BytesRefBuilder; import org.opensearch.common.util.BytesRefHash; import org.opensearch.common.util.SetBackedScalingCuckooFilter; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.fielddata.SortedBinaryDocValues; import org.opensearch.search.DocValueFormat; import org.opensearch.search.aggregations.Aggregator; diff --git a/server/src/main/java/org/opensearch/search/aggregations/metrics/AbstractHDRPercentilesAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/metrics/AbstractHDRPercentilesAggregator.java index 18324995458d0..8bf0158298395 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/metrics/AbstractHDRPercentilesAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/metrics/AbstractHDRPercentilesAggregator.java @@ -38,7 +38,7 @@ import org.opensearch.common.util.ArrayUtils; import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.ObjectArray; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.fielddata.SortedNumericDoubleValues; import org.opensearch.search.DocValueFormat; diff --git a/server/src/main/java/org/opensearch/search/aggregations/metrics/AbstractHyperLogLogPlusPlus.java b/server/src/main/java/org/opensearch/search/aggregations/metrics/AbstractHyperLogLogPlusPlus.java index d1e850f23e1ce..37cdd2227fc8a 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/metrics/AbstractHyperLogLogPlusPlus.java +++ b/server/src/main/java/org/opensearch/search/aggregations/metrics/AbstractHyperLogLogPlusPlus.java @@ -35,7 +35,7 @@ import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.util.BigArrays; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import java.io.IOException; import java.util.HashMap; diff --git a/server/src/main/java/org/opensearch/search/aggregations/metrics/AbstractTDigestPercentilesAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/metrics/AbstractTDigestPercentilesAggregator.java index 09640e25d3b8d..dcc07a45d0528 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/metrics/AbstractTDigestPercentilesAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/metrics/AbstractTDigestPercentilesAggregator.java @@ -37,7 +37,7 @@ import org.opensearch.common.util.ArrayUtils; import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.ObjectArray; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.fielddata.SortedNumericDoubleValues; import org.opensearch.search.DocValueFormat; import org.opensearch.search.aggregations.Aggregator; diff --git a/server/src/main/java/org/opensearch/search/aggregations/metrics/AvgAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/metrics/AvgAggregator.java index e1c853f707ff0..0ba66ccd880d4 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/metrics/AvgAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/metrics/AvgAggregator.java @@ -36,7 +36,7 @@ import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.DoubleArray; import org.opensearch.common.util.LongArray; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.fielddata.SortedNumericDoubleValues; import org.opensearch.search.DocValueFormat; import org.opensearch.search.aggregations.Aggregator; diff --git a/server/src/main/java/org/opensearch/search/aggregations/metrics/CardinalityAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/metrics/CardinalityAggregator.java index 89ff05af049d6..77d8c3cdcf46b 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/metrics/CardinalityAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/metrics/CardinalityAggregator.java @@ -46,8 +46,8 @@ import org.opensearch.common.util.BitArray; import org.opensearch.common.util.LongArray; import org.opensearch.common.util.ObjectArray; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.fielddata.SortedBinaryDocValues; import org.opensearch.index.fielddata.SortedNumericDoubleValues; import org.opensearch.search.aggregations.Aggregator; diff --git a/server/src/main/java/org/opensearch/search/aggregations/metrics/ExtendedStatsAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/metrics/ExtendedStatsAggregator.java index 64b2b70982b7a..dba6cad69bec8 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/metrics/ExtendedStatsAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/metrics/ExtendedStatsAggregator.java @@ -37,7 +37,7 @@ import org.opensearch.common.util.DoubleArray; import org.opensearch.common.util.LongArray; import org.opensearch.core.ParseField; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.fielddata.SortedNumericDoubleValues; import org.opensearch.search.DocValueFormat; import org.opensearch.search.aggregations.Aggregator; diff --git a/server/src/main/java/org/opensearch/search/aggregations/metrics/GeoCentroidAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/metrics/GeoCentroidAggregator.java index 58536e8926804..39790357ff9ca 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/metrics/GeoCentroidAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/metrics/GeoCentroidAggregator.java @@ -37,7 +37,7 @@ import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.DoubleArray; import org.opensearch.common.util.LongArray; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.fielddata.MultiGeoPointValues; import org.opensearch.search.aggregations.Aggregator; import org.opensearch.search.aggregations.InternalAggregation; diff --git a/server/src/main/java/org/opensearch/search/aggregations/metrics/HyperLogLogPlusPlus.java b/server/src/main/java/org/opensearch/search/aggregations/metrics/HyperLogLogPlusPlus.java index 8c73bb91d5159..0adee55fc00e5 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/metrics/HyperLogLogPlusPlus.java +++ b/server/src/main/java/org/opensearch/search/aggregations/metrics/HyperLogLogPlusPlus.java @@ -39,8 +39,8 @@ import org.opensearch.common.util.ByteArray; import org.opensearch.common.util.ByteUtils; import org.opensearch.common.util.IntArray; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import java.nio.ByteBuffer; import java.nio.ByteOrder; diff --git a/server/src/main/java/org/opensearch/search/aggregations/metrics/HyperLogLogPlusPlusSparse.java b/server/src/main/java/org/opensearch/search/aggregations/metrics/HyperLogLogPlusPlusSparse.java index 702f10e5b39e7..62ebe0974b955 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/metrics/HyperLogLogPlusPlusSparse.java +++ b/server/src/main/java/org/opensearch/search/aggregations/metrics/HyperLogLogPlusPlusSparse.java @@ -34,8 +34,8 @@ import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.IntArray; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; /** * AbstractHyperLogLogPlusPlus instance that only supports linear counting. The maximum number of hashes supported diff --git a/server/src/main/java/org/opensearch/search/aggregations/metrics/MaxAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/metrics/MaxAggregator.java index e3562ef99eeab..aac16ca3f46cf 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/metrics/MaxAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/metrics/MaxAggregator.java @@ -39,7 +39,7 @@ import org.apache.lucene.util.Bits; import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.DoubleArray; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.fielddata.NumericDoubleValues; import org.opensearch.index.fielddata.SortedNumericDoubleValues; import org.opensearch.search.DocValueFormat; diff --git a/server/src/main/java/org/opensearch/search/aggregations/metrics/MedianAbsoluteDeviationAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/metrics/MedianAbsoluteDeviationAggregator.java index 6358281fa1523..fbea8a192fa9a 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/metrics/MedianAbsoluteDeviationAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/metrics/MedianAbsoluteDeviationAggregator.java @@ -37,7 +37,7 @@ import org.opensearch.common.Nullable; import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.ObjectArray; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.fielddata.SortedNumericDoubleValues; import org.opensearch.search.DocValueFormat; import org.opensearch.search.aggregations.Aggregator; diff --git a/server/src/main/java/org/opensearch/search/aggregations/metrics/MinAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/metrics/MinAggregator.java index 23b750dde74eb..6c81879fd2b8e 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/metrics/MinAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/metrics/MinAggregator.java @@ -39,7 +39,7 @@ import org.apache.lucene.util.Bits; import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.DoubleArray; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.fielddata.NumericDoubleValues; import org.opensearch.index.fielddata.SortedNumericDoubleValues; import org.opensearch.search.DocValueFormat; diff --git a/server/src/main/java/org/opensearch/search/aggregations/metrics/ScriptedMetricAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/metrics/ScriptedMetricAggregator.java index 674d735bfc187..2f4d68227ca42 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/metrics/ScriptedMetricAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/metrics/ScriptedMetricAggregator.java @@ -39,7 +39,7 @@ import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.util.CollectionUtils; import org.opensearch.common.util.ObjectArray; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.script.Script; import org.opensearch.script.ScriptedMetricAggContexts; import org.opensearch.script.ScriptedMetricAggContexts.MapScript; diff --git a/server/src/main/java/org/opensearch/search/aggregations/metrics/StatsAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/metrics/StatsAggregator.java index 577609095963e..d3636fffb5256 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/metrics/StatsAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/metrics/StatsAggregator.java @@ -36,7 +36,7 @@ import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.DoubleArray; import org.opensearch.common.util.LongArray; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.fielddata.SortedNumericDoubleValues; import org.opensearch.search.DocValueFormat; import org.opensearch.search.aggregations.Aggregator; diff --git a/server/src/main/java/org/opensearch/search/aggregations/metrics/SumAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/metrics/SumAggregator.java index fb92b091f4968..097c0658067d2 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/metrics/SumAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/metrics/SumAggregator.java @@ -35,7 +35,7 @@ import org.apache.lucene.search.ScoreMode; import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.DoubleArray; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.fielddata.SortedNumericDoubleValues; import org.opensearch.search.DocValueFormat; import org.opensearch.search.aggregations.Aggregator; diff --git a/server/src/main/java/org/opensearch/search/aggregations/metrics/TopHitsAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/metrics/TopHitsAggregator.java index f1f4c4acdf4b9..5242461fa0054 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/metrics/TopHitsAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/metrics/TopHitsAggregator.java @@ -53,7 +53,7 @@ import org.opensearch.common.lucene.Lucene; import org.opensearch.common.lucene.search.TopDocsAndMaxScore; import org.opensearch.common.util.LongObjectPagedHashMap; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.search.SearchHit; import org.opensearch.search.SearchHits; import org.opensearch.search.aggregations.Aggregator; diff --git a/server/src/main/java/org/opensearch/search/aggregations/metrics/ValueCountAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/metrics/ValueCountAggregator.java index e6d4d09403689..7e526add36ba7 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/metrics/ValueCountAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/metrics/ValueCountAggregator.java @@ -36,7 +36,7 @@ import org.apache.lucene.search.ScoreMode; import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.LongArray; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.fielddata.MultiGeoPointValues; import org.opensearch.index.fielddata.SortedBinaryDocValues; import org.opensearch.search.aggregations.Aggregator; diff --git a/server/src/main/java/org/opensearch/search/aggregations/metrics/WeightedAvgAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/metrics/WeightedAvgAggregator.java index bf752711fa30f..b0cfafc8366f7 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/metrics/WeightedAvgAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/metrics/WeightedAvgAggregator.java @@ -35,7 +35,7 @@ import org.apache.lucene.search.ScoreMode; import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.DoubleArray; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.fielddata.SortedNumericDoubleValues; import org.opensearch.search.DocValueFormat; import org.opensearch.search.aggregations.AggregationExecutionException; diff --git a/server/src/main/java/org/opensearch/search/internal/ContextIndexSearcher.java b/server/src/main/java/org/opensearch/search/internal/ContextIndexSearcher.java index f02d28c2c375b..f1650b686f070 100644 --- a/server/src/main/java/org/opensearch/search/internal/ContextIndexSearcher.java +++ b/server/src/main/java/org/opensearch/search/internal/ContextIndexSearcher.java @@ -64,7 +64,7 @@ import org.apache.lucene.util.SparseFixedBitSet; import org.opensearch.cluster.metadata.DataStream; import org.opensearch.common.lucene.search.TopDocsAndMaxScore; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.search.DocValueFormat; import org.opensearch.search.SearchService; import org.opensearch.search.dfs.AggregatedDfs; diff --git a/server/src/main/java/org/opensearch/search/internal/PitReaderContext.java b/server/src/main/java/org/opensearch/search/internal/PitReaderContext.java index 0d60df27a6b85..5c2a9f82f98e4 100644 --- a/server/src/main/java/org/opensearch/search/internal/PitReaderContext.java +++ b/server/src/main/java/org/opensearch/search/internal/PitReaderContext.java @@ -10,8 +10,8 @@ import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.common.SetOnce; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.IndexService; import org.opensearch.index.engine.Engine; import org.opensearch.index.engine.Segment; diff --git a/server/src/main/java/org/opensearch/search/internal/ReaderContext.java b/server/src/main/java/org/opensearch/search/internal/ReaderContext.java index 46adbf136a150..549638c204f49 100644 --- a/server/src/main/java/org/opensearch/search/internal/ReaderContext.java +++ b/server/src/main/java/org/opensearch/search/internal/ReaderContext.java @@ -33,8 +33,8 @@ package org.opensearch.search.internal; import org.opensearch.common.util.concurrent.AbstractRefCounted; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.IndexService; import org.opensearch.index.engine.Engine; import org.opensearch.index.shard.IndexShard; diff --git a/server/src/main/java/org/opensearch/search/internal/SearchContext.java b/server/src/main/java/org/opensearch/search/internal/SearchContext.java index 79a5b89cfa881..9c6a0498ec848 100644 --- a/server/src/main/java/org/opensearch/search/internal/SearchContext.java +++ b/server/src/main/java/org/opensearch/search/internal/SearchContext.java @@ -40,8 +40,8 @@ import org.opensearch.common.Nullable; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.BigArrays; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.cache.bitset.BitsetFilterCache; import org.opensearch.index.mapper.MappedFieldType; import org.opensearch.index.mapper.MapperService; diff --git a/server/src/main/java/org/opensearch/search/sort/BucketedSort.java b/server/src/main/java/org/opensearch/search/sort/BucketedSort.java index 3d598ab60259e..43a98fb5b1b39 100644 --- a/server/src/main/java/org/opensearch/search/sort/BucketedSort.java +++ b/server/src/main/java/org/opensearch/search/sort/BucketedSort.java @@ -42,8 +42,8 @@ import org.opensearch.common.util.FloatArray; import org.opensearch.common.util.IntArray; import org.opensearch.common.util.LongArray; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.search.DocValueFormat; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/tasks/TaskManager.java b/server/src/main/java/org/opensearch/tasks/TaskManager.java index 6aeba47766842..443a6c0853884 100644 --- a/server/src/main/java/org/opensearch/tasks/TaskManager.java +++ b/server/src/main/java/org/opensearch/tasks/TaskManager.java @@ -59,8 +59,8 @@ import org.opensearch.common.util.concurrent.ConcurrentCollections; import org.opensearch.common.util.concurrent.ConcurrentMapLong; import org.opensearch.common.util.concurrent.ThreadContext; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TcpChannel; diff --git a/server/src/main/java/org/opensearch/transport/InboundAggregator.java b/server/src/main/java/org/opensearch/transport/InboundAggregator.java index 1aebbae7b37cd..c8cfaa8339e33 100644 --- a/server/src/main/java/org/opensearch/transport/InboundAggregator.java +++ b/server/src/main/java/org/opensearch/transport/InboundAggregator.java @@ -38,8 +38,8 @@ import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.bytes.CompositeBytesReference; import org.opensearch.common.bytes.ReleasableBytesReference; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import java.io.IOException; import java.util.ArrayList; diff --git a/server/src/main/java/org/opensearch/transport/InboundDecoder.java b/server/src/main/java/org/opensearch/transport/InboundDecoder.java index 3196b0715c942..96d8967be1841 100644 --- a/server/src/main/java/org/opensearch/transport/InboundDecoder.java +++ b/server/src/main/java/org/opensearch/transport/InboundDecoder.java @@ -38,7 +38,7 @@ import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.util.PageCacheRecycler; import org.opensearch.common.util.io.IOUtils; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import java.io.IOException; import java.util.function.Consumer; diff --git a/server/src/main/java/org/opensearch/transport/InboundMessage.java b/server/src/main/java/org/opensearch/transport/InboundMessage.java index 0b82e4d746990..31262ab22a0ea 100644 --- a/server/src/main/java/org/opensearch/transport/InboundMessage.java +++ b/server/src/main/java/org/opensearch/transport/InboundMessage.java @@ -35,8 +35,8 @@ import org.opensearch.common.bytes.ReleasableBytesReference; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.util.io.IOUtils; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/transport/InboundPipeline.java b/server/src/main/java/org/opensearch/transport/InboundPipeline.java index d20b15059936e..be269dd229f22 100644 --- a/server/src/main/java/org/opensearch/transport/InboundPipeline.java +++ b/server/src/main/java/org/opensearch/transport/InboundPipeline.java @@ -37,8 +37,8 @@ import org.opensearch.common.bytes.CompositeBytesReference; import org.opensearch.common.bytes.ReleasableBytesReference; import org.opensearch.common.util.PageCacheRecycler; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import java.io.IOException; import java.util.ArrayDeque; diff --git a/server/src/main/java/org/opensearch/transport/OutboundHandler.java b/server/src/main/java/org/opensearch/transport/OutboundHandler.java index 8d7217e4cd1f0..4c54b4bd8b6ff 100644 --- a/server/src/main/java/org/opensearch/transport/OutboundHandler.java +++ b/server/src/main/java/org/opensearch/transport/OutboundHandler.java @@ -48,8 +48,8 @@ import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.concurrent.ThreadContext; import org.opensearch.common.util.io.IOUtils; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.threadpool.ThreadPool; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/transport/RequestHandlerRegistry.java b/server/src/main/java/org/opensearch/transport/RequestHandlerRegistry.java index 83bdf185b8a5f..b292e6b75ebdd 100644 --- a/server/src/main/java/org/opensearch/transport/RequestHandlerRegistry.java +++ b/server/src/main/java/org/opensearch/transport/RequestHandlerRegistry.java @@ -35,8 +35,8 @@ import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.Writeable; import org.opensearch.common.util.concurrent.ThreadContext; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.search.internal.ShardSearchRequest; import org.opensearch.tasks.CancellableTask; import org.opensearch.tasks.Task; diff --git a/server/src/main/java/org/opensearch/transport/TaskTransportChannel.java b/server/src/main/java/org/opensearch/transport/TaskTransportChannel.java index 6fcef7619a703..4dceee4c48d4d 100644 --- a/server/src/main/java/org/opensearch/transport/TaskTransportChannel.java +++ b/server/src/main/java/org/opensearch/transport/TaskTransportChannel.java @@ -33,7 +33,7 @@ package org.opensearch.transport; import org.opensearch.Version; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/transport/TcpTransportChannel.java b/server/src/main/java/org/opensearch/transport/TcpTransportChannel.java index b396d63b99eb8..e25003648794d 100644 --- a/server/src/main/java/org/opensearch/transport/TcpTransportChannel.java +++ b/server/src/main/java/org/opensearch/transport/TcpTransportChannel.java @@ -33,7 +33,7 @@ package org.opensearch.transport; import org.opensearch.Version; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.search.query.QuerySearchResult; import java.io.IOException; diff --git a/server/src/main/java/org/opensearch/transport/TransportService.java b/server/src/main/java/org/opensearch/transport/TransportService.java index 8cd46a42952ae..cb324035bcd62 100644 --- a/server/src/main/java/org/opensearch/transport/TransportService.java +++ b/server/src/main/java/org/opensearch/transport/TransportService.java @@ -57,7 +57,7 @@ import org.opensearch.common.util.concurrent.AbstractRunnable; import org.opensearch.common.util.concurrent.ThreadContext; import org.opensearch.common.util.io.IOUtils; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.core.common.Strings; import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; import org.opensearch.node.NodeClosedException; diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/node/tasks/TaskManagerTestCase.java b/server/src/test/java/org/opensearch/action/admin/cluster/node/tasks/TaskManagerTestCase.java index 010524e1fed66..843fd85d81197 100644 --- a/server/src/test/java/org/opensearch/action/admin/cluster/node/tasks/TaskManagerTestCase.java +++ b/server/src/test/java/org/opensearch/action/admin/cluster/node/tasks/TaskManagerTestCase.java @@ -55,7 +55,7 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.transport.BoundTransportAddress; import org.opensearch.common.util.PageCacheRecycler; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.indices.breaker.NoneCircuitBreakerService; import org.opensearch.tasks.TaskCancellationService; import org.opensearch.tasks.TaskManager; diff --git a/server/src/test/java/org/opensearch/action/resync/TransportResyncReplicationActionTests.java b/server/src/test/java/org/opensearch/action/resync/TransportResyncReplicationActionTests.java index 39dfde96c0fdf..e0c436a943b93 100644 --- a/server/src/test/java/org/opensearch/action/resync/TransportResyncReplicationActionTests.java +++ b/server/src/test/java/org/opensearch/action/resync/TransportResyncReplicationActionTests.java @@ -53,7 +53,7 @@ import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.PageCacheRecycler; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.Index; import org.opensearch.index.IndexService; import org.opensearch.index.IndexSettings; diff --git a/server/src/test/java/org/opensearch/action/search/MockSearchPhaseContext.java b/server/src/test/java/org/opensearch/action/search/MockSearchPhaseContext.java index 799387573b514..f5a705c0e1033 100644 --- a/server/src/test/java/org/opensearch/action/search/MockSearchPhaseContext.java +++ b/server/src/test/java/org/opensearch/action/search/MockSearchPhaseContext.java @@ -37,7 +37,7 @@ import org.opensearch.action.OriginalIndices; import org.opensearch.common.Nullable; import org.opensearch.common.util.concurrent.AtomicArray; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.search.SearchPhaseResult; import org.opensearch.search.SearchShardTarget; import org.opensearch.search.internal.InternalSearchResponse; diff --git a/server/src/test/java/org/opensearch/action/support/replication/TransportReplicationActionTests.java b/server/src/test/java/org/opensearch/action/support/replication/TransportReplicationActionTests.java index cb39e5e0ce8c5..d0168a4b863da 100644 --- a/server/src/test/java/org/opensearch/action/support/replication/TransportReplicationActionTests.java +++ b/server/src/test/java/org/opensearch/action/support/replication/TransportReplicationActionTests.java @@ -76,7 +76,7 @@ import org.opensearch.common.network.NetworkService; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.PageCacheRecycler; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.Index; import org.opensearch.index.IndexNotFoundException; import org.opensearch.index.IndexService; diff --git a/server/src/test/java/org/opensearch/action/support/replication/TransportReplicationAllPermitsAcquisitionTests.java b/server/src/test/java/org/opensearch/action/support/replication/TransportReplicationAllPermitsAcquisitionTests.java index b82cc6510f94a..58ce94e6bd581 100644 --- a/server/src/test/java/org/opensearch/action/support/replication/TransportReplicationAllPermitsAcquisitionTests.java +++ b/server/src/test/java/org/opensearch/action/support/replication/TransportReplicationAllPermitsAcquisitionTests.java @@ -57,7 +57,7 @@ import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.IndexService; import org.opensearch.index.engine.InternalEngineFactory; import org.opensearch.index.shard.IndexShard; diff --git a/server/src/test/java/org/opensearch/action/support/replication/TransportWriteActionForIndexingPressureTests.java b/server/src/test/java/org/opensearch/action/support/replication/TransportWriteActionForIndexingPressureTests.java index 0db21af565b86..0956fddc36d01 100644 --- a/server/src/test/java/org/opensearch/action/support/replication/TransportWriteActionForIndexingPressureTests.java +++ b/server/src/test/java/org/opensearch/action/support/replication/TransportWriteActionForIndexingPressureTests.java @@ -21,7 +21,7 @@ import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.Index; import org.opensearch.index.IndexService; import org.opensearch.index.IndexingPressureService; diff --git a/server/src/test/java/org/opensearch/action/support/replication/TransportWriteActionTests.java b/server/src/test/java/org/opensearch/action/support/replication/TransportWriteActionTests.java index 18d003b1eb79b..a252991d7e119 100644 --- a/server/src/test/java/org/opensearch/action/support/replication/TransportWriteActionTests.java +++ b/server/src/test/java/org/opensearch/action/support/replication/TransportWriteActionTests.java @@ -53,7 +53,7 @@ import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.settings.Settings; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.Index; import org.opensearch.index.IndexService; import org.opensearch.index.IndexingPressureService; diff --git a/server/src/test/java/org/opensearch/cluster/coordination/ElectionSchedulerFactoryTests.java b/server/src/test/java/org/opensearch/cluster/coordination/ElectionSchedulerFactoryTests.java index 2ebf099f2b0cb..997bdc2cf362b 100644 --- a/server/src/test/java/org/opensearch/cluster/coordination/ElectionSchedulerFactoryTests.java +++ b/server/src/test/java/org/opensearch/cluster/coordination/ElectionSchedulerFactoryTests.java @@ -35,7 +35,7 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.settings.Settings.Builder; import org.opensearch.common.unit.TimeValue; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.test.OpenSearchTestCase; import java.util.concurrent.atomic.AtomicBoolean; diff --git a/server/src/test/java/org/opensearch/cluster/coordination/PreVoteCollectorTests.java b/server/src/test/java/org/opensearch/cluster/coordination/PreVoteCollectorTests.java index 19e400e27ab00..947b2273e4d37 100644 --- a/server/src/test/java/org/opensearch/cluster/coordination/PreVoteCollectorTests.java +++ b/server/src/test/java/org/opensearch/cluster/coordination/PreVoteCollectorTests.java @@ -38,7 +38,7 @@ import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.settings.Settings; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.monitor.StatusInfo; import org.opensearch.test.OpenSearchTestCase; import org.opensearch.test.transport.MockTransport; diff --git a/server/src/test/java/org/opensearch/cluster/service/TaskExecutorTests.java b/server/src/test/java/org/opensearch/cluster/service/TaskExecutorTests.java index 08c989de647b8..64a828446cf6f 100644 --- a/server/src/test/java/org/opensearch/cluster/service/TaskExecutorTests.java +++ b/server/src/test/java/org/opensearch/cluster/service/TaskExecutorTests.java @@ -39,7 +39,7 @@ import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.OpenSearchExecutors; import org.opensearch.common.util.concurrent.PrioritizedOpenSearchThreadPoolExecutor; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.test.OpenSearchTestCase; import org.opensearch.threadpool.TestThreadPool; import org.opensearch.threadpool.ThreadPool; diff --git a/server/src/test/java/org/opensearch/common/ReleasablesTests.java b/server/src/test/java/org/opensearch/common/ReleasablesTests.java index b9739a26b7b1c..4b5699418d430 100644 --- a/server/src/test/java/org/opensearch/common/ReleasablesTests.java +++ b/server/src/test/java/org/opensearch/common/ReleasablesTests.java @@ -32,8 +32,8 @@ package org.opensearch.common; import org.opensearch.test.OpenSearchTestCase; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import java.util.concurrent.atomic.AtomicInteger; diff --git a/server/src/test/java/org/opensearch/common/util/BinarySearcherTests.java b/server/src/test/java/org/opensearch/common/util/BinarySearcherTests.java index 294426c37f7af..bc64a4980947c 100644 --- a/server/src/test/java/org/opensearch/common/util/BinarySearcherTests.java +++ b/server/src/test/java/org/opensearch/common/util/BinarySearcherTests.java @@ -33,7 +33,7 @@ package org.opensearch.common.util; import org.opensearch.common.settings.Settings; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.indices.breaker.NoneCircuitBreakerService; import org.opensearch.test.OpenSearchTestCase; import org.junit.Before; diff --git a/server/src/test/java/org/opensearch/common/util/concurrent/KeyedLockTests.java b/server/src/test/java/org/opensearch/common/util/concurrent/KeyedLockTests.java index 8746c9ab72e91..77969e042c98d 100644 --- a/server/src/test/java/org/opensearch/common/util/concurrent/KeyedLockTests.java +++ b/server/src/test/java/org/opensearch/common/util/concurrent/KeyedLockTests.java @@ -32,7 +32,7 @@ package org.opensearch.common.util.concurrent; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.test.OpenSearchTestCase; import org.hamcrest.Matchers; diff --git a/server/src/test/java/org/opensearch/common/util/concurrent/ReleasableLockTests.java b/server/src/test/java/org/opensearch/common/util/concurrent/ReleasableLockTests.java index d22ef45e0aaf5..d88a281a66d89 100644 --- a/server/src/test/java/org/opensearch/common/util/concurrent/ReleasableLockTests.java +++ b/server/src/test/java/org/opensearch/common/util/concurrent/ReleasableLockTests.java @@ -33,7 +33,7 @@ package org.opensearch.common.util.concurrent; import org.opensearch.common.unit.TimeValue; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.test.OpenSearchTestCase; import java.util.ArrayList; diff --git a/server/src/test/java/org/opensearch/http/DefaultRestChannelTests.java b/server/src/test/java/org/opensearch/http/DefaultRestChannelTests.java index f5748713e8887..c0f6a9ba68383 100644 --- a/server/src/test/java/org/opensearch/http/DefaultRestChannelTests.java +++ b/server/src/test/java/org/opensearch/http/DefaultRestChannelTests.java @@ -44,7 +44,7 @@ import org.opensearch.common.util.MockBigArrays; import org.opensearch.common.util.MockPageCacheRecycler; import org.opensearch.common.xcontent.json.JsonXContent; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.indices.breaker.NoneCircuitBreakerService; import org.opensearch.rest.BytesRestResponse; diff --git a/server/src/test/java/org/opensearch/index/IndexingPressureServiceTests.java b/server/src/test/java/org/opensearch/index/IndexingPressureServiceTests.java index a41b8b4d3ca74..055ab63cd03f3 100644 --- a/server/src/test/java/org/opensearch/index/IndexingPressureServiceTests.java +++ b/server/src/test/java/org/opensearch/index/IndexingPressureServiceTests.java @@ -20,7 +20,7 @@ import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.shard.ShardId; import org.opensearch.index.stats.IndexingPressurePerShardStats; import org.opensearch.index.stats.IndexingPressureStats; diff --git a/server/src/test/java/org/opensearch/index/IndexingPressureTests.java b/server/src/test/java/org/opensearch/index/IndexingPressureTests.java index 69fc23b1293ad..dd2f381a7fc64 100644 --- a/server/src/test/java/org/opensearch/index/IndexingPressureTests.java +++ b/server/src/test/java/org/opensearch/index/IndexingPressureTests.java @@ -33,7 +33,7 @@ package org.opensearch.index; import org.opensearch.common.settings.Settings; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; import org.opensearch.index.stats.IndexingPressureStats; import org.opensearch.test.OpenSearchTestCase; diff --git a/server/src/test/java/org/opensearch/index/ShardIndexingPressureConcurrentExecutionTests.java b/server/src/test/java/org/opensearch/index/ShardIndexingPressureConcurrentExecutionTests.java index 13874828caa90..e0189c1713124 100644 --- a/server/src/test/java/org/opensearch/index/ShardIndexingPressureConcurrentExecutionTests.java +++ b/server/src/test/java/org/opensearch/index/ShardIndexingPressureConcurrentExecutionTests.java @@ -15,7 +15,7 @@ import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; import org.opensearch.index.shard.ShardId; import org.opensearch.index.stats.IndexingPressurePerShardStats; diff --git a/server/src/test/java/org/opensearch/index/ShardIndexingPressureTests.java b/server/src/test/java/org/opensearch/index/ShardIndexingPressureTests.java index 6ced3e0149016..77593660a7d12 100644 --- a/server/src/test/java/org/opensearch/index/ShardIndexingPressureTests.java +++ b/server/src/test/java/org/opensearch/index/ShardIndexingPressureTests.java @@ -9,7 +9,7 @@ package org.opensearch.index; import org.opensearch.cluster.service.ClusterService; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; diff --git a/server/src/test/java/org/opensearch/index/engine/LiveVersionMapTests.java b/server/src/test/java/org/opensearch/index/engine/LiveVersionMapTests.java index 16c07104b5035..b1e033232420b 100644 --- a/server/src/test/java/org/opensearch/index/engine/LiveVersionMapTests.java +++ b/server/src/test/java/org/opensearch/index/engine/LiveVersionMapTests.java @@ -37,7 +37,7 @@ import org.apache.lucene.util.Constants; import org.apache.lucene.tests.util.RamUsageTester; import org.apache.lucene.tests.util.TestUtil; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.translog.Translog; import org.opensearch.test.OpenSearchTestCase; diff --git a/server/src/test/java/org/opensearch/index/engine/SoftDeletesPolicyTests.java b/server/src/test/java/org/opensearch/index/engine/SoftDeletesPolicyTests.java index 643d4cd28da6e..07d49b25595b4 100644 --- a/server/src/test/java/org/opensearch/index/engine/SoftDeletesPolicyTests.java +++ b/server/src/test/java/org/opensearch/index/engine/SoftDeletesPolicyTests.java @@ -35,7 +35,7 @@ import org.apache.lucene.document.LongPoint; import org.apache.lucene.search.PointRangeQuery; import org.apache.lucene.search.Query; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.seqno.RetentionLease; import org.opensearch.index.seqno.RetentionLeases; import org.opensearch.test.OpenSearchTestCase; diff --git a/server/src/test/java/org/opensearch/index/replication/RecoveryDuringReplicationTests.java b/server/src/test/java/org/opensearch/index/replication/RecoveryDuringReplicationTests.java index d73cdfd3fe93f..b733f81fdb701 100644 --- a/server/src/test/java/org/opensearch/index/replication/RecoveryDuringReplicationTests.java +++ b/server/src/test/java/org/opensearch/index/replication/RecoveryDuringReplicationTests.java @@ -50,7 +50,7 @@ import org.opensearch.common.lucene.uid.Versions; import org.opensearch.common.settings.Settings; import org.opensearch.common.xcontent.XContentType; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.IndexSettings; import org.opensearch.index.VersionType; import org.opensearch.index.engine.DocIdSeqNoAndSource; diff --git a/server/src/test/java/org/opensearch/index/seqno/RetentionLeaseActionsTests.java b/server/src/test/java/org/opensearch/index/seqno/RetentionLeaseActionsTests.java index 73513d776464f..936e8a65156c0 100644 --- a/server/src/test/java/org/opensearch/index/seqno/RetentionLeaseActionsTests.java +++ b/server/src/test/java/org/opensearch/index/seqno/RetentionLeaseActionsTests.java @@ -38,7 +38,7 @@ import org.opensearch.action.admin.indices.stats.IndicesStatsResponse; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.IndexService; import org.opensearch.index.IndexSettings; import org.opensearch.index.shard.ShardId; diff --git a/server/src/test/java/org/opensearch/index/shard/IndexShardOperationPermitsTests.java b/server/src/test/java/org/opensearch/index/shard/IndexShardOperationPermitsTests.java index 4dcab737787aa..5e6062a252938 100644 --- a/server/src/test/java/org/opensearch/index/shard/IndexShardOperationPermitsTests.java +++ b/server/src/test/java/org/opensearch/index/shard/IndexShardOperationPermitsTests.java @@ -37,7 +37,7 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.util.concurrent.OpenSearchThreadPoolExecutor; import org.opensearch.common.util.concurrent.ThreadContext; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; import org.opensearch.test.OpenSearchTestCase; import org.opensearch.threadpool.TestThreadPool; diff --git a/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java index 67f149c3cb5ae..1c0ebf17285f7 100644 --- a/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java @@ -88,8 +88,8 @@ import org.opensearch.common.util.concurrent.ConcurrentCollections; import org.opensearch.common.xcontent.XContentFactory; import org.opensearch.common.xcontent.XContentType; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.env.NodeEnvironment; diff --git a/server/src/test/java/org/opensearch/index/shard/RefreshListenersTests.java b/server/src/test/java/org/opensearch/index/shard/RefreshListenersTests.java index 39c91ffa5f482..7d878c488a88e 100644 --- a/server/src/test/java/org/opensearch/index/shard/RefreshListenersTests.java +++ b/server/src/test/java/org/opensearch/index/shard/RefreshListenersTests.java @@ -52,7 +52,7 @@ import org.opensearch.common.util.concurrent.ThreadContext; import org.opensearch.common.xcontent.XContentType; import org.opensearch.common.util.io.IOUtils; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.Index; import org.opensearch.index.IndexSettings; import org.opensearch.index.codec.CodecService; 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 f70eec877dbc9..1c2ddf43f8274 100644 --- a/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java +++ b/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java @@ -23,7 +23,7 @@ import org.opensearch.common.concurrent.GatedCloseable; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.engine.InternalEngineFactory; import org.opensearch.index.remote.RemoteRefreshSegmentPressureService; import org.opensearch.index.remote.RemoteRefreshSegmentTracker; diff --git a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java index 8c0e7be30ad19..0c859c5f6a64a 100644 --- a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java @@ -27,7 +27,7 @@ import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.CancellableThreads; import org.opensearch.common.xcontent.XContentType; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.IndexSettings; import org.opensearch.index.engine.DocIdSeqNoAndSource; import org.opensearch.index.engine.InternalEngine; diff --git a/server/src/test/java/org/opensearch/index/translog/TranslogDeletionPolicyTests.java b/server/src/test/java/org/opensearch/index/translog/TranslogDeletionPolicyTests.java index fba68427285ed..86ac0c9405eae 100644 --- a/server/src/test/java/org/opensearch/index/translog/TranslogDeletionPolicyTests.java +++ b/server/src/test/java/org/opensearch/index/translog/TranslogDeletionPolicyTests.java @@ -38,7 +38,7 @@ import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.io.IOUtils; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.shard.ShardId; import org.opensearch.test.OpenSearchTestCase; import org.mockito.Mockito; diff --git a/server/src/test/java/org/opensearch/indices/recovery/LocalStorePeerRecoverySourceHandlerTests.java b/server/src/test/java/org/opensearch/indices/recovery/LocalStorePeerRecoverySourceHandlerTests.java index cc691ba82c67c..5654dad580ab6 100644 --- a/server/src/test/java/org/opensearch/indices/recovery/LocalStorePeerRecoverySourceHandlerTests.java +++ b/server/src/test/java/org/opensearch/indices/recovery/LocalStorePeerRecoverySourceHandlerTests.java @@ -72,7 +72,7 @@ import org.opensearch.common.util.concurrent.ConcurrentCollections; import org.opensearch.common.xcontent.XContentType; import org.opensearch.common.util.io.IOUtils; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.index.IndexSettings; import org.opensearch.index.engine.Engine; import org.opensearch.index.engine.RecoveryEngineException; diff --git a/server/src/test/java/org/opensearch/search/aggregations/metrics/InternalCardinalityTests.java b/server/src/test/java/org/opensearch/search/aggregations/metrics/InternalCardinalityTests.java index 76cf2710553e5..8a513d9f45dde 100644 --- a/server/src/test/java/org/opensearch/search/aggregations/metrics/InternalCardinalityTests.java +++ b/server/src/test/java/org/opensearch/search/aggregations/metrics/InternalCardinalityTests.java @@ -36,7 +36,7 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.util.MockBigArrays; import org.opensearch.common.util.MockPageCacheRecycler; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.indices.breaker.NoneCircuitBreakerService; import org.opensearch.search.aggregations.ParsedAggregation; import org.opensearch.test.InternalAggregationTestCase; diff --git a/server/src/test/java/org/opensearch/search/sort/BucketedSortTestCase.java b/server/src/test/java/org/opensearch/search/sort/BucketedSortTestCase.java index 042089f85ced7..98c04eaf9415a 100644 --- a/server/src/test/java/org/opensearch/search/sort/BucketedSortTestCase.java +++ b/server/src/test/java/org/opensearch/search/sort/BucketedSortTestCase.java @@ -39,7 +39,7 @@ import org.opensearch.common.util.IntArray; import org.opensearch.common.util.MockBigArrays; import org.opensearch.common.util.MockPageCacheRecycler; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.indices.breaker.NoneCircuitBreakerService; import org.opensearch.search.DocValueFormat; import org.opensearch.test.OpenSearchTestCase; diff --git a/server/src/test/java/org/opensearch/tasks/TaskManagerTests.java b/server/src/test/java/org/opensearch/tasks/TaskManagerTests.java index 73da1ec6ffe48..31f3c0a7da094 100644 --- a/server/src/test/java/org/opensearch/tasks/TaskManagerTests.java +++ b/server/src/test/java/org/opensearch/tasks/TaskManagerTests.java @@ -37,8 +37,8 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.concurrent.ConcurrentCollections; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.test.OpenSearchTestCase; import org.opensearch.threadpool.RunnableTaskExecutionListener; import org.opensearch.threadpool.TestThreadPool; diff --git a/server/src/test/java/org/opensearch/transport/InboundPipelineTests.java b/server/src/test/java/org/opensearch/transport/InboundPipelineTests.java index fbaac1a03bebe..12742c3228bfd 100644 --- a/server/src/test/java/org/opensearch/transport/InboundPipelineTests.java +++ b/server/src/test/java/org/opensearch/transport/InboundPipelineTests.java @@ -47,7 +47,7 @@ import org.opensearch.common.util.PageCacheRecycler; import org.opensearch.common.util.concurrent.ThreadContext; import org.opensearch.common.util.io.Streams; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.test.OpenSearchTestCase; import java.io.IOException; diff --git a/server/src/test/java/org/opensearch/transport/TransportDecompressorTests.java b/server/src/test/java/org/opensearch/transport/TransportDecompressorTests.java index 4675f12b3e086..d5cd6dc49be12 100644 --- a/server/src/test/java/org/opensearch/transport/TransportDecompressorTests.java +++ b/server/src/test/java/org/opensearch/transport/TransportDecompressorTests.java @@ -42,7 +42,7 @@ import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.util.PageCacheRecycler; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.test.OpenSearchTestCase; import java.io.IOException; diff --git a/test/framework/src/main/java/org/opensearch/cluster/coordination/AbstractCoordinatorTestCase.java b/test/framework/src/main/java/org/opensearch/cluster/coordination/AbstractCoordinatorTestCase.java index 2e1891f94fabe..1b42b2f70920f 100644 --- a/test/framework/src/main/java/org/opensearch/cluster/coordination/AbstractCoordinatorTestCase.java +++ b/test/framework/src/main/java/org/opensearch/cluster/coordination/AbstractCoordinatorTestCase.java @@ -72,7 +72,7 @@ import org.opensearch.common.util.MockBigArrays; import org.opensearch.common.util.MockPageCacheRecycler; import org.opensearch.common.util.concurrent.PrioritizedOpenSearchThreadPoolExecutor; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.discovery.DiscoveryModule; import org.opensearch.discovery.SeedHostsProvider; import org.opensearch.env.NodeEnvironment; diff --git a/test/framework/src/main/java/org/opensearch/index/replication/OpenSearchIndexLevelReplicationTestCase.java b/test/framework/src/main/java/org/opensearch/index/replication/OpenSearchIndexLevelReplicationTestCase.java index 9caaf09a217e4..b815a819756b7 100644 --- a/test/framework/src/main/java/org/opensearch/index/replication/OpenSearchIndexLevelReplicationTestCase.java +++ b/test/framework/src/main/java/org/opensearch/index/replication/OpenSearchIndexLevelReplicationTestCase.java @@ -79,8 +79,8 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.xcontent.XContentType; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.index.Index; import org.opensearch.index.IndexSettings; import org.opensearch.index.engine.DocIdSeqNoAndSource; diff --git a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java index 22b6693ee09ec..659f473403ec8 100644 --- a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java @@ -74,7 +74,7 @@ import org.opensearch.common.util.BigArrays; import org.opensearch.common.xcontent.XContentType; import org.opensearch.common.util.io.IOUtils; -import org.opensearch.core.common.lease.Releasable; +import org.opensearch.common.lease.Releasable; import org.opensearch.env.NodeEnvironment; import org.opensearch.index.Index; import org.opensearch.index.IndexSettings; diff --git a/test/framework/src/main/java/org/opensearch/search/aggregations/AggregatorTestCase.java b/test/framework/src/main/java/org/opensearch/search/aggregations/AggregatorTestCase.java index 56a9455c7af34..48eea430aca76 100644 --- a/test/framework/src/main/java/org/opensearch/search/aggregations/AggregatorTestCase.java +++ b/test/framework/src/main/java/org/opensearch/search/aggregations/AggregatorTestCase.java @@ -71,8 +71,8 @@ import org.opensearch.common.util.BigArrays; import org.opensearch.common.util.MockBigArrays; import org.opensearch.common.util.MockPageCacheRecycler; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.core.xcontent.ContextParser; import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.index.Index; 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 a3612167f16c3..38617d09b703d 100644 --- a/test/framework/src/main/java/org/opensearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/opensearch/test/InternalTestCluster.java @@ -84,7 +84,7 @@ 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.lease.Releasables; +import org.opensearch.common.lease.Releasables; import org.opensearch.core.common.Strings; import org.opensearch.core.util.FileSystemUtils; import org.opensearch.env.Environment; diff --git a/test/framework/src/main/java/org/opensearch/transport/nio/MockNioTransport.java b/test/framework/src/main/java/org/opensearch/transport/nio/MockNioTransport.java index aab513b31929b..cc6f08d427860 100644 --- a/test/framework/src/main/java/org/opensearch/transport/nio/MockNioTransport.java +++ b/test/framework/src/main/java/org/opensearch/transport/nio/MockNioTransport.java @@ -51,8 +51,8 @@ import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.PageCacheRecycler; import org.opensearch.common.util.io.IOUtils; -import org.opensearch.core.common.lease.Releasable; -import org.opensearch.core.common.lease.Releasables; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.lease.Releasables; import org.opensearch.indices.breaker.CircuitBreakerService; import org.opensearch.nio.BytesChannelContext; import org.opensearch.nio.BytesWriteHandler; From e049338c714db77517c5e8bb94b6b24a7b0a3281 Mon Sep 17 00:00:00 2001 From: Nathan Myles Date: Fri, 16 Jun 2023 13:55:30 -0300 Subject: [PATCH 033/109] Fix mapping char_filter when mapping a hashtag (#7591) Signed-off-by: Nathan Myles Issue: https://github.com/opensearch-project/OpenSearch/issues/7308 --- CHANGELOG.md | 1 + .../common/MappingCharFilterFactory.java | 2 +- .../common/MappingCharFilterFactoryTests.java | 8 +++---- .../test/analysis-common/50_char_filters.yml | 18 +++++++++++++++ .../opensearch/index/analysis/Analysis.java | 23 ++++++++++++++++++- 5 files changed, 46 insertions(+), 6 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index b6f2b534eb379..21b443991e069 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -93,6 +93,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Support OpenSSL Provider with default Netty allocator ([#5460](https://github.com/opensearch-project/OpenSearch/pull/5460)) - Replaces ZipInputStream with ZipFile to fix Zip Slip vulnerability ([#7230](https://github.com/opensearch-project/OpenSearch/pull/7230)) - Add missing validation/parsing of SearchBackpressureMode of SearchBackpressureSettings ([#7541](https://github.com/opensearch-project/OpenSearch/pull/7541)) +- Fix mapping char_filter when mapping a hashtag ([#7591](https://github.com/opensearch-project/OpenSearch/pull/7591)) ### Security diff --git a/modules/analysis-common/src/main/java/org/opensearch/analysis/common/MappingCharFilterFactory.java b/modules/analysis-common/src/main/java/org/opensearch/analysis/common/MappingCharFilterFactory.java index d6d9f8975f2fc..bd241de749f11 100644 --- a/modules/analysis-common/src/main/java/org/opensearch/analysis/common/MappingCharFilterFactory.java +++ b/modules/analysis-common/src/main/java/org/opensearch/analysis/common/MappingCharFilterFactory.java @@ -54,7 +54,7 @@ public class MappingCharFilterFactory extends AbstractCharFilterFactory implemen MappingCharFilterFactory(IndexSettings indexSettings, Environment env, String name, Settings settings) { super(indexSettings, name); - List> rules = Analysis.parseWordList(env, settings, "mappings", this::parse); + List> rules = Analysis.parseWordList(env, settings, "mappings", this::parse, false); if (rules == null) { throw new IllegalArgumentException("mapping requires either `mappings` or `mappings_path` to be configured"); } diff --git a/modules/analysis-common/src/test/java/org/opensearch/analysis/common/MappingCharFilterFactoryTests.java b/modules/analysis-common/src/test/java/org/opensearch/analysis/common/MappingCharFilterFactoryTests.java index 7d059ff9ce1da..387eb4a377007 100644 --- a/modules/analysis-common/src/test/java/org/opensearch/analysis/common/MappingCharFilterFactoryTests.java +++ b/modules/analysis-common/src/test/java/org/opensearch/analysis/common/MappingCharFilterFactoryTests.java @@ -37,11 +37,11 @@ public static CharFilterFactory create(String... rules) throws IOException { public void testRulesOk() throws IOException { MappingCharFilterFactory mappingCharFilterFactory = (MappingCharFilterFactory) create( - "# This is a comment", + "# => _hashtag_", ":) => _happy_", ":( => _sad_" ); - CharFilter inputReader = (CharFilter) mappingCharFilterFactory.create(new StringReader("I'm so :)")); + CharFilter inputReader = (CharFilter) mappingCharFilterFactory.create(new StringReader("I'm so :), I'm so :( #confused")); char[] tempBuff = new char[14]; StringBuilder output = new StringBuilder(); while (true) { @@ -49,7 +49,7 @@ public void testRulesOk() throws IOException { if (length == -1) break; output.append(tempBuff, 0, length); } - assertEquals("I'm so _happy_", output.toString()); + assertEquals("I'm so _happy_, I'm so _sad_ _hashtag_confused", output.toString()); } public void testRuleError() { @@ -64,7 +64,7 @@ public void testRuleError() { } public void testRulePartError() { - RuntimeException ex = expectThrows(RuntimeException.class, () -> create("# This is a comment", ":) => _happy_", "a:b")); + RuntimeException ex = expectThrows(RuntimeException.class, () -> create("# => _hashtag_", ":) => _happy_", "a:b")); assertEquals("Line [3]: Invalid mapping rule : [a:b]", ex.getMessage()); } } diff --git a/modules/analysis-common/src/yamlRestTest/resources/rest-api-spec/test/analysis-common/50_char_filters.yml b/modules/analysis-common/src/yamlRestTest/resources/rest-api-spec/test/analysis-common/50_char_filters.yml index 67e68428c07c7..0078575ae8e57 100644 --- a/modules/analysis-common/src/yamlRestTest/resources/rest-api-spec/test/analysis-common/50_char_filters.yml +++ b/modules/analysis-common/src/yamlRestTest/resources/rest-api-spec/test/analysis-common/50_char_filters.yml @@ -57,3 +57,21 @@ - match: { detail.tokenizer.tokens.0.start_offset: 0 } - match: { detail.tokenizer.tokens.0.end_offset: 15 } - match: { detail.tokenizer.tokens.0.position: 0 } +--- +"mapping_with_hashtag": + - do: + indices.analyze: + body: + text: 'test #test @test' + tokenizer: standard + filter: + - lowercase + char_filter: + - type: mapping + mappings: + - "# => _hashsign_" + - "@ => _atsign_" + - length: { tokens: 3 } + - match: { tokens.0.token: test } + - match: { tokens.1.token: _hashsign_test } + - match: { tokens.2.token: _atsign_test } 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 52860fbf1dc3b..f4465c9dffac6 100644 --- a/server/src/main/java/org/opensearch/index/analysis/Analysis.java +++ b/server/src/main/java/org/opensearch/index/analysis/Analysis.java @@ -222,6 +222,16 @@ public static List parseWordList(Environment env, Settings settings, Stri return parseWordList(env, settings, settingPrefix + "_path", settingPrefix, parser); } + public static List parseWordList( + Environment env, + Settings settings, + String settingPrefix, + CustomMappingRuleParser parser, + boolean removeComments + ) { + return parseWordList(env, settings, settingPrefix + "_path", settingPrefix, parser, removeComments); + } + /** * Parses a list of words from the specified settings or from a file, with the given parser. * @@ -236,6 +246,17 @@ public static List parseWordList( String settingPath, String settingList, CustomMappingRuleParser parser + ) { + return parseWordList(env, settings, settingPath, settingList, parser, true); + } + + public static List parseWordList( + Environment env, + Settings settings, + String settingPath, + String settingList, + CustomMappingRuleParser parser, + boolean removeComments ) { List words = getWordList(env, settings, settingPath, settingList); if (words == null) { @@ -245,7 +266,7 @@ public static List parseWordList( int lineNum = 0; for (String word : words) { lineNum++; - if (word.startsWith("#") == false) { + if (removeComments == false || word.startsWith("#") == false) { try { rules.add(parser.apply(word)); } catch (RuntimeException ex) { From a31e159ad9965536e9f0a8ffce0f3d50243705a8 Mon Sep 17 00:00:00 2001 From: Stephen Crawford <65832608+scrawfor99@users.noreply.github.com> Date: Fri, 16 Jun 2023 13:49:13 -0400 Subject: [PATCH 034/109] Update all guava (#8107) * Update all guava Signed-off-by: Stephen Crawford * changelog Signed-off-by: Stephen Crawford * remove ignors Signed-off-by: Stephen Crawford * Fix ignores Signed-off-by: Stephen Crawford * Update plugins/repository-gcs/build.gradle Co-authored-by: Andriy Redko Signed-off-by: Stephen Crawford <65832608+scrawfor99@users.noreply.github.com> * Update plugins/repository-azure/build.gradle Co-authored-by: Andriy Redko Signed-off-by: Stephen Crawford <65832608+scrawfor99@users.noreply.github.com> --------- Signed-off-by: Stephen Crawford Signed-off-by: Stephen Crawford <65832608+scrawfor99@users.noreply.github.com> Co-authored-by: Andriy Redko --- CHANGELOG.md | 3 ++- buildSrc/version.properties | 2 +- plugins/ingest-attachment/build.gradle | 1 - plugins/ingest-attachment/licenses/guava-31.1-jre.jar.sha1 | 1 - plugins/ingest-attachment/licenses/guava-32.0.1-jre.jar.sha1 | 1 + plugins/repository-azure/build.gradle | 3 +-- plugins/repository-azure/licenses/guava-31.1-jre.jar.sha1 | 1 - plugins/repository-azure/licenses/guava-32.0.1-jre.jar.sha1 | 1 + plugins/repository-gcs/build.gradle | 3 +-- plugins/repository-gcs/licenses/guava-31.1-jre.jar.sha1 | 1 - plugins/repository-gcs/licenses/guava-32.0.1-jre.jar.sha1 | 1 + plugins/repository-hdfs/build.gradle | 1 - plugins/repository-hdfs/licenses/guava-31.1-jre.jar.sha1 | 1 - plugins/repository-hdfs/licenses/guava-32.0.1-jre.jar.sha1 | 1 + qa/evil-tests/build.gradle | 2 +- 15 files changed, 10 insertions(+), 13 deletions(-) delete mode 100644 plugins/ingest-attachment/licenses/guava-31.1-jre.jar.sha1 create mode 100644 plugins/ingest-attachment/licenses/guava-32.0.1-jre.jar.sha1 delete mode 100644 plugins/repository-azure/licenses/guava-31.1-jre.jar.sha1 create mode 100644 plugins/repository-azure/licenses/guava-32.0.1-jre.jar.sha1 delete mode 100644 plugins/repository-gcs/licenses/guava-31.1-jre.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/guava-32.0.1-jre.jar.sha1 delete mode 100644 plugins/repository-hdfs/licenses/guava-31.1-jre.jar.sha1 create mode 100644 plugins/repository-hdfs/licenses/guava-32.0.1-jre.jar.sha1 diff --git a/CHANGELOG.md b/CHANGELOG.md index 21b443991e069..76f0069dfa079 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -53,6 +53,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `io.projectreactor:reactor-core` from 3.5.1 to 3.5.6 in /plugins/repository-azure ([#8016](https://github.com/opensearch-project/OpenSearch/pull/8016)) - Bump `com.google.guava:guava` from 32.0.0-jre to 32.0.1-jre in /distribution/tools/upgrade-cli ([#8011](https://github.com/opensearch-project/OpenSearch/pull/8011)) - Bump `com.google.guava:guava` from 32.0.0-jre to 32.0.1-jre in /distribution/tools/plugin-cli ([#8012](https://github.com/opensearch-project/OpenSearch/pull/8012)) +- Bump all `com.google.guava:guava` to 32.0.1-jre ([#8107](https://github.com/opensearch-project/OpenSearch/pull/8107)) ### Changed - [CCR] Add getHistoryOperationsFromTranslog method to fetch the history snapshot from translogs ([#3948](https://github.com/opensearch-project/OpenSearch/pull/3948)) @@ -66,7 +67,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Change `com.amazonaws.sdk.ec2MetadataServiceEndpointOverride` to `aws.ec2MetadataServiceEndpoint` ([7372](https://github.com/opensearch-project/OpenSearch/pull/7372/)) - Change `com.amazonaws.sdk.stsEndpointOverride` to `aws.stsEndpointOverride` ([7372](https://github.com/opensearch-project/OpenSearch/pull/7372/)) - Align range and default value for deletes_pct_allowed in merge policy ([#7730](https://github.com/opensearch-project/OpenSearch/pull/7730)) -- Rename QueryPhase actors like Suggest, Rescore to be processors rather than phase ([#8025](https://github.com/opensearch-project/OpenSearch/pull/8025)) +- Rename QueryPhase actors like Suggest, Rescore to be processors rather than phase ([#8025](https://github.com/opensearch-project/OpenSearch/pull/8025)) ### Deprecated diff --git a/buildSrc/version.properties b/buildSrc/version.properties index 7a4c7321df268..33fad06328b80 100644 --- a/buildSrc/version.properties +++ b/buildSrc/version.properties @@ -22,7 +22,7 @@ jettison = 1.5.4 woodstox = 6.4.0 kotlin = 1.7.10 antlr4 = 4.11.1 -guava = 31.1-jre +guava = 32.0.1-jre protobuf = 3.22.3 jakarta_annotation = 1.3.5 diff --git a/plugins/ingest-attachment/build.gradle b/plugins/ingest-attachment/build.gradle index 5f8d7dbcd9ccd..1f3c80909733c 100644 --- a/plugins/ingest-attachment/build.gradle +++ b/plugins/ingest-attachment/build.gradle @@ -126,7 +126,6 @@ thirdPartyAudit { 'com.google.common.hash.LittleEndianByteArray$UnsafeByteArray', 'com.google.common.hash.LittleEndianByteArray$UnsafeByteArray$1', 'com.google.common.hash.LittleEndianByteArray$UnsafeByteArray$2', - 'com.google.common.hash.LittleEndianByteArray$UnsafeByteArray$3', 'com.google.common.hash.Striped64', 'com.google.common.hash.Striped64$1', 'com.google.common.hash.Striped64$Cell', diff --git a/plugins/ingest-attachment/licenses/guava-31.1-jre.jar.sha1 b/plugins/ingest-attachment/licenses/guava-31.1-jre.jar.sha1 deleted file mode 100644 index e57390ebe1299..0000000000000 --- a/plugins/ingest-attachment/licenses/guava-31.1-jre.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -60458f877d055d0c9114d9e1a2efb737b4bc282c \ No newline at end of file diff --git a/plugins/ingest-attachment/licenses/guava-32.0.1-jre.jar.sha1 b/plugins/ingest-attachment/licenses/guava-32.0.1-jre.jar.sha1 new file mode 100644 index 0000000000000..80dc9e9308a6c --- /dev/null +++ b/plugins/ingest-attachment/licenses/guava-32.0.1-jre.jar.sha1 @@ -0,0 +1 @@ +6e5d51a72d142f2d40a57dfb897188b36a95b489 \ No newline at end of file diff --git a/plugins/repository-azure/build.gradle b/plugins/repository-azure/build.gradle index d711950f1392d..da61be7b5cfc4 100644 --- a/plugins/repository-azure/build.gradle +++ b/plugins/repository-azure/build.gradle @@ -69,7 +69,7 @@ dependencies { api "com.fasterxml.jackson.module:jackson-module-jaxb-annotations:${versions.jackson}" api 'org.codehaus.woodstox:stax2-api:4.2.1' implementation "com.fasterxml.woodstox:woodstox-core:${versions.woodstox}" - runtimeOnly 'com.google.guava:guava:31.1-jre' + runtimeOnly "com.google.guava:guava:${versions.guava}" api 'org.apache.commons:commons-lang3:3.12.0' testImplementation project(':test:fixtures:azure-fixture') } @@ -206,7 +206,6 @@ thirdPartyAudit { 'com.google.common.hash.Striped64$Cell', 'com.google.common.hash.LittleEndianByteArray$UnsafeByteArray$1', 'com.google.common.hash.LittleEndianByteArray$UnsafeByteArray$2', - 'com.google.common.hash.LittleEndianByteArray$UnsafeByteArray$3', 'com.google.common.util.concurrent.AbstractFuture$UnsafeAtomicHelper', 'com.google.common.util.concurrent.AbstractFuture$UnsafeAtomicHelper$1', 'com.google.common.hash.LittleEndianByteArray$UnsafeByteArray', diff --git a/plugins/repository-azure/licenses/guava-31.1-jre.jar.sha1 b/plugins/repository-azure/licenses/guava-31.1-jre.jar.sha1 deleted file mode 100644 index e57390ebe1299..0000000000000 --- a/plugins/repository-azure/licenses/guava-31.1-jre.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -60458f877d055d0c9114d9e1a2efb737b4bc282c \ No newline at end of file diff --git a/plugins/repository-azure/licenses/guava-32.0.1-jre.jar.sha1 b/plugins/repository-azure/licenses/guava-32.0.1-jre.jar.sha1 new file mode 100644 index 0000000000000..80dc9e9308a6c --- /dev/null +++ b/plugins/repository-azure/licenses/guava-32.0.1-jre.jar.sha1 @@ -0,0 +1 @@ +6e5d51a72d142f2d40a57dfb897188b36a95b489 \ No newline at end of file diff --git a/plugins/repository-gcs/build.gradle b/plugins/repository-gcs/build.gradle index 0b13a2e09afbf..5d7a4c64ae34a 100644 --- a/plugins/repository-gcs/build.gradle +++ b/plugins/repository-gcs/build.gradle @@ -54,7 +54,7 @@ versions << [ dependencies { api 'com.google.cloud:google-cloud-storage:1.113.1' api 'com.google.cloud:google-cloud-core:2.5.10' - runtimeOnly 'com.google.guava:guava:31.1-jre' + runtimeOnly "com.google.guava:guava:${versions.guava}" api 'com.google.guava:failureaccess:1.0.1' api 'com.google.http-client:google-http-client:1.43.2' api "commons-logging:commons-logging:${versions.commonslogging}" @@ -110,7 +110,6 @@ thirdPartyAudit { 'com.google.common.hash.LittleEndianByteArray$UnsafeByteArray', 'com.google.common.hash.LittleEndianByteArray$UnsafeByteArray$1', 'com.google.common.hash.LittleEndianByteArray$UnsafeByteArray$2', - 'com.google.common.hash.LittleEndianByteArray$UnsafeByteArray$3', 'com.google.common.util.concurrent.AbstractFuture$UnsafeAtomicHelper', 'com.google.common.util.concurrent.AbstractFuture$UnsafeAtomicHelper$1', 'com.google.common.hash.LittleEndianByteArray$UnsafeByteArray', diff --git a/plugins/repository-gcs/licenses/guava-31.1-jre.jar.sha1 b/plugins/repository-gcs/licenses/guava-31.1-jre.jar.sha1 deleted file mode 100644 index e57390ebe1299..0000000000000 --- a/plugins/repository-gcs/licenses/guava-31.1-jre.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -60458f877d055d0c9114d9e1a2efb737b4bc282c \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/guava-32.0.1-jre.jar.sha1 b/plugins/repository-gcs/licenses/guava-32.0.1-jre.jar.sha1 new file mode 100644 index 0000000000000..80dc9e9308a6c --- /dev/null +++ b/plugins/repository-gcs/licenses/guava-32.0.1-jre.jar.sha1 @@ -0,0 +1 @@ +6e5d51a72d142f2d40a57dfb897188b36a95b489 \ No newline at end of file diff --git a/plugins/repository-hdfs/build.gradle b/plugins/repository-hdfs/build.gradle index 7edc4def0b199..7ac54544f7a1b 100644 --- a/plugins/repository-hdfs/build.gradle +++ b/plugins/repository-hdfs/build.gradle @@ -359,7 +359,6 @@ thirdPartyAudit { 'com.google.common.hash.LittleEndianByteArray$UnsafeByteArray', 'com.google.common.hash.LittleEndianByteArray$UnsafeByteArray$1', 'com.google.common.hash.LittleEndianByteArray$UnsafeByteArray$2', - 'com.google.common.hash.LittleEndianByteArray$UnsafeByteArray$3', 'com.google.common.hash.Striped64', 'com.google.common.hash.Striped64$1', 'com.google.common.hash.Striped64$Cell', diff --git a/plugins/repository-hdfs/licenses/guava-31.1-jre.jar.sha1 b/plugins/repository-hdfs/licenses/guava-31.1-jre.jar.sha1 deleted file mode 100644 index e57390ebe1299..0000000000000 --- a/plugins/repository-hdfs/licenses/guava-31.1-jre.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -60458f877d055d0c9114d9e1a2efb737b4bc282c \ No newline at end of file diff --git a/plugins/repository-hdfs/licenses/guava-32.0.1-jre.jar.sha1 b/plugins/repository-hdfs/licenses/guava-32.0.1-jre.jar.sha1 new file mode 100644 index 0000000000000..80dc9e9308a6c --- /dev/null +++ b/plugins/repository-hdfs/licenses/guava-32.0.1-jre.jar.sha1 @@ -0,0 +1 @@ +6e5d51a72d142f2d40a57dfb897188b36a95b489 \ No newline at end of file diff --git a/qa/evil-tests/build.gradle b/qa/evil-tests/build.gradle index 19dc72c0c784f..3d97aab077251 100644 --- a/qa/evil-tests/build.gradle +++ b/qa/evil-tests/build.gradle @@ -33,7 +33,7 @@ * integration, change default filesystem impl, mess with arbitrary * threads, etc. */ - + import org.opensearch.gradle.info.BuildParams apply plugin: 'opensearch.testclusters' From a6b984ba5a2f57cf2e3260706e65da72cabaa9a8 Mon Sep 17 00:00:00 2001 From: Ralph Ursprung <39383228+rursprung@users.noreply.github.com> Date: Fri, 16 Jun 2023 20:28:15 +0200 Subject: [PATCH 035/109] remove unnecessary empty `build.gradle` files (#4723) * remove unnecessary empty `build.gradle` files these have no impact, as the targets are added by the parent `build.gradle`. thus they do not need to exist and are currently just cluttering the workspace and confusing people trying to work in that area. see also the discussion on opensearch-build#99 for some details. Signed-off-by: Ralph Ursprung * Revert removal of distribution/docker/docker-export/build.gradle Signed-off-by: Kartik Ganesh --------- Signed-off-by: Ralph Ursprung Signed-off-by: Kartik Ganesh Co-authored-by: Kartik Ganesh --- distribution/archives/darwin-arm64-tar/build.gradle | 13 ------------- distribution/archives/darwin-tar/build.gradle | 13 ------------- distribution/archives/linux-arm64-tar/build.gradle | 13 ------------- distribution/archives/linux-tar/build.gradle | 13 ------------- .../archives/no-jdk-darwin-arm64-tar/build.gradle | 13 ------------- .../archives/no-jdk-darwin-tar/build.gradle | 13 ------------- distribution/archives/no-jdk-linux-tar/build.gradle | 13 ------------- .../archives/no-jdk-windows-zip/build.gradle | 13 ------------- distribution/archives/windows-zip/build.gradle | 13 ------------- .../docker/docker-arm64-export/build.gradle | 13 ------------- .../docker/docker-s390x-export/build.gradle | 13 ------------- distribution/packages/arm64-deb/build.gradle | 13 ------------- distribution/packages/arm64-rpm/build.gradle | 13 ------------- distribution/packages/deb/build.gradle | 13 ------------- distribution/packages/no-jdk-deb/build.gradle | 13 ------------- distribution/packages/no-jdk-rpm/build.gradle | 13 ------------- distribution/packages/rpm/build.gradle | 13 ------------- 17 files changed, 221 deletions(-) delete mode 100644 distribution/archives/darwin-arm64-tar/build.gradle delete mode 100644 distribution/archives/darwin-tar/build.gradle delete mode 100644 distribution/archives/linux-arm64-tar/build.gradle delete mode 100644 distribution/archives/linux-tar/build.gradle delete mode 100644 distribution/archives/no-jdk-darwin-arm64-tar/build.gradle delete mode 100644 distribution/archives/no-jdk-darwin-tar/build.gradle delete mode 100644 distribution/archives/no-jdk-linux-tar/build.gradle delete mode 100644 distribution/archives/no-jdk-windows-zip/build.gradle delete mode 100644 distribution/archives/windows-zip/build.gradle delete mode 100644 distribution/docker/docker-arm64-export/build.gradle delete mode 100644 distribution/docker/docker-s390x-export/build.gradle delete mode 100644 distribution/packages/arm64-deb/build.gradle delete mode 100644 distribution/packages/arm64-rpm/build.gradle delete mode 100644 distribution/packages/deb/build.gradle delete mode 100644 distribution/packages/no-jdk-deb/build.gradle delete mode 100644 distribution/packages/no-jdk-rpm/build.gradle delete mode 100644 distribution/packages/rpm/build.gradle diff --git a/distribution/archives/darwin-arm64-tar/build.gradle b/distribution/archives/darwin-arm64-tar/build.gradle deleted file mode 100644 index bb3e3a302c8d6..0000000000000 --- a/distribution/archives/darwin-arm64-tar/build.gradle +++ /dev/null @@ -1,13 +0,0 @@ -/* - * 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. - * - * Modifications Copyright OpenSearch Contributors. See - * GitHub history for details. - */ - -// This file is intentionally blank. All configuration of the -// distribution is done in the parent project. diff --git a/distribution/archives/darwin-tar/build.gradle b/distribution/archives/darwin-tar/build.gradle deleted file mode 100644 index bb3e3a302c8d6..0000000000000 --- a/distribution/archives/darwin-tar/build.gradle +++ /dev/null @@ -1,13 +0,0 @@ -/* - * 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. - * - * Modifications Copyright OpenSearch Contributors. See - * GitHub history for details. - */ - -// This file is intentionally blank. All configuration of the -// distribution is done in the parent project. diff --git a/distribution/archives/linux-arm64-tar/build.gradle b/distribution/archives/linux-arm64-tar/build.gradle deleted file mode 100644 index bb3e3a302c8d6..0000000000000 --- a/distribution/archives/linux-arm64-tar/build.gradle +++ /dev/null @@ -1,13 +0,0 @@ -/* - * 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. - * - * Modifications Copyright OpenSearch Contributors. See - * GitHub history for details. - */ - -// This file is intentionally blank. All configuration of the -// distribution is done in the parent project. diff --git a/distribution/archives/linux-tar/build.gradle b/distribution/archives/linux-tar/build.gradle deleted file mode 100644 index bb3e3a302c8d6..0000000000000 --- a/distribution/archives/linux-tar/build.gradle +++ /dev/null @@ -1,13 +0,0 @@ -/* - * 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. - * - * Modifications Copyright OpenSearch Contributors. See - * GitHub history for details. - */ - -// This file is intentionally blank. All configuration of the -// distribution is done in the parent project. diff --git a/distribution/archives/no-jdk-darwin-arm64-tar/build.gradle b/distribution/archives/no-jdk-darwin-arm64-tar/build.gradle deleted file mode 100644 index bb3e3a302c8d6..0000000000000 --- a/distribution/archives/no-jdk-darwin-arm64-tar/build.gradle +++ /dev/null @@ -1,13 +0,0 @@ -/* - * 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. - * - * Modifications Copyright OpenSearch Contributors. See - * GitHub history for details. - */ - -// This file is intentionally blank. All configuration of the -// distribution is done in the parent project. diff --git a/distribution/archives/no-jdk-darwin-tar/build.gradle b/distribution/archives/no-jdk-darwin-tar/build.gradle deleted file mode 100644 index bb3e3a302c8d6..0000000000000 --- a/distribution/archives/no-jdk-darwin-tar/build.gradle +++ /dev/null @@ -1,13 +0,0 @@ -/* - * 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. - * - * Modifications Copyright OpenSearch Contributors. See - * GitHub history for details. - */ - -// This file is intentionally blank. All configuration of the -// distribution is done in the parent project. diff --git a/distribution/archives/no-jdk-linux-tar/build.gradle b/distribution/archives/no-jdk-linux-tar/build.gradle deleted file mode 100644 index bb3e3a302c8d6..0000000000000 --- a/distribution/archives/no-jdk-linux-tar/build.gradle +++ /dev/null @@ -1,13 +0,0 @@ -/* - * 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. - * - * Modifications Copyright OpenSearch Contributors. See - * GitHub history for details. - */ - -// This file is intentionally blank. All configuration of the -// distribution is done in the parent project. diff --git a/distribution/archives/no-jdk-windows-zip/build.gradle b/distribution/archives/no-jdk-windows-zip/build.gradle deleted file mode 100644 index bb3e3a302c8d6..0000000000000 --- a/distribution/archives/no-jdk-windows-zip/build.gradle +++ /dev/null @@ -1,13 +0,0 @@ -/* - * 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. - * - * Modifications Copyright OpenSearch Contributors. See - * GitHub history for details. - */ - -// This file is intentionally blank. All configuration of the -// distribution is done in the parent project. diff --git a/distribution/archives/windows-zip/build.gradle b/distribution/archives/windows-zip/build.gradle deleted file mode 100644 index bb3e3a302c8d6..0000000000000 --- a/distribution/archives/windows-zip/build.gradle +++ /dev/null @@ -1,13 +0,0 @@ -/* - * 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. - * - * Modifications Copyright OpenSearch Contributors. See - * GitHub history for details. - */ - -// This file is intentionally blank. All configuration of the -// distribution is done in the parent project. diff --git a/distribution/docker/docker-arm64-export/build.gradle b/distribution/docker/docker-arm64-export/build.gradle deleted file mode 100644 index 3506c4e39c234..0000000000000 --- a/distribution/docker/docker-arm64-export/build.gradle +++ /dev/null @@ -1,13 +0,0 @@ -/* - * 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. - * - * Modifications Copyright OpenSearch Contributors. See - * GitHub history for details. - */ - -// This file is intentionally blank. All configuration of the -// export is done in the parent project. diff --git a/distribution/docker/docker-s390x-export/build.gradle b/distribution/docker/docker-s390x-export/build.gradle deleted file mode 100644 index 3506c4e39c234..0000000000000 --- a/distribution/docker/docker-s390x-export/build.gradle +++ /dev/null @@ -1,13 +0,0 @@ -/* - * 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. - * - * Modifications Copyright OpenSearch Contributors. See - * GitHub history for details. - */ - -// This file is intentionally blank. All configuration of the -// export is done in the parent project. diff --git a/distribution/packages/arm64-deb/build.gradle b/distribution/packages/arm64-deb/build.gradle deleted file mode 100644 index bb3e3a302c8d6..0000000000000 --- a/distribution/packages/arm64-deb/build.gradle +++ /dev/null @@ -1,13 +0,0 @@ -/* - * 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. - * - * Modifications Copyright OpenSearch Contributors. See - * GitHub history for details. - */ - -// This file is intentionally blank. All configuration of the -// distribution is done in the parent project. diff --git a/distribution/packages/arm64-rpm/build.gradle b/distribution/packages/arm64-rpm/build.gradle deleted file mode 100644 index bb3e3a302c8d6..0000000000000 --- a/distribution/packages/arm64-rpm/build.gradle +++ /dev/null @@ -1,13 +0,0 @@ -/* - * 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. - * - * Modifications Copyright OpenSearch Contributors. See - * GitHub history for details. - */ - -// This file is intentionally blank. All configuration of the -// distribution is done in the parent project. diff --git a/distribution/packages/deb/build.gradle b/distribution/packages/deb/build.gradle deleted file mode 100644 index bb3e3a302c8d6..0000000000000 --- a/distribution/packages/deb/build.gradle +++ /dev/null @@ -1,13 +0,0 @@ -/* - * 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. - * - * Modifications Copyright OpenSearch Contributors. See - * GitHub history for details. - */ - -// This file is intentionally blank. All configuration of the -// distribution is done in the parent project. diff --git a/distribution/packages/no-jdk-deb/build.gradle b/distribution/packages/no-jdk-deb/build.gradle deleted file mode 100644 index bb3e3a302c8d6..0000000000000 --- a/distribution/packages/no-jdk-deb/build.gradle +++ /dev/null @@ -1,13 +0,0 @@ -/* - * 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. - * - * Modifications Copyright OpenSearch Contributors. See - * GitHub history for details. - */ - -// This file is intentionally blank. All configuration of the -// distribution is done in the parent project. diff --git a/distribution/packages/no-jdk-rpm/build.gradle b/distribution/packages/no-jdk-rpm/build.gradle deleted file mode 100644 index bb3e3a302c8d6..0000000000000 --- a/distribution/packages/no-jdk-rpm/build.gradle +++ /dev/null @@ -1,13 +0,0 @@ -/* - * 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. - * - * Modifications Copyright OpenSearch Contributors. See - * GitHub history for details. - */ - -// This file is intentionally blank. All configuration of the -// distribution is done in the parent project. diff --git a/distribution/packages/rpm/build.gradle b/distribution/packages/rpm/build.gradle deleted file mode 100644 index bb3e3a302c8d6..0000000000000 --- a/distribution/packages/rpm/build.gradle +++ /dev/null @@ -1,13 +0,0 @@ -/* - * 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. - * - * Modifications Copyright OpenSearch Contributors. See - * GitHub history for details. - */ - -// This file is intentionally blank. All configuration of the -// distribution is done in the parent project. From fb0f0e41b0e83d3703b9a6adcec0a56d434b587b Mon Sep 17 00:00:00 2001 From: "Daniel (dB.) Doubrovkine" Date: Fri, 16 Jun 2023 14:50:31 -0400 Subject: [PATCH 036/109] Remove default s3 region (#7989) * Defaulting repository-s3 region to empty string Signed-off-by: Raghuvansh Raj * Added region us-west-2 for S3ThirdPartyTests Signed-off-by: Raghuvansh Raj * Always specify a region. Signed-off-by: dblock * Fix: ./gradlew :plugins:repository-s3:yamlRestTest runs locally. Signed-off-by: dblock --------- Signed-off-by: Raghuvansh Raj Signed-off-by: dblock Co-authored-by: Raghuvansh Raj --- plugins/repository-s3/build.gradle | 23 ++++++++++++++++--- .../s3/S3BlobStoreRepositoryTests.java | 12 ++++------ .../s3/S3RepositoryThirdPartyTests.java | 1 + .../repositories/s3/S3ClientSettings.java | 2 +- .../opensearch/repositories/s3/S3Service.java | 10 +++++--- .../s3/S3BlobContainerRetriesTests.java | 2 ++ .../s3/S3ClientSettingsTests.java | 4 ++-- .../repositories/s3/S3ServiceTests.java | 2 +- .../20_repository_permanent_credentials.yml | 7 ++++++ .../30_repository_temporary_credentials.yml | 6 +++++ .../40_repository_ec2_credentials.yml | 2 ++ .../50_repository_ecs_credentials.yml | 2 ++ .../60_repository_eks_credentials.yml | 1 + 13 files changed, 56 insertions(+), 18 deletions(-) diff --git a/plugins/repository-s3/build.gradle b/plugins/repository-s3/build.gradle index 5bd9747ade9da..2250f2fc88f05 100644 --- a/plugins/repository-s3/build.gradle +++ b/plugins/repository-s3/build.gradle @@ -148,12 +148,14 @@ String s3PermanentAccessKey = System.getenv("amazon_s3_access_key") String s3PermanentSecretKey = System.getenv("amazon_s3_secret_key") String s3PermanentBucket = System.getenv("amazon_s3_bucket") String s3PermanentBasePath = System.getenv("amazon_s3_base_path") +String s3PermanentRegion = System.getenv("amazon_s3_region") String s3TemporaryAccessKey = System.getenv("amazon_s3_access_key_temporary") String s3TemporarySecretKey = System.getenv("amazon_s3_secret_key_temporary") String s3TemporarySessionToken = System.getenv("amazon_s3_session_token_temporary") String s3TemporaryBucket = System.getenv("amazon_s3_bucket_temporary") String s3TemporaryBasePath = System.getenv("amazon_s3_base_path_temporary") +String s3TemporaryRegion = System.getenv("amazon_s3_region_temporary") String s3EC2Bucket = System.getenv("amazon_s3_bucket_ec2") String s3EC2BasePath = System.getenv("amazon_s3_base_path_ec2") @@ -166,28 +168,40 @@ String s3EKSBasePath = System.getenv("amazon_s3_base_path_eks") boolean s3DisableChunkedEncoding = (new Random(Long.parseUnsignedLong(BuildParams.testSeed.tokenize(':').get(0), 16))).nextBoolean() +// TODO: remove after https://github.com/opensearch-project/opensearch-build/issues/3615 +if (s3PermanentBucket && !s3PermanentRegion) { + s3PermanentRegion = "us-west-2" +} + +if (s3TemporaryBucket && !s3TemporaryRegion) { + s3TemporaryRegion = "us-west-2" +} +// ---- + // If all these variables are missing then we are testing against the internal fixture instead, which has the following // credentials hard-coded in. -if (!s3PermanentAccessKey && !s3PermanentSecretKey && !s3PermanentBucket && !s3PermanentBasePath) { +if (!s3PermanentAccessKey && !s3PermanentSecretKey && !s3PermanentBucket && !s3PermanentBasePath && !s3PermanentRegion) { s3PermanentAccessKey = 'access_key' s3PermanentSecretKey = 'secret_key' s3PermanentBucket = 'bucket' s3PermanentBasePath = 'base_path' + s3PermanentRegion = 'region' apply plugin: 'opensearch.test.fixtures' useFixture = true -} else if (!s3PermanentAccessKey || !s3PermanentSecretKey || !s3PermanentBucket || !s3PermanentBasePath) { +} else if (!s3PermanentAccessKey || !s3PermanentSecretKey || !s3PermanentBucket || !s3PermanentBasePath || !s3PermanentRegion) { throw new IllegalArgumentException("not all options specified to run against external S3 service as permanent credentials are present") } -if (!s3TemporaryAccessKey && !s3TemporarySecretKey && !s3TemporaryBucket && !s3TemporaryBasePath && !s3TemporarySessionToken) { +if (!s3TemporaryAccessKey && !s3TemporarySecretKey && !s3TemporaryBucket && !s3TemporaryBasePath && !s3TemporarySessionToken && !s3TemporaryRegion) { s3TemporaryAccessKey = 'session_token_access_key' s3TemporarySecretKey = 'session_token_secret_key' s3TemporaryBucket = 'session_token_bucket' s3TemporaryBasePath = 'session_token_base_path' s3TemporarySessionToken = 'session_token' + s3TemporaryRegion = 'session_token_region' } else if (!s3TemporaryAccessKey || !s3TemporarySecretKey || !s3TemporaryBucket || !s3TemporaryBasePath || !s3TemporarySessionToken) { throw new IllegalArgumentException("not all options specified to run against external S3 service as temporary credentials are present") @@ -208,8 +222,10 @@ processYamlRestTestResources { Map expansions = [ 'permanent_bucket': s3PermanentBucket, 'permanent_base_path': s3PermanentBasePath + "_integration_tests_" + BuildParams.testSeed, + 'permanent_region': s3PermanentRegion, 'temporary_bucket': s3TemporaryBucket, 'temporary_base_path': s3TemporaryBasePath + "_integration_tests_" + BuildParams.testSeed, + 'temporary_region': s3TemporaryRegion, 'ec2_bucket': s3EC2Bucket, 'ec2_base_path': s3EC2BasePath, 'ecs_bucket': s3ECSBucket, @@ -384,6 +400,7 @@ TaskProvider s3ThirdPartyTest = tasks.register("s3ThirdPartyTest", Test) { systemProperty 'test.s3.account', s3PermanentAccessKey systemProperty 'test.s3.key', s3PermanentSecretKey systemProperty 'test.s3.bucket', s3PermanentBucket + systemProperty 'test.s3.region', s3PermanentRegion nonInputProperties.systemProperty 'test.s3.base', s3PermanentBasePath + "_third_party_tests_" + BuildParams.testSeed if (useFixture) { nonInputProperties.systemProperty 'test.s3.endpoint', "${-> fixtureAddress('minio-fixture', 'minio-fixture', '9000') }" diff --git a/plugins/repository-s3/src/internalClusterTest/java/org/opensearch/repositories/s3/S3BlobStoreRepositoryTests.java b/plugins/repository-s3/src/internalClusterTest/java/org/opensearch/repositories/s3/S3BlobStoreRepositoryTests.java index 1bbc973dd712f..61268cf00a77a 100644 --- a/plugins/repository-s3/src/internalClusterTest/java/org/opensearch/repositories/s3/S3BlobStoreRepositoryTests.java +++ b/plugins/repository-s3/src/internalClusterTest/java/org/opensearch/repositories/s3/S3BlobStoreRepositoryTests.java @@ -72,15 +72,12 @@ @OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST) public class S3BlobStoreRepositoryTests extends OpenSearchMockAPIBasedRepositoryIntegTestCase { - private String region; + private final String region = "test-region"; private String signerOverride; private String previousOpenSearchPathConf; @Override public void setUp() throws Exception { - if (randomBoolean()) { - region = "test-region"; - } signerOverride = AwsRequestSigner.VERSION_FOUR_SIGNER.getName(); previousOpenSearchPathConf = SocketAccess.doPrivileged(() -> System.setProperty("opensearch.path.conf", "config")); super.setUp(); @@ -147,9 +144,8 @@ protected Settings nodeSettings(int nodeOrdinal) { if (signerOverride != null) { builder.put(S3ClientSettings.SIGNER_OVERRIDE.getConcreteSettingForNamespace("test").getKey(), signerOverride); } - if (region != null) { - builder.put(S3ClientSettings.REGION.getConcreteSettingForNamespace("test").getKey(), region); - } + + builder.put(S3ClientSettings.REGION.getConcreteSettingForNamespace("test").getKey(), region); return builder.build(); } @@ -218,7 +214,7 @@ private void validateAuthHeader(HttpExchange exchange) { if ("AWS4SignerType".equals(signerOverride)) { assertThat(authorizationHeaderV4, containsString("aws4_request")); } - if (region != null && authorizationHeaderV4 != null) { + if (authorizationHeaderV4 != null) { assertThat(authorizationHeaderV4, containsString("/" + region + "/s3/")); } } diff --git a/plugins/repository-s3/src/internalClusterTest/java/org/opensearch/repositories/s3/S3RepositoryThirdPartyTests.java b/plugins/repository-s3/src/internalClusterTest/java/org/opensearch/repositories/s3/S3RepositoryThirdPartyTests.java index 301fd288bc672..c8b1670bfdd83 100644 --- a/plugins/repository-s3/src/internalClusterTest/java/org/opensearch/repositories/s3/S3RepositoryThirdPartyTests.java +++ b/plugins/repository-s3/src/internalClusterTest/java/org/opensearch/repositories/s3/S3RepositoryThirdPartyTests.java @@ -91,6 +91,7 @@ protected SecureSettings credentials() { protected void createRepository(String repoName) { Settings.Builder settings = Settings.builder() .put("bucket", System.getProperty("test.s3.bucket")) + .put("region", System.getProperty("test.s3.region", "us-west-2")) .put("base_path", System.getProperty("test.s3.base", "testpath")); final String endpoint = System.getProperty("test.s3.endpoint"); if (endpoint != null) { 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 66439aa820c12..5f6be6ac01e76 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 @@ -204,7 +204,7 @@ final class S3ClientSettings { static final Setting.AffixSetting REGION = Setting.affixKeySetting( PREFIX, "region", - key -> new Setting<>(key, "us-west-2", Function.identity(), Property.NodeScope) + key -> new Setting<>(key, "", Function.identity(), Property.NodeScope) ); /** An override for the signer to use. */ 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 f63c691a920ad..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 @@ -219,7 +219,9 @@ AmazonS3WithCredentials buildClient(final S3ClientSettings clientSettings) { // We do this because directly constructing the client is deprecated (was already deprecated in 1.1.223 too) // so this change removes that usage of a deprecated API. builder.endpointOverride(URI.create(endpoint)); - builder.region(Region.of(clientSettings.region)); + if (Strings.hasText(clientSettings.region)) { + builder.region(Region.of(clientSettings.region)); + } if (clientSettings.pathStyleAccess) { builder.forcePathStyle(true); } @@ -352,9 +354,11 @@ static AwsCredentialsProvider buildCredentials(Logger logger, S3ClientSettings c if (irsaCredentials != null) { logger.debug("Using IRSA credentials"); - final Region region = Region.of(clientSettings.region); StsClient stsClient = SocketAccess.doPrivileged(() -> { - StsClientBuilder builder = StsClient.builder().region(region); + StsClientBuilder builder = StsClient.builder(); + if (Strings.hasText(clientSettings.region)) { + builder.region(Region.of(clientSettings.region)); + } final String stsEndpoint = System.getProperty(STS_ENDPOINT_OVERRIDE_SYSTEM_PROPERTY); if (stsEndpoint != null) { diff --git a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobContainerRetriesTests.java b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobContainerRetriesTests.java index 9ef83d09f56f6..9f5ebc5afe017 100644 --- a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobContainerRetriesTests.java +++ b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobContainerRetriesTests.java @@ -75,6 +75,7 @@ import static org.hamcrest.Matchers.is; import static org.opensearch.repositories.s3.S3ClientSettings.DISABLE_CHUNKED_ENCODING; import static org.opensearch.repositories.s3.S3ClientSettings.ENDPOINT_SETTING; +import static org.opensearch.repositories.s3.S3ClientSettings.REGION; import static org.opensearch.repositories.s3.S3ClientSettings.MAX_RETRIES_SETTING; import static org.opensearch.repositories.s3.S3ClientSettings.READ_TIMEOUT_SETTING; @@ -133,6 +134,7 @@ protected BlobContainer createBlobContainer( final InetSocketAddress address = httpServer.getAddress(); final String endpoint = "http://" + InetAddresses.toUriString(address.getAddress()) + ":" + address.getPort(); clientSettings.put(ENDPOINT_SETTING.getConcreteSettingForNamespace(clientName).getKey(), endpoint); + clientSettings.put(REGION.getConcreteSettingForNamespace(clientName).getKey(), "region"); if (maxRetries != null) { clientSettings.put(MAX_RETRIES_SETTING.getConcreteSettingForNamespace(clientName).getKey(), maxRetries); diff --git a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3ClientSettingsTests.java b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3ClientSettingsTests.java index 5124fe60e30e7..130b8efca0512 100644 --- a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3ClientSettingsTests.java +++ b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3ClientSettingsTests.java @@ -286,7 +286,7 @@ public void testRegionCanBeSet() { Settings.builder().put("s3.client.other.region", region).build(), configPath() ); - assertThat(settings.get("default").region, is("us-west-2")); + assertThat(settings.get("default").region, is("")); assertThat(settings.get("other").region, is(region)); try ( S3Service s3Service = new S3Service(configPath()); @@ -303,7 +303,7 @@ public void testSignerOverrideCanBeSet() { Settings.builder().put("s3.client.other.signer_override", signerOverride).build(), configPath() ); - assertThat(settings.get("default").region, is("us-west-2")); + assertThat(settings.get("default").region, is("")); assertThat(settings.get("other").signerOverride, is(signerOverride)); ClientOverrideConfiguration defaultConfiguration = SocketAccess.doPrivileged( diff --git a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3ServiceTests.java b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3ServiceTests.java index 6facea7d534ca..400905eec8b1c 100644 --- a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3ServiceTests.java +++ b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3ServiceTests.java @@ -40,7 +40,7 @@ public class S3ServiceTests extends AbstractS3RepositoryTestCase { public void testCachedClientsAreReleased() { final S3Service s3Service = new S3Service(configPath()); - final Settings settings = Settings.builder().put("endpoint", "http://first").build(); + final Settings settings = Settings.builder().put("endpoint", "http://first").put("region", "region").build(); final RepositoryMetadata metadata1 = new RepositoryMetadata("first", "s3", settings); final RepositoryMetadata metadata2 = new RepositoryMetadata("second", "s3", settings); final S3ClientSettings clientSettings = s3Service.settings(metadata2); diff --git a/plugins/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/20_repository_permanent_credentials.yml b/plugins/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/20_repository_permanent_credentials.yml index a0c2d2e593a47..5590122e26b86 100644 --- a/plugins/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/20_repository_permanent_credentials.yml +++ b/plugins/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/20_repository_permanent_credentials.yml @@ -11,6 +11,7 @@ setup: type: s3 settings: bucket: ${permanent_bucket} + region: ${permanent_region} client: integration_test_permanent base_path: "${permanent_base_path}" canned_acl: private @@ -42,6 +43,7 @@ setup: type: s3 settings: bucket: ${permanent_bucket} + region: ${permanent_region} client: integration_test_permanent base_path: "${permanent_base_path}" endpoint: 127.0.0.1:5 @@ -57,6 +59,7 @@ setup: type: s3 settings: bucket: ${permanent_bucket} + region: ${permanent_region} client: integration_test_permanent base_path: "${permanent_base_path}" endpoint: 127.0.0.1:5 @@ -275,6 +278,7 @@ setup: type: s3 settings: bucket: zHHkfSqlbnBsbpSgvCYtxrEfFLqghXtyPvvvKPNBnRCicNHQLE + region: ${permanent_region} client: integration_test_permanent --- @@ -288,6 +292,7 @@ setup: type: s3 settings: bucket: repository_permanent + region: ${permanent_region} client: unknown --- @@ -302,6 +307,7 @@ setup: settings: readonly: true bucket: zHHkfSqlbnBsbpSgvCYtxrEfFLqghXtyPvvvKPNBnRCicNHQLE + region: ${permanent_region} client: integration_test_permanent --- @@ -316,6 +322,7 @@ setup: settings: readonly: true bucket: repository_permanent + region: ${permanent_region} client: unknown --- diff --git a/plugins/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/30_repository_temporary_credentials.yml b/plugins/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/30_repository_temporary_credentials.yml index d26a07eec85dc..741e96110ea96 100644 --- a/plugins/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/30_repository_temporary_credentials.yml +++ b/plugins/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/30_repository_temporary_credentials.yml @@ -11,6 +11,7 @@ setup: type: s3 settings: bucket: ${temporary_bucket} + region: ${temporary_region} client: integration_test_temporary base_path: "${temporary_base_path}" canned_acl: private @@ -26,6 +27,7 @@ setup: repository: repository_temporary - match: { repository_temporary.settings.bucket : ${temporary_bucket} } + - match: { repository_temporary.settings.region : ${temporary_region} } - match: { repository_temporary.settings.client : "integration_test_temporary" } - match: { repository_temporary.settings.base_path : "${temporary_base_path}" } - match: { repository_temporary.settings.canned_acl : "private" } @@ -185,6 +187,7 @@ setup: type: s3 settings: bucket: zHHkfSqlbnBsbpSgvCYtxrEfFLqghXtyPvvvKPNBnRCicNHQLE + region: ${temporary_region} client: integration_test_temporary --- @@ -198,6 +201,7 @@ setup: type: s3 settings: bucket: repository_temporary + region: ${temporary_region} client: unknown --- @@ -212,6 +216,7 @@ setup: settings: readonly: true bucket: zHHkfSqlbnBsbpSgvCYtxrEfFLqghXtyPvvvKPNBnRCicNHQLE + region: ${temporary_region} client: integration_test_temporary --- @@ -226,6 +231,7 @@ setup: settings: readonly: true bucket: repository_temporary + region: ${temporary_region} client: unknown --- diff --git a/plugins/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/40_repository_ec2_credentials.yml b/plugins/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/40_repository_ec2_credentials.yml index 6d3b174b99863..8d4349845a1f6 100644 --- a/plugins/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/40_repository_ec2_credentials.yml +++ b/plugins/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/40_repository_ec2_credentials.yml @@ -11,6 +11,7 @@ setup: type: s3 settings: bucket: ${ec2_bucket} + region: region client: integration_test_ec2 base_path: "${ec2_base_path}" canned_acl: private @@ -30,6 +31,7 @@ setup: - match: { repository_ec2.settings.base_path : "${ec2_base_path}" } - match: { repository_ec2.settings.canned_acl : "private" } - match: { repository_ec2.settings.storage_class : "standard" } + - is_false: repository_ec2.settings.region - is_false: repository_ec2.settings.access_key - is_false: repository_ec2.settings.secret_key - is_false: repository_ec2.settings.session_token diff --git a/plugins/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/50_repository_ecs_credentials.yml b/plugins/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/50_repository_ecs_credentials.yml index 79d1e3d9c3bbe..8650af2d29852 100644 --- a/plugins/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/50_repository_ecs_credentials.yml +++ b/plugins/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/50_repository_ecs_credentials.yml @@ -11,6 +11,7 @@ setup: type: s3 settings: bucket: ${ecs_bucket} + region: region client: integration_test_ecs base_path: "${ecs_base_path}" canned_acl: private @@ -30,6 +31,7 @@ setup: - match: { repository_ecs.settings.base_path : "${ecs_base_path}" } - match: { repository_ecs.settings.canned_acl : "private" } - match: { repository_ecs.settings.storage_class : "standard" } + - is_false: repository_ecs.settings.region - is_false: repository_ecs.settings.access_key - is_false: repository_ecs.settings.secret_key - is_false: repository_ecs.settings.session_token diff --git a/plugins/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/60_repository_eks_credentials.yml b/plugins/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/60_repository_eks_credentials.yml index 15f2c9612a2ba..e01d3e87eb35f 100644 --- a/plugins/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/60_repository_eks_credentials.yml +++ b/plugins/repository-s3/src/yamlRestTest/resources/rest-api-spec/test/repository_s3/60_repository_eks_credentials.yml @@ -11,6 +11,7 @@ setup: type: s3 settings: bucket: ${eks_bucket} + region: region client: integration_test_eks base_path: "${eks_base_path}" canned_acl: private From 422807518717f4dde21ecd275bdacb9c8a4ec929 Mon Sep 17 00:00:00 2001 From: "Daniel (dB.) Doubrovkine" Date: Fri, 16 Jun 2023 14:51:12 -0400 Subject: [PATCH 037/109] Document testing of discovery-ec2 and repository-s3. (#7868) Signed-off-by: dblock --- plugins/discovery-ec2/README.md | 94 +++++++++++++++++++ .../discovery/ec2/Ec2NameResolver.java | 2 +- plugins/repository-s3/README.md | 25 +++++ 3 files changed, 120 insertions(+), 1 deletion(-) create mode 100644 plugins/discovery-ec2/README.md create mode 100644 plugins/repository-s3/README.md diff --git a/plugins/discovery-ec2/README.md b/plugins/discovery-ec2/README.md new file mode 100644 index 0000000000000..6daeb6305b988 --- /dev/null +++ b/plugins/discovery-ec2/README.md @@ -0,0 +1,94 @@ +# discovery-ec2 + +The discovery-ec2 plugin allows OpenSearch to find the master-eligible nodes in a cluster running on AWS EC2 by querying the AWS Instance Metadata Service ([IMDS](https://docs.aws.amazon.com/AWSEC2/latest/UserGuide/ec2-instance-metadata.html)) API for the addresses of the EC2 instances running these nodes. + +This is typically configured as follows in `opensearch.yml`. + +```yml +cluster.initial_cluster_manager_nodes: ["seed"] +discovery.seed_providers: ec2 +discovery.ec2.tag.role: my-cluster-manager +``` + +The implementation in the discovery-ec2 plugin queries for instances of a given type (e.g. private or public IP), with a tag, or running in a specific availability zone. + +## Testing + +### Unit and Integration Tests + +``` +./gradlew :plugins:discovery-ec2:check +``` + +### Manual Testing + +The following instructions manually exercise the plugin. + +Setup a 5-node (Ubuntu) cluster on EC2, with 3 of them tagged with `role: my-cluster-manager`, and a custom TCP rule to expose ports 9200 to 9300 to allow TCP traffic. Default EC2 configuration only allows SSH TCP traffic and hence only exposes port 22. + +Set the following properties. + +- `network.host`: Set this to the IP of the eth0 network adapter, can be fetched by `ipconfig` or by running `hostname -I` on Linux hosts. +- `discovery.seed_hosts`: List of all nodes in the cluster. +- `cluster.initial_cluster_manager_nodes`: List of all initial cluster manager nodes. +- `discovery.seed_providers`: Set this to `ec2` for EC2 IMDS based discovery. +- `discovery.ec2.tag.role: my-cluster-manager`: Filter out only those nodes with value `my-cluster-manager` for the `role` tag. +- `discovery.ec2.region`: Optionally set to your region, e.g. `us-east-1`, by default the plugin uses the region of the current EC2 instance. + +While sending the request to IMDS, specified discovery settings for finding master-eligible nodes are being set. You will see the following in the logs. + +``` +[2023-05-31T15:22:39,274][DEBUG][o.o.d.e.AwsEc2SeedHostsProvider] [ip-172-31-73-184] using host_type [private_ip], tags [{role=[my-cluster-manager]}], groups [[]] with any_group [true], availability_zones [[]] +``` + +The nodes getting added as eligible masters are ones with the role tag set to `my-cluster-manager`. + +``` +[2023-05-31T15:23:03,676][TRACE][o.o.d.e.AwsEc2SeedHostsProvider] [ip-172-31-73-184] finding seed nodes... +[2023-05-31T15:23:03,677][TRACE][o.o.d.e.AwsEc2SeedHostsProvider] [ip-172-31-73-184] adding i-01fa1736e8566c693, address 172.31.73.184, transport_address 172.31.73.184:9300 +[2023-05-31T15:23:03,677][TRACE][o.o.d.e.AwsEc2SeedHostsProvider] [ip-172-31-73-184] adding i-03d70a4521045cc3b, address 172.31.74.169, transport_address 172.31.74.169:9300 +[2023-05-31T15:23:03,677][TRACE][o.o.d.e.AwsEc2SeedHostsProvider] [ip-172-31-73-184] adding i-0c6ffdd10ebd3c2f1, address 172.31.74.156, transport_address 172.31.74.156:9300 +``` + +## Troubleshooting + +### Trace Level Logs + +Enable `TRACE`-level logging in `opensearch.yml` to see more output. + +```yml +logger.org.opensearch.discovery.ec2: trace +``` + +### Sample IMDS Query + +You may need to query IMDS without running OpenSearch. Use [this sample](https://github.com/dblock/aws-imds-sample) or copy the following code that makes a query to IMDS that looks for `running`, `pending` or `stopped` instances. + +```java +DefaultCredentialsProvider credentialsProviderChain = DefaultCredentialsProvider.create(); + +RetryPolicy retryPolicy = RetryPolicy.builder() + .numRetries(10) + .build(); + +Ec2Client client = Ec2Client.builder() + .httpClientBuilder(ApacheHttpClient.builder()) + .credentialsProvider(credentialsProviderChain) + .build(); + +DescribeInstancesRequest describeInstancesRequest = DescribeInstancesRequest.builder() + .filters( + Filter.builder() + .name("instance-state-name") + .values("running", "pending", "stopped") + .build() + ).build(); + +DescribeInstancesResponse describeInstancesResponse = client.describeInstances(describeInstancesRequest); +for (final Reservation reservation : describeInstancesResponse.reservations()) { + System.out.println(reservation.reservationId()); + for (final Instance instance : reservation.instances()) { + System.out.println("\t" + instance.publicDnsName()); + } +} +``` diff --git a/plugins/discovery-ec2/src/main/java/org/opensearch/discovery/ec2/Ec2NameResolver.java b/plugins/discovery-ec2/src/main/java/org/opensearch/discovery/ec2/Ec2NameResolver.java index 4a78572e2e18f..3f7a8e1e03353 100644 --- a/plugins/discovery-ec2/src/main/java/org/opensearch/discovery/ec2/Ec2NameResolver.java +++ b/plugins/discovery-ec2/src/main/java/org/opensearch/discovery/ec2/Ec2NameResolver.java @@ -117,7 +117,7 @@ public InetAddress[] resolve(Ec2HostnameType type) throws IOException { String metadataResult = urlReader.readLine(); if (metadataResult == null || metadataResult.length() == 0) { - throw new IOException("no gce metadata returned from [" + url + "] for [" + type.configName + "]"); + throw new IOException("no ec2 metadata returned from [" + url + "] for [" + type.configName + "]"); } logger.debug("obtained ec2 hostname from ec2 meta-data url {}: {}", url, metadataResult); // only one address: because we explicitly ask for only one via the Ec2HostnameType diff --git a/plugins/repository-s3/README.md b/plugins/repository-s3/README.md new file mode 100644 index 0000000000000..03007e03b633e --- /dev/null +++ b/plugins/repository-s3/README.md @@ -0,0 +1,25 @@ +# repository-s3 + +The repository-s3 plugin enables the use of S3 as a place to store snapshots. + +## Testing + +### Unit Tests + +``` +./gradlew :plugins:repository-s3:test +``` + +### Integration Tests + +Integration tests require several environment variables. + +- `amazon_s3_bucket`: Name of the S3 bucket to use. +- `amazon_s3_access_key`: The access key ID (`AWS_ACCESS_KEY_ID`) with r/w access to the S3 bucket. +- `amazon_s3_secret_key`: The secret access key (`AWS_SECRET_ACCESS_KEY`). +- `amazon_s3_base_path`: A relative path inside the S3 bucket, e.g. `opensearch`. +- `AWS_REGION`: The region in which the S3 bucket was created. While S3 buckets are global, credentials must scoped to a specific region and cross-region access is not allowed. (TODO: rename this to `amazon_s3_region` in https://github.com/opensearch-project/opensearch-build/issues/3615 and https://github.com/opensearch-project/OpenSearch/pull/7974.) + +``` +AWS_REGION=us-west-2 amazon_s3_access_key=$AWS_ACCESS_KEY_ID amazon_s3_secret_key=$AWS_SECRET_ACCESS_KEY amazon_s3_base_path=path amazon_s3_bucket=dblock-opensearch ./gradlew :plugins:repository-s3:s3ThirdPartyTest +``` From 90678c2d76ed1c31aebc85a23fe5be12f6e81417 Mon Sep 17 00:00:00 2001 From: Sorabh Date: Fri, 16 Jun 2023 18:22:35 -0700 Subject: [PATCH 038/109] With only GlobalAggregation in request causes unnecessary wrapping with MultiCollector (#8125) Signed-off-by: Sorabh Hamirwasia --- CHANGELOG.md | 1 + .../aggregation/AggregationProfilerIT.java | 76 ++++++++++++++++++- .../opensearch/search/query/QueryPhase.java | 26 ++++--- 3 files changed, 90 insertions(+), 13 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 76f0069dfa079..b0ccf9ec008b8 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -95,6 +95,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Replaces ZipInputStream with ZipFile to fix Zip Slip vulnerability ([#7230](https://github.com/opensearch-project/OpenSearch/pull/7230)) - Add missing validation/parsing of SearchBackpressureMode of SearchBackpressureSettings ([#7541](https://github.com/opensearch-project/OpenSearch/pull/7541)) - Fix mapping char_filter when mapping a hashtag ([#7591](https://github.com/opensearch-project/OpenSearch/pull/7591)) +- With only GlobalAggregation in request causes unnecessary wrapping with MultiCollector ([#8125](https://github.com/opensearch-project/OpenSearch/pull/8125)) ### Security diff --git a/server/src/internalClusterTest/java/org/opensearch/search/profile/aggregation/AggregationProfilerIT.java b/server/src/internalClusterTest/java/org/opensearch/search/profile/aggregation/AggregationProfilerIT.java index 0e9e409efae59..0f08c537d74d8 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/profile/aggregation/AggregationProfilerIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/profile/aggregation/AggregationProfilerIT.java @@ -32,14 +32,19 @@ package org.opensearch.search.profile.aggregation; +import org.hamcrest.core.IsNull; import org.opensearch.action.index.IndexRequestBuilder; import org.opensearch.action.search.SearchResponse; import org.opensearch.search.aggregations.Aggregator.SubAggCollectionMode; import org.opensearch.search.aggregations.BucketOrder; +import org.opensearch.search.aggregations.InternalAggregation; +import org.opensearch.search.aggregations.bucket.global.Global; import org.opensearch.search.aggregations.bucket.sampler.DiversifiedOrdinalsSamplerAggregator; import org.opensearch.search.aggregations.bucket.terms.GlobalOrdinalsStringTermsAggregator; +import org.opensearch.search.aggregations.metrics.Stats; import org.opensearch.search.profile.ProfileResult; import org.opensearch.search.profile.ProfileShardResult; +import org.opensearch.search.profile.query.QueryProfileShardResult; import org.opensearch.test.OpenSearchIntegTestCase; import java.util.ArrayList; @@ -48,11 +53,15 @@ import java.util.Set; import java.util.stream.Collectors; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.sameInstance; import static org.opensearch.common.xcontent.XContentFactory.jsonBuilder; import static org.opensearch.search.aggregations.AggregationBuilders.avg; import static org.opensearch.search.aggregations.AggregationBuilders.diversifiedSampler; +import static org.opensearch.search.aggregations.AggregationBuilders.global; import static org.opensearch.search.aggregations.AggregationBuilders.histogram; import static org.opensearch.search.aggregations.AggregationBuilders.max; +import static org.opensearch.search.aggregations.AggregationBuilders.stats; import static org.opensearch.search.aggregations.AggregationBuilders.terms; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertSearchResponse; @@ -95,6 +104,7 @@ public class AggregationProfilerIT extends OpenSearchIntegTestCase { private static final String NUMBER_FIELD = "number"; private static final String TAG_FIELD = "tag"; private static final String STRING_FIELD = "string_field"; + private final int numDocs = 5; @Override protected int numberOfShards() { @@ -118,7 +128,7 @@ protected void setupSuiteScopeCluster() throws Exception { randomStrings[i] = randomAlphaOfLength(10); } - for (int i = 0; i < 5; i++) { + for (int i = 0; i < numDocs; i++) { builders.add( client().prepareIndex("idx") .setSource( @@ -633,4 +643,68 @@ public void testNoProfile() { assertThat(profileResults, notNullValue()); assertThat(profileResults.size(), equalTo(0)); } + + public void testGlobalAggWithStatsSubAggregatorProfile() { + boolean profileEnabled = true; + SearchResponse response = client().prepareSearch("idx") + .addAggregation(global("global").subAggregation(stats("value_stats").field(NUMBER_FIELD))) + .setProfile(profileEnabled) + .get(); + + assertSearchResponse(response); + + Global global = response.getAggregations().get("global"); + assertThat(global, IsNull.notNullValue()); + assertThat(global.getName(), equalTo("global")); + assertThat(global.getDocCount(), equalTo((long) numDocs)); + assertThat((long) ((InternalAggregation) global).getProperty("_count"), equalTo((long) numDocs)); + assertThat(global.getAggregations().asList().isEmpty(), is(false)); + + Stats stats = global.getAggregations().get("value_stats"); + assertThat((Stats) ((InternalAggregation) global).getProperty("value_stats"), sameInstance(stats)); + assertThat(stats, IsNull.notNullValue()); + assertThat(stats.getName(), equalTo("value_stats")); + + Map profileResults = response.getProfileResults(); + assertThat(profileResults, notNullValue()); + assertThat(profileResults.size(), equalTo(getNumShards("idx").numPrimaries)); + for (ProfileShardResult profileShardResult : profileResults.values()) { + assertThat(profileShardResult, notNullValue()); + List queryProfileShardResults = profileShardResult.getQueryProfileResults(); + assertEquals(queryProfileShardResults.size(), 2); + // ensure there is no multi collector getting added with only global agg + for (QueryProfileShardResult queryProfileShardResult : queryProfileShardResults) { + assertEquals(queryProfileShardResult.getQueryResults().size(), 1); + if (queryProfileShardResult.getQueryResults().get(0).getQueryName().equals("MatchAllDocsQuery")) { + assertEquals(0, queryProfileShardResult.getQueryResults().get(0).getProfiledChildren().size()); + assertEquals("search_top_hits", queryProfileShardResult.getCollectorResult().getReason()); + assertEquals(0, queryProfileShardResult.getCollectorResult().getProfiledChildren().size()); + } else if (queryProfileShardResult.getQueryResults().get(0).getQueryName().equals("ConstantScoreQuery")) { + assertEquals(1, queryProfileShardResult.getQueryResults().get(0).getProfiledChildren().size()); + assertEquals("aggregation_global", queryProfileShardResult.getCollectorResult().getReason()); + assertEquals(0, queryProfileShardResult.getCollectorResult().getProfiledChildren().size()); + } else { + fail("unexpected profile shard result in the response"); + } + } + AggregationProfileShardResult aggProfileResults = profileShardResult.getAggregationProfileResults(); + assertThat(aggProfileResults, notNullValue()); + List aggProfileResultsList = aggProfileResults.getProfileResults(); + assertThat(aggProfileResultsList, notNullValue()); + assertEquals(1, aggProfileResultsList.size()); + ProfileResult globalAggResult = aggProfileResultsList.get(0); + assertThat(globalAggResult, notNullValue()); + assertEquals("GlobalAggregator", globalAggResult.getQueryName()); + assertEquals("global", globalAggResult.getLuceneDescription()); + assertEquals(1, globalAggResult.getProfiledChildren().size()); + assertThat(globalAggResult.getTime(), greaterThan(0L)); + Map breakdown = globalAggResult.getTimeBreakdown(); + assertThat(breakdown, notNullValue()); + assertEquals(BREAKDOWN_KEYS, breakdown.keySet()); + assertThat(breakdown.get(INITIALIZE), greaterThan(0L)); + assertThat(breakdown.get(COLLECT), greaterThan(0L)); + assertThat(breakdown.get(BUILD_AGGREGATION).longValue(), greaterThan(0L)); + assertEquals(0, breakdown.get(REDUCE).intValue()); + } + } } diff --git a/server/src/main/java/org/opensearch/search/query/QueryPhase.java b/server/src/main/java/org/opensearch/search/query/QueryPhase.java index 06be6683b5e4c..069b410280d63 100644 --- a/server/src/main/java/org/opensearch/search/query/QueryPhase.java +++ b/server/src/main/java/org/opensearch/search/query/QueryPhase.java @@ -39,6 +39,7 @@ import org.apache.lucene.search.BooleanClause; import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.Collector; +import org.apache.lucene.search.CollectorManager; import org.apache.lucene.search.FieldDoc; import org.apache.lucene.search.Query; import org.apache.lucene.search.ScoreDoc; @@ -71,6 +72,7 @@ import java.io.IOException; import java.util.LinkedList; +import java.util.List; import java.util.Map; import java.util.Objects; import java.util.concurrent.ExecutorService; @@ -234,19 +236,19 @@ static boolean executeInternal(SearchContext searchContext, QueryPhaseSearcher q // this collector can filter documents during the collection hasFilterCollector = true; } - if (searchContext.queryCollectorManagers().isEmpty() == false) { - // plug in additional collectors, like aggregations except global aggregations - collectors.add( - createMultiCollectorContext( - searchContext.queryCollectorManagers() - .entrySet() - .stream() - .filter(entry -> !(entry.getKey().equals(GlobalAggCollectorManager.class))) - .map(Map.Entry::getValue) - .collect(Collectors.toList()) - ) - ); + + // plug in additional collectors, like aggregations except global aggregations + final List> managersExceptGlobalAgg = searchContext + .queryCollectorManagers() + .entrySet() + .stream() + .filter(entry -> !(entry.getKey().equals(GlobalAggCollectorManager.class))) + .map(Map.Entry::getValue) + .collect(Collectors.toList()); + if (managersExceptGlobalAgg.isEmpty() == false) { + collectors.add(createMultiCollectorContext(managersExceptGlobalAgg)); } + if (searchContext.minimumScore() != null) { // apply the minimum score after multi collector so we filter aggs as well collectors.add(createMinScoreCollectorContext(searchContext.minimumScore())); From b5b0042dd49cf09f64490649820d5070d5fe2920 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 19 Jun 2023 11:12:16 -0400 Subject: [PATCH 039/109] Bump com.netflix.nebula:gradle-info-plugin from 12.1.3 to 12.1.4 (#8139) * Bump com.netflix.nebula:gradle-info-plugin from 12.1.3 to 12.1.4 Bumps [com.netflix.nebula:gradle-info-plugin](https://github.com/nebula-plugins/gradle-info-plugin) from 12.1.3 to 12.1.4. - [Release notes](https://github.com/nebula-plugins/gradle-info-plugin/releases) - [Changelog](https://github.com/nebula-plugins/gradle-info-plugin/blob/main/CHANGELOG.md) - [Commits](https://github.com/nebula-plugins/gradle-info-plugin/compare/v12.1.3...v12.1.4) --- updated-dependencies: - dependency-name: com.netflix.nebula:gradle-info-plugin dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] * Update changelog Signed-off-by: dependabot[bot] --------- Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: dependabot[bot] --- CHANGELOG.md | 3 ++- buildSrc/build.gradle | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index b0ccf9ec008b8..85b81d775d1fd 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -116,6 +116,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `commons-io:commons-io` from 2.12.0 to 2.13.0 (#8014, #8013, #8010) - Bump `com.diffplug.spotless` from 6.18.0 to 6.19.0 (#8007) - Bump `org.jruby.joni:joni` from 2.1.48 to 2.2.1 (#8015) +- Bump `com.netflix.nebula:gradle-info-plugin` from 12.1.3 to 12.1.4 (#8139) ### Changed - Replace jboss-annotations-api_1.2_spec with jakarta.annotation-api ([#7836](https://github.com/opensearch-project/OpenSearch/pull/7836)) @@ -136,4 +137,4 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), ### Security [Unreleased 3.0]: https://github.com/opensearch-project/OpenSearch/compare/2.x...HEAD -[Unreleased 2.x]: https://github.com/opensearch-project/OpenSearch/compare/2.8...2.x +[Unreleased 2.x]: https://github.com/opensearch-project/OpenSearch/compare/2.8...2.x \ No newline at end of file diff --git a/buildSrc/build.gradle b/buildSrc/build.gradle index 56e00ace22dd6..da0598db7666b 100644 --- a/buildSrc/build.gradle +++ b/buildSrc/build.gradle @@ -107,7 +107,7 @@ dependencies { api 'org.apache.ant:ant:1.10.13' api 'com.netflix.nebula:gradle-extra-configurations-plugin:10.0.0' api 'com.netflix.nebula:nebula-publishing-plugin:20.3.0' - api 'com.netflix.nebula:gradle-info-plugin:12.1.3' + api 'com.netflix.nebula:gradle-info-plugin:12.1.4' api 'org.apache.rat:apache-rat:0.15' api 'commons-io:commons-io:2.13.0' api "net.java.dev.jna:jna:5.13.0" From 1d2c57d761a9615eb39f50827e6f3343f78a4ffe Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 19 Jun 2023 09:28:25 -0700 Subject: [PATCH 040/109] Bump com.azure:azure-storage-common from 12.21.1 to 12.21.2 in /plugins/repository-azure (#8142) * Bump com.azure:azure-storage-common in /plugins/repository-azure Bumps [com.azure:azure-storage-common](https://github.com/Azure/azure-sdk-for-java) from 12.21.1 to 12.21.2. - [Release notes](https://github.com/Azure/azure-sdk-for-java/releases) - [Commits](https://github.com/Azure/azure-sdk-for-java/compare/azure-storage-blob_12.21.1...azure-storage-common_12.21.2) --- updated-dependencies: - dependency-name: com.azure:azure-storage-common dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] * Updating SHAs Signed-off-by: dependabot[bot] * Update changelog Signed-off-by: dependabot[bot] --------- Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: dependabot[bot] --- CHANGELOG.md | 1 + plugins/repository-azure/build.gradle | 2 +- .../licenses/azure-storage-common-12.21.1.jar.sha1 | 1 - .../licenses/azure-storage-common-12.21.2.jar.sha1 | 1 + 4 files changed, 3 insertions(+), 2 deletions(-) delete mode 100644 plugins/repository-azure/licenses/azure-storage-common-12.21.1.jar.sha1 create mode 100644 plugins/repository-azure/licenses/azure-storage-common-12.21.2.jar.sha1 diff --git a/CHANGELOG.md b/CHANGELOG.md index 85b81d775d1fd..e22f1a715a8e2 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -117,6 +117,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `com.diffplug.spotless` from 6.18.0 to 6.19.0 (#8007) - Bump `org.jruby.joni:joni` from 2.1.48 to 2.2.1 (#8015) - Bump `com.netflix.nebula:gradle-info-plugin` from 12.1.3 to 12.1.4 (#8139) +- Bump `com.azure:azure-storage-common` from 12.21.1 to 12.21.2 (#8142) ### Changed - Replace jboss-annotations-api_1.2_spec with jakarta.annotation-api ([#7836](https://github.com/opensearch-project/OpenSearch/pull/7836)) diff --git a/plugins/repository-azure/build.gradle b/plugins/repository-azure/build.gradle index da61be7b5cfc4..48a49af165542 100644 --- a/plugins/repository-azure/build.gradle +++ b/plugins/repository-azure/build.gradle @@ -46,7 +46,7 @@ opensearchplugin { dependencies { api 'com.azure:azure-core:1.39.0' api 'com.azure:azure-json:1.0.1' - api 'com.azure:azure-storage-common:12.21.1' + api 'com.azure:azure-storage-common:12.21.2' api 'com.azure:azure-core-http-netty:1.12.8' api "io.netty:netty-codec-dns:${versions.netty}" api "io.netty:netty-codec-socks:${versions.netty}" diff --git a/plugins/repository-azure/licenses/azure-storage-common-12.21.1.jar.sha1 b/plugins/repository-azure/licenses/azure-storage-common-12.21.1.jar.sha1 deleted file mode 100644 index 97baa70f28f05..0000000000000 --- a/plugins/repository-azure/licenses/azure-storage-common-12.21.1.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -3dd1e186f4e0815eb9f5c1a5bf3090ff4467a03a \ No newline at end of file diff --git a/plugins/repository-azure/licenses/azure-storage-common-12.21.2.jar.sha1 b/plugins/repository-azure/licenses/azure-storage-common-12.21.2.jar.sha1 new file mode 100644 index 0000000000000..b3c73774764df --- /dev/null +++ b/plugins/repository-azure/licenses/azure-storage-common-12.21.2.jar.sha1 @@ -0,0 +1 @@ +d2676d4fc40a501bd5d0437b8d2bfb9926022bea \ No newline at end of file From b7603b9f7bd687550e4c60fda03938b650c56730 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 19 Jun 2023 10:23:32 -0700 Subject: [PATCH 041/109] Bump com.networknt:json-schema-validator from 1.0.83 to 1.0.84 in /buildSrc (#8141) * Bump com.networknt:json-schema-validator in /buildSrc Bumps [com.networknt:json-schema-validator](https://github.com/networknt/json-schema-validator) from 1.0.83 to 1.0.84. - [Release notes](https://github.com/networknt/json-schema-validator/releases) - [Changelog](https://github.com/networknt/json-schema-validator/blob/master/CHANGELOG.md) - [Commits](https://github.com/networknt/json-schema-validator/compare/1.0.83...1.0.84) --- updated-dependencies: - dependency-name: com.networknt:json-schema-validator dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] * Update changelog Signed-off-by: dependabot[bot] --------- Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: dependabot[bot] --- CHANGELOG.md | 1 + buildSrc/build.gradle | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index e22f1a715a8e2..30f8ef55cc92b 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -118,6 +118,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `org.jruby.joni:joni` from 2.1.48 to 2.2.1 (#8015) - Bump `com.netflix.nebula:gradle-info-plugin` from 12.1.3 to 12.1.4 (#8139) - Bump `com.azure:azure-storage-common` from 12.21.1 to 12.21.2 (#8142) +- Bump `com.networknt:json-schema-validator` from 1.0.83 to 1.0.84 (#8141) ### Changed - Replace jboss-annotations-api_1.2_spec with jakarta.annotation-api ([#7836](https://github.com/opensearch-project/OpenSearch/pull/7836)) diff --git a/buildSrc/build.gradle b/buildSrc/build.gradle index da0598db7666b..314d27602b4ad 100644 --- a/buildSrc/build.gradle +++ b/buildSrc/build.gradle @@ -118,7 +118,7 @@ dependencies { api 'com.avast.gradle:gradle-docker-compose-plugin:0.16.12' api "org.yaml:snakeyaml:${props.getProperty('snakeyaml')}" api 'org.apache.maven:maven-model:3.9.2' - api 'com.networknt:json-schema-validator:1.0.83' + api 'com.networknt:json-schema-validator:1.0.84' api 'org.jruby.jcodings:jcodings:1.0.58' api 'org.jruby.joni:joni:2.1.48' api "com.fasterxml.jackson.core:jackson-databind:${props.getProperty('jackson_databind')}" From 5a968a8ec59926ce4b696c13473f88e4559f79c9 Mon Sep 17 00:00:00 2001 From: Nick Knize Date: Mon, 19 Jun 2023 15:25:39 -0500 Subject: [PATCH 042/109] [Refactor] StreamIO and OpenSearchException foundation to core library (#8035) This commit refactors the Stream IO classes from the server to core library to support cloud native and serverless extensions. The following classes are refactored: * StreamInput * StreamOutput * Writeable * NamedWriteable * NamedWriteableRegistry * Index * ShardId * BytesReference * ByteArray * BigArray * SecureString * Text * ParsingException * RestStatus The namespace is left unchanged but will be refactored in a follow up commit to avoid split package in order to support jigsaw modularity. Signed-off-by: Nicholas Walter Knize --- .../org/opensearch/BaseExceptionsHelper.java | 283 ----- .../opensearch/BaseOpenSearchException.java | 535 ---------- .../java/org/opensearch/ExceptionsHelper.java | 83 +- .../org/opensearch/OpenSearchException.java | 980 ++++++++++++++++++ .../action/ShardOperationFailedException.java | 0 .../org/opensearch/action/package-info.java | 10 + .../DefaultShardOperationFailedException.java | 6 +- .../action/support/package-info.java | 10 + .../opensearch/common/ParsingException.java | 0 .../common/bytes/AbstractBytesReference.java | 0 .../opensearch/common/bytes/BytesArray.java | 0 .../common/bytes/BytesReference.java | 0 .../common/bytes/CompositeBytesReference.java | 0 .../common/bytes/PagedBytesReference.java | 7 +- .../opensearch/common/bytes/package-info.java | 10 + .../common/compress/NotXContentException.java | 0 .../common/compress/package-info.java | 10 + .../opensearch/common/io/package-info.java | 10 + .../common/io/stream/BytesStream.java | 0 .../io/stream/InputStreamStreamInput.java | 4 +- .../common/io/stream/NamedWriteable.java | 0 .../io/stream/NamedWriteableRegistry.java | 1 + .../NotSerializableExceptionWrapper.java | 7 +- .../common/io/stream/StreamInput.java | 323 +++++- .../common/io/stream/StreamOutput.java | 159 ++- .../io/stream/Writeable.java} | 69 +- .../common/io/stream/package-info.java | 3 +- .../org/opensearch/common/package-info.java | 10 + .../common/settings/SecureString.java | 0 .../common/settings/package-info.java | 10 + .../java/org/opensearch/common/text/Text.java | 0 .../opensearch/common/text/package-info.java | 10 + .../org/opensearch/common/util/BigArray.java | 0 .../org/opensearch/common/util/ByteArray.java | 0 .../opensearch/common/util/package-info.java | 10 + .../common/xcontent/XContentParserUtils.java | 0 .../common/xcontent/package-info.java | 10 + .../core/common/io/package-info.java | 10 + .../common/io/stream/BaseStreamInput.java | 341 ------ .../common/io/stream/BaseStreamOutput.java | 150 --- .../main/java/org/opensearch/index/Index.java | 0 .../org/opensearch/index/package-info.java | 10 + .../org/opensearch/index/shard/ShardId.java | 4 +- .../opensearch/index/shard/package-info.java | 10 + .../IndexShardSnapshotException.java | 0 .../IndexShardSnapshotFailedException.java | 0 .../index/snapshots/package-info.java | 10 + .../java/org/opensearch/rest/RestStatus.java | 0 .../org/opensearch/rest/package-info.java | 10 + .../geogrid/GeoTileGridParserTests.java | 4 +- .../mustache/MultiSearchTemplateResponse.java | 4 +- .../index/rankeval/RankEvalRequestIT.java | 4 +- .../index/rankeval/RankEvalResponse.java | 3 +- .../reindex/remote/RemoteResponseParsers.java | 2 +- .../reindex/AsyncBulkByScrollActionTests.java | 3 +- .../http/DetailedErrorsEnabledIT.java | 2 +- .../opensearch/action/RejectionActionIT.java | 4 +- .../action/search/TransportSearchIT.java | 8 +- .../org/opensearch/blocks/SimpleBlocksIT.java | 6 +- .../breaker/CircuitBreakerServiceIT.java | 4 +- .../indices/state/CloseIndexIT.java | 4 +- .../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 +- .../SharedClusterSnapshotRestoreIT.java | 4 +- ...on.java => OpenSearchServerException.java} | 510 +-------- .../action/TaskOperationFailure.java | 5 +- .../NodesReloadSecureSettingsResponse.java | 4 +- .../indices/close/CloseIndexResponse.java | 4 +- .../readonly/AddIndexBlockResponse.java | 4 +- .../shards/IndicesShardStoresResponse.java | 4 +- .../action/bulk/BulkItemResponse.java | 5 +- .../action/bulk/TransportBulkAction.java | 4 +- .../action/bulk/TransportShardBulkAction.java | 4 +- .../action/get/MultiGetResponse.java | 3 +- .../ingest/SimulateDocumentBaseResult.java | 3 +- .../ingest/SimulateProcessorResult.java | 5 +- .../search/AbstractSearchAsyncAction.java | 6 +- .../action/search/MultiSearchResponse.java | 5 +- .../search/SearchPhaseExecutionException.java | 18 +- .../action/search/ShardSearchFailure.java | 9 +- .../action/support/TransportActions.java | 4 +- .../replication/ReplicationOperation.java | 7 +- .../replication/ReplicationResponse.java | 6 +- .../termvectors/MultiTermVectorsResponse.java | 4 +- .../action/shard/ShardStateAction.java | 3 +- .../cluster/routing/UnassignedInfo.java | 4 +- .../routing/allocation/FailedShard.java | 4 +- .../allocation/NodeAllocationResult.java | 4 +- .../java/org/opensearch/common/Strings.java | 4 +- .../org/opensearch/common/io/Streams.java | 31 +- .../common/io/stream/Streamables.java | 13 +- .../common/io/stream/Writeable.java | 88 -- .../common/lucene/search/Queries.java | 4 +- .../common/util/PageCacheRecycler.java | 3 +- .../opensearch/gateway/AsyncShardFetch.java | 4 +- .../index/engine/InternalEngine.java | 6 +- .../index/mapper/MappedFieldType.java | 5 +- .../index/query/MoreLikeThisQueryBuilder.java | 3 +- .../index/reindex/BulkByScrollTask.java | 3 +- .../index/reindex/ReindexRequest.java | 2 +- .../index/reindex/ScrollableHitSource.java | 4 +- .../opensearch/index/shard/StoreRecovery.java | 4 +- .../org/opensearch/index/store/Store.java | 3 +- .../recovery/PeerRecoveryTargetService.java | 6 +- .../recovery/RetryableTransportClient.java | 6 +- .../SegmentReplicationTargetService.java | 4 +- .../replication/common/ReplicationTarget.java | 4 +- .../opensearch/rest/BytesRestResponse.java | 17 +- .../java/org/opensearch/script/Script.java | 2 +- .../DirectCandidateGeneratorBuilder.java | 4 +- .../tasks/TaskCancellationService.java | 6 +- .../org/opensearch/tasks/TaskManager.java | 5 +- .../java/org/opensearch/tasks/TaskResult.java | 4 +- .../opensearch/tasks/TaskResultsService.java | 4 +- .../transport/TransportService.java | 2 + .../ExceptionSerializationTests.java | 8 +- .../org/opensearch/ExceptionsHelperTests.java | 6 +- .../opensearch/OpenSearchExceptionTests.java | 54 +- .../action/bulk/BulkItemResponseTests.java | 2 +- .../action/bulk/BulkResponseTests.java | 2 +- .../search/MultiSearchRequestTests.java | 2 +- .../client/AbstractClientHeadersTestCase.java | 4 +- .../common/io/stream/BytesStreamsTests.java | 2 +- .../rest/BytesRestResponseTests.java | 6 +- .../pipeline/DerivativeAggregatorTests.java | 4 +- .../transport/TransportActionProxyTests.java | 4 +- .../test/OpenSearchIntegTestCase.java | 4 +- .../AbstractSimpleTransportTestCase.java | 11 +- 133 files changed, 1993 insertions(+), 2178 deletions(-) delete mode 100644 libs/core/src/main/java/org/opensearch/BaseExceptionsHelper.java delete mode 100644 libs/core/src/main/java/org/opensearch/BaseOpenSearchException.java rename {server => libs/core}/src/main/java/org/opensearch/ExceptionsHelper.java (82%) create mode 100644 libs/core/src/main/java/org/opensearch/OpenSearchException.java rename {server => libs/core}/src/main/java/org/opensearch/action/ShardOperationFailedException.java (100%) create mode 100644 libs/core/src/main/java/org/opensearch/action/package-info.java rename {server => libs/core}/src/main/java/org/opensearch/action/support/DefaultShardOperationFailedException.java (97%) create mode 100644 libs/core/src/main/java/org/opensearch/action/support/package-info.java rename {server => libs/core}/src/main/java/org/opensearch/common/ParsingException.java (100%) rename {server => libs/core}/src/main/java/org/opensearch/common/bytes/AbstractBytesReference.java (100%) rename {server => libs/core}/src/main/java/org/opensearch/common/bytes/BytesArray.java (100%) rename {server => libs/core}/src/main/java/org/opensearch/common/bytes/BytesReference.java (100%) rename {server => libs/core}/src/main/java/org/opensearch/common/bytes/CompositeBytesReference.java (100%) rename {server => libs/core}/src/main/java/org/opensearch/common/bytes/PagedBytesReference.java (94%) create mode 100644 libs/core/src/main/java/org/opensearch/common/bytes/package-info.java rename {server => libs/core}/src/main/java/org/opensearch/common/compress/NotXContentException.java (100%) create mode 100644 libs/core/src/main/java/org/opensearch/common/compress/package-info.java create mode 100644 libs/core/src/main/java/org/opensearch/common/io/package-info.java rename {server => libs/core}/src/main/java/org/opensearch/common/io/stream/BytesStream.java (100%) rename {server => libs/core}/src/main/java/org/opensearch/common/io/stream/InputStreamStreamInput.java (97%) rename {server => libs/core}/src/main/java/org/opensearch/common/io/stream/NamedWriteable.java (100%) rename {server => libs/core}/src/main/java/org/opensearch/common/io/stream/NamedWriteableRegistry.java (99%) rename {server => libs/core}/src/main/java/org/opensearch/common/io/stream/NotSerializableExceptionWrapper.java (93%) rename {server => libs/core}/src/main/java/org/opensearch/common/io/stream/StreamInput.java (74%) rename {server => libs/core}/src/main/java/org/opensearch/common/io/stream/StreamOutput.java (89%) rename libs/core/src/main/java/org/opensearch/{core/common/io/stream/BaseWriteable.java => common/io/stream/Writeable.java} (64%) rename libs/core/src/main/java/org/opensearch/{core => }/common/io/stream/package-info.java (83%) create mode 100644 libs/core/src/main/java/org/opensearch/common/package-info.java rename {server => libs/core}/src/main/java/org/opensearch/common/settings/SecureString.java (100%) create mode 100644 libs/core/src/main/java/org/opensearch/common/settings/package-info.java rename {server => libs/core}/src/main/java/org/opensearch/common/text/Text.java (100%) create mode 100644 libs/core/src/main/java/org/opensearch/common/text/package-info.java rename {server => libs/core}/src/main/java/org/opensearch/common/util/BigArray.java (100%) rename {server => libs/core}/src/main/java/org/opensearch/common/util/ByteArray.java (100%) create mode 100644 libs/core/src/main/java/org/opensearch/common/util/package-info.java rename {server => libs/core}/src/main/java/org/opensearch/common/xcontent/XContentParserUtils.java (100%) create mode 100644 libs/core/src/main/java/org/opensearch/common/xcontent/package-info.java create mode 100644 libs/core/src/main/java/org/opensearch/core/common/io/package-info.java delete mode 100644 libs/core/src/main/java/org/opensearch/core/common/io/stream/BaseStreamInput.java delete mode 100644 libs/core/src/main/java/org/opensearch/core/common/io/stream/BaseStreamOutput.java rename {server => libs/core}/src/main/java/org/opensearch/index/Index.java (100%) create mode 100644 libs/core/src/main/java/org/opensearch/index/package-info.java rename {server => libs/core}/src/main/java/org/opensearch/index/shard/ShardId.java (96%) create mode 100644 libs/core/src/main/java/org/opensearch/index/shard/package-info.java rename {server => libs/core}/src/main/java/org/opensearch/index/snapshots/IndexShardSnapshotException.java (100%) rename {server => libs/core}/src/main/java/org/opensearch/index/snapshots/IndexShardSnapshotFailedException.java (100%) create mode 100644 libs/core/src/main/java/org/opensearch/index/snapshots/package-info.java rename {server => libs/core}/src/main/java/org/opensearch/rest/RestStatus.java (100%) create mode 100644 libs/core/src/main/java/org/opensearch/rest/package-info.java rename server/src/main/java/org/opensearch/{OpenSearchException.java => OpenSearchServerException.java} (67%) delete mode 100644 server/src/main/java/org/opensearch/common/io/stream/Writeable.java diff --git a/libs/core/src/main/java/org/opensearch/BaseExceptionsHelper.java b/libs/core/src/main/java/org/opensearch/BaseExceptionsHelper.java deleted file mode 100644 index 2b282d1fc48c3..0000000000000 --- a/libs/core/src/main/java/org/opensearch/BaseExceptionsHelper.java +++ /dev/null @@ -1,283 +0,0 @@ -/* - * 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 deleted file mode 100644 index 248673d11797a..0000000000000 --- a/libs/core/src/main/java/org/opensearch/BaseOpenSearchException.java +++ /dev/null @@ -1,535 +0,0 @@ -/* - * 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.CheckedFunction; -import org.opensearch.common.Nullable; -import org.opensearch.core.common.Strings; -import org.opensearch.core.common.io.stream.BaseStreamInput; -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.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; - -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; - } - - /** - * An ExceptionHandle for registering Exceptions that can be serialized over the transport wire - * - * @opensearch.internal - */ - protected static abstract class BaseOpenSearchExceptionHandle { - final Class exceptionClass; - final CheckedFunction constructor; - final int id; - final Version versionAdded; - - BaseOpenSearchExceptionHandle( - Class exceptionClass, - CheckedFunction constructor, - int id, - Version versionAdded - ) { - // We need the exceptionClass because you can't dig it out of the constructor reliably. - this.exceptionClass = exceptionClass; - this.constructor = constructor; - this.versionAdded = versionAdded; - this.id = id; - } - } - - @SuppressWarnings("unchecked") - public static BaseOpenSearchException readException(T input, int id) throws IOException { - CheckedFunction opensearchException = (CheckedFunction< - T, - ? extends BaseOpenSearchException, - IOException>) OpenSearchExceptionHandleRegistry.getSupplier(id); - if (opensearchException == null) { - throw new IllegalStateException("unknown exception for id: " + id); - } - return opensearchException.apply(input); - } - - /** - * Registry of ExceptionHandlers - * - * @opensearch.internal - */ - public static class OpenSearchExceptionHandleRegistry { - /** Registry mapping from unique Ordinal to the Exception Constructor */ - private static final Map< - Integer, - CheckedFunction> ID_TO_SUPPLIER_REGISTRY = - new ConcurrentHashMap<>(); - /** Registry mapping from Exception class to the Exception Handler */ - private static final Map< - Class, - BaseOpenSearchExceptionHandle> CLASS_TO_OPENSEARCH_EXCEPTION_HANDLE_REGISTRY = new ConcurrentHashMap<>(); - - /** returns the Exception constructor function from a given ordinal */ - public static CheckedFunction getSupplier(final int id) { - return ID_TO_SUPPLIER_REGISTRY.get(id); - } - - /** registers the Exception handler */ - public static void registerExceptionHandle(final BaseOpenSearchExceptionHandle handle) { - ID_TO_SUPPLIER_REGISTRY.put(handle.id, handle.constructor); - CLASS_TO_OPENSEARCH_EXCEPTION_HANDLE_REGISTRY.put(handle.exceptionClass, handle); - } - - /** Gets the unique ordinal id of the Exception from the given class */ - public static int getId(final Class exception) { - return CLASS_TO_OPENSEARCH_EXCEPTION_HANDLE_REGISTRY.get(exception).id; - } - - /** returns a set of ids */ - public static Set ids() { - return ID_TO_SUPPLIER_REGISTRY.keySet(); - } - - /** returns a collection of handles */ - public static Collection handles() { - return CLASS_TO_OPENSEARCH_EXCEPTION_HANDLE_REGISTRY.values(); - } - - /** checks that the exception class is registered */ - public static boolean isRegistered(final Class exception, final Version version) { - BaseOpenSearchExceptionHandle openSearchExceptionHandle = CLASS_TO_OPENSEARCH_EXCEPTION_HANDLE_REGISTRY.get(exception); - if (openSearchExceptionHandle != null) { - return version.onOrAfter(openSearchExceptionHandle.versionAdded); - } - return false; - } - - /** returns a set of registered exception classes */ - public static Set> getRegisteredKeys() { // for testing - return CLASS_TO_OPENSEARCH_EXCEPTION_HANDLE_REGISTRY.keySet(); - } - } -} diff --git a/server/src/main/java/org/opensearch/ExceptionsHelper.java b/libs/core/src/main/java/org/opensearch/ExceptionsHelper.java similarity index 82% rename from server/src/main/java/org/opensearch/ExceptionsHelper.java rename to libs/core/src/main/java/org/opensearch/ExceptionsHelper.java index 111ac8e5ef276..385dc24a0c25d 100644 --- a/server/src/main/java/org/opensearch/ExceptionsHelper.java +++ b/libs/core/src/main/java/org/opensearch/ExceptionsHelper.java @@ -33,6 +33,8 @@ 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; @@ -46,6 +48,8 @@ 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; @@ -60,12 +64,18 @@ import java.util.function.Predicate; import java.util.stream.Collectors; +import static org.opensearch.OpenSearchException.getExceptionSimpleClassName; + /** * Helper class for OpenSearch Exceptions * * @opensearch.internal */ -public final class ExceptionsHelper extends BaseExceptionsHelper { +public final class ExceptionsHelper { + private static final Logger logger = LogManager.getLogger(ExceptionsHelper.class); + + // utility class: no ctor + private ExceptionsHelper() {} public static RuntimeException convertToRuntime(Exception e) { if (e instanceof RuntimeException) { @@ -98,6 +108,77 @@ 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 getExceptionSimpleClassName(t) + "[" + 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(getExceptionSimpleClassName(t)); + 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 getExceptionSimpleClassName(t) + "[" + 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/libs/core/src/main/java/org/opensearch/OpenSearchException.java b/libs/core/src/main/java/org/opensearch/OpenSearchException.java new file mode 100644 index 0000000000000..707b53dfa58f9 --- /dev/null +++ b/libs/core/src/main/java/org/opensearch/OpenSearchException.java @@ -0,0 +1,980 @@ +/* + * 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.CheckedFunction; +import org.opensearch.common.Nullable; +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.core.ParseField; +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 org.opensearch.core.xcontent.XContentParser; +import org.opensearch.index.Index; +import org.opensearch.index.shard.ShardId; +import org.opensearch.rest.RestStatus; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +import static java.util.Collections.emptyMap; +import static org.opensearch.OpenSearchException.OpenSearchExceptionHandleRegistry.registerExceptionHandle; +import static org.opensearch.common.xcontent.XContentParserUtils.ensureExpectedToken; +import static org.opensearch.common.xcontent.XContentParserUtils.ensureFieldName; + +import static java.util.Collections.singletonMap; + +/** + * A core library base class for all opensearch exceptions. + * + * @opensearch.internal + */ +public class OpenSearchException extends RuntimeException implements Writeable, ToXContentFragment { + + protected static final Version UNKNOWN_VERSION_ADDED = Version.fromId(0); + + /** + * 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. + */ + private static final String REST_EXCEPTION_SKIP_CAUSE = "rest.exception.cause.skip"; + /** + * 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; + private static final boolean REST_EXCEPTION_SKIP_CAUSE_DEFAULT = false; + 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 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 OPENSEARCH_PREFIX_KEY = "opensearch."; + + 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"; + + protected final Map> metadata = new HashMap<>(); + protected final Map> headers = new HashMap<>(); + + static { + registerExceptionHandle( + new OpenSearchExceptionHandle( + org.opensearch.index.snapshots.IndexShardSnapshotFailedException.class, + org.opensearch.index.snapshots.IndexShardSnapshotFailedException::new, + 0, + UNKNOWN_VERSION_ADDED + ) + ); + registerExceptionHandle( + new OpenSearchExceptionHandle( + org.opensearch.common.ParsingException.class, + org.opensearch.common.ParsingException::new, + 40, + UNKNOWN_VERSION_ADDED + ) + ); + registerExceptionHandle( + new OpenSearchExceptionHandle( + org.opensearch.common.io.stream.NotSerializableExceptionWrapper.class, + org.opensearch.common.io.stream.NotSerializableExceptionWrapper::new, + 62, + UNKNOWN_VERSION_ADDED + ) + ); + } + + /** + * Construct a OpenSearchException with the specified cause exception. + */ + public OpenSearchException(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 OpenSearchException(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 OpenSearchException(String msg, Throwable cause, Object... args) { + super(LoggerMessageFormat.format(msg, args), cause); + } + + public OpenSearchException(StreamInput in) throws IOException { + this(in.readOptionalString(), in.readException()); + readStackTrace(this, in); + headers.putAll(in.readMapOfLists(StreamInput::readString, StreamInput::readString)); + metadata.putAll(in.readMapOfLists(StreamInput::readString, StreamInput::readString)); + } + + @Override + public void writeTo(final StreamOutput out) throws IOException { + out.writeOptionalString(this.getMessage()); + out.writeException(this.getCause()); + writeStackTraces(this, out, StreamOutput::writeException); + out.writeMapOfLists(headers, StreamOutput::writeString, StreamOutput::writeString); + out.writeMapOfLists(metadata, StreamOutput::writeString, StreamOutput::writeString); + } + + /** + * 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_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(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; + } + + /** + * Returns the rest status code associated with this exception. + */ + public RestStatus status() { + Throwable cause = unwrapCause(); + if (cause == this) { + return RestStatus.INTERNAL_SERVER_ERROR; + } else { + return ExceptionsHelper.status(cause); + } + } + + /** + * 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 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; + } + + @SuppressWarnings("unchecked") + public static OpenSearchException readException(T input, int id) throws IOException { + CheckedFunction opensearchException = (CheckedFunction< + T, + ? extends OpenSearchException, + IOException>) OpenSearchExceptionHandleRegistry.getSupplier(id); + if (opensearchException == null) { + throw new IllegalStateException("unknown exception for id: " + id); + } + return opensearchException.apply(input); + } + + /** + * Returns true iff the given class is a registered for an exception to be read. + */ + public static boolean isRegistered(final Class exception, Version version) { + return OpenSearchExceptionHandleRegistry.isRegistered(exception, version); + } + + static Set> getRegisteredKeys() { // for testing + return OpenSearchExceptionHandleRegistry.getRegisteredKeys(); + } + + /** + * Returns the serialization id the given exception. + */ + public static int getId(final Class exception) { + return OpenSearchExceptionHandleRegistry.getId(exception); + } + + 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, + 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 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(); + } + } + + 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(); + } + } + } + + /** + * 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 + * the type, the reason and the cause of the exception. It also recursively parses the + * tree structure of the cause, returning it as a tree structure of {@link OpenSearchException} + * instances. + */ + public static OpenSearchException fromXContent(XContentParser parser) throws IOException { + XContentParser.Token token = parser.nextToken(); + ensureExpectedToken(XContentParser.Token.FIELD_NAME, token, parser); + return innerFromXContent(parser, false); + } + + public static OpenSearchException innerFromXContent(XContentParser parser, boolean parseRootCauses) throws IOException { + XContentParser.Token token = parser.currentToken(); + ensureExpectedToken(XContentParser.Token.FIELD_NAME, token, parser); + + String type = null, reason = null, stack = null; + OpenSearchException cause = null; + Map> metadata = new HashMap<>(); + Map> headers = new HashMap<>(); + List rootCauses = new ArrayList<>(); + List suppressed = new ArrayList<>(); + + for (; token == XContentParser.Token.FIELD_NAME; token = parser.nextToken()) { + String currentFieldName = parser.currentName(); + token = parser.nextToken(); + + if (token.isValue()) { + if (TYPE.equals(currentFieldName)) { + type = parser.text(); + } else if (REASON.equals(currentFieldName)) { + reason = parser.text(); + } else if (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)) { + cause = fromXContent(parser); + } else if (HEADER.equals(currentFieldName)) { + while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { + if (token == XContentParser.Token.FIELD_NAME) { + currentFieldName = parser.currentName(); + } else { + List values = headers.getOrDefault(currentFieldName, new ArrayList<>()); + if (token == XContentParser.Token.VALUE_STRING) { + values.add(parser.text()); + } else if (token == XContentParser.Token.START_ARRAY) { + while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { + if (token == XContentParser.Token.VALUE_STRING) { + values.add(parser.text()); + } else { + parser.skipChildren(); + } + } + } else if (token == XContentParser.Token.START_OBJECT) { + parser.skipChildren(); + } + headers.put(currentFieldName, values); + } + } + } else { + // Any additional metadata object added by the metadataToXContent method is ignored + // and skipped, so that the parser does not fail on unknown fields. The parser only + // support metadata key-pairs and metadata arrays of values. + parser.skipChildren(); + } + } else if (token == XContentParser.Token.START_ARRAY) { + if (parseRootCauses && ROOT_CAUSE.equals(currentFieldName)) { + while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { + rootCauses.add(fromXContent(parser)); + } + } else if (SUPPRESSED.match(currentFieldName, parser.getDeprecationHandler())) { + while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { + suppressed.add(fromXContent(parser)); + } + } else { + // Parse the array and add each item to the corresponding list of metadata. + // Arrays of objects are not supported yet and just ignored and skipped. + List values = new ArrayList<>(); + while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { + if (token == XContentParser.Token.VALUE_STRING) { + values.add(parser.text()); + } else { + parser.skipChildren(); + } + } + if (values.size() > 0) { + if (metadata.containsKey(currentFieldName)) { + values.addAll(metadata.get(currentFieldName)); + } + metadata.put(currentFieldName, values); + } + } + } + } + + OpenSearchException e = new OpenSearchException(buildMessage(type, reason, stack), cause) { + }; + for (Map.Entry> entry : metadata.entrySet()) { + // subclasses can print out additional metadata through the metadataToXContent method. Simple key-value pairs will be + // parsed back and become part of this metadata set, while objects and arrays are not supported when parsing back. + // Those key-value pairs become part of the metadata set and inherit the "opensearch." prefix as that is currently required + // 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_PREFIX_KEY + entry.getKey(), entry.getValue()); + } + for (Map.Entry> header : headers.entrySet()) { + e.addHeader(header.getKey(), header.getValue()); + } + + // Adds root causes as suppressed exception. This way they are not lost + // after parsing and can be retrieved using getSuppressed() method. + for (OpenSearchException rootCause : rootCauses) { + e.addSuppressed(rootCause); + } + for (OpenSearchException s : suppressed) { + e.addSuppressed(s); + } + return e; + } + + /** + * Static toXContent helper method that renders {@link OpenSearchException} or {@link Throwable} instances + * as XContent, delegating the rendering to {@link OpenSearchException#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 = 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 {@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 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 ToXContent.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)} + */ + public static OpenSearchException failureFromXContent(XContentParser parser) throws IOException { + XContentParser.Token token = parser.currentToken(); + ensureFieldName(parser, token, ERROR); + + token = parser.nextToken(); + if (token.isValue()) { + return new OpenSearchException(buildMessage("exception", parser.text(), null)) { + }; + } + + ensureExpectedToken(XContentParser.Token.START_OBJECT, token, parser); + token = parser.nextToken(); + + // Root causes are parsed in the innerFromXContent() and are added as suppressed exceptions. + 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 OpenSearchException.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 = getExceptionSimpleClassName(ex); + 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); + } + + public static String getExceptionSimpleClassName(final Throwable ex) { + String simpleName = ex.getClass().getSimpleName(); + if (Strings.isEmpty(simpleName)) { + simpleName = "OpenSearchException"; + } + return 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(); + if (metadata.containsKey(INDEX_METADATA_KEY)) { + builder.append(getIndex()); + if (metadata.containsKey(SHARD_METADATA_KEY)) { + builder.append('[').append(getShardId()).append(']'); + } + builder.append(' '); + } + return builder.append(ExceptionsHelper.detailedMessage(this).trim()).toString(); + } + + /** + * Deserializes stacktrace elements as well as suppressed exceptions from the given output stream and + * adds it to the given exception. + */ + public static T readStackTrace(T throwable, StreamInput in) throws IOException { + throwable.setStackTrace(in.readArray(i -> { + final String declaringClasss = i.readString(); + final String fileName = i.readOptionalString(); + final String methodName = i.readString(); + final int lineNumber = i.readVInt(); + return new StackTraceElement(declaringClasss, methodName, fileName, lineNumber); + }, StackTraceElement[]::new)); + + int numSuppressed = in.readVInt(); + for (int i = 0; i < numSuppressed; i++) { + throwable.addSuppressed(in.readException()); + } + return throwable; + } + + /** + * Serializes the given exceptions stacktrace elements as well as it's suppressed exceptions to the given output stream. + */ + public static T writeStackTraces( + T throwable, + StreamOutput out, + Writer exceptionWriter + ) throws IOException { + out.writeArray((o, v) -> { + o.writeString(v.getClassName()); + o.writeOptionalString(v.getFileName()); + o.writeString(v.getMethodName()); + o.writeVInt(v.getLineNumber()); + }, throwable.getStackTrace()); + out.writeArray(exceptionWriter, throwable.getSuppressed()); + return throwable; + } + + 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(); + } + + /** + * Returns an array of all registered handle IDs. These are the IDs for every registered + * exception. + * + * @return an array of all registered handle IDs + */ + static int[] ids() { + return OpenSearchExceptionHandleRegistry.ids().stream().mapToInt(i -> i).toArray(); + } + + /** + * Returns an array of all registered pairs of handle IDs and exception classes. These pairs are + * provided for every registered exception. + * + * @return an array of all registered pairs of handle IDs and exception classes + */ + @SuppressWarnings("unchecked") + static Tuple>[] classes() { + final Tuple>[] ts = OpenSearchExceptionHandleRegistry.handles() + .stream() + .map(h -> Tuple.tuple(h.id, h.exceptionClass)) + .toArray(Tuple[]::new); + return ts; + } + + public Index getIndex() { + List index = getMetadata(INDEX_METADATA_KEY); + if (index != null && index.isEmpty() == false) { + List index_uuid = getMetadata(INDEX_METADATA_KEY_UUID); + return new Index(index.get(0), index_uuid.get(0)); + } + + return null; + } + + public void setIndex(Index index) { + if (index != null) { + addMetadata(INDEX_METADATA_KEY, index.getName()); + addMetadata(INDEX_METADATA_KEY_UUID, index.getUUID()); + } + } + + public void setIndex(String index) { + if (index != null) { + setIndex(new Index(index, Strings.UNKNOWN_UUID_VALUE)); + } + } + + public ShardId getShardId() { + List shard = getMetadata(SHARD_METADATA_KEY); + if (shard != null && shard.isEmpty() == false) { + return new ShardId(getIndex(), Integer.parseInt(shard.get(0))); + } + return null; + } + + public void setShard(ShardId shardId) { + if (shardId != null) { + setIndex(shardId.getIndex()); + addMetadata(SHARD_METADATA_KEY, Integer.toString(shardId.id())); + } + } + + /** + * This is the list of Exceptions OpenSearch can throw over the wire or save into a corruption marker. Each value in the enum is a + * single exception tying the Class to an id for use of the encode side and the id back to a constructor for use on the decode side. As + * such its ok if the exceptions to change names so long as their constructor can still read the exception. Each exception is listed + * in id order. If you want to remove an exception leave a tombstone comment and mark the id as null in + * ExceptionSerializationTests.testIds.ids. + * + * @opensearch.internal + */ + protected static class OpenSearchExceptionHandle { + final Class exceptionClass; + final CheckedFunction constructor; + final int id; + final Version versionAdded; + + OpenSearchExceptionHandle( + Class exceptionClass, + CheckedFunction constructor, + int id, + Version versionAdded + ) { + // We need the exceptionClass because you can't dig it out of the constructor reliably. + this.exceptionClass = exceptionClass; + this.constructor = constructor; + this.versionAdded = versionAdded; + this.id = id; + } + } + + /** + * Registry of ExceptionHandlers + * + * @opensearch.internal + */ + public static class OpenSearchExceptionHandleRegistry { + /** Registry mapping from unique Ordinal to the Exception Constructor */ + private static final Map< + Integer, + CheckedFunction> ID_TO_SUPPLIER_REGISTRY = new ConcurrentHashMap<>(); + /** Registry mapping from Exception class to the Exception Handler */ + private static final Map< + Class, + OpenSearchExceptionHandle> CLASS_TO_OPENSEARCH_EXCEPTION_HANDLE_REGISTRY = new ConcurrentHashMap<>(); + + /** returns the Exception constructor function from a given ordinal */ + public static CheckedFunction getSupplier(final int id) { + return ID_TO_SUPPLIER_REGISTRY.get(id); + } + + /** registers the Exception handler */ + public static void registerExceptionHandle(final OpenSearchExceptionHandle handle) { + ID_TO_SUPPLIER_REGISTRY.put(handle.id, handle.constructor); + CLASS_TO_OPENSEARCH_EXCEPTION_HANDLE_REGISTRY.put(handle.exceptionClass, handle); + } + + /** Gets the unique ordinal id of the Exception from the given class */ + public static int getId(final Class exception) { + return CLASS_TO_OPENSEARCH_EXCEPTION_HANDLE_REGISTRY.get(exception).id; + } + + /** returns a set of ids */ + public static Set ids() { + return ID_TO_SUPPLIER_REGISTRY.keySet(); + } + + /** returns a collection of handles */ + public static Collection handles() { + return CLASS_TO_OPENSEARCH_EXCEPTION_HANDLE_REGISTRY.values(); + } + + /** checks that the exception class is registered */ + public static boolean isRegistered(final Class exception, final Version version) { + OpenSearchExceptionHandle openSearchExceptionHandle = CLASS_TO_OPENSEARCH_EXCEPTION_HANDLE_REGISTRY.get(exception); + if (openSearchExceptionHandle != null) { + return version.onOrAfter(openSearchExceptionHandle.versionAdded); + } + return false; + } + + /** returns a set of registered exception classes */ + public static Set> getRegisteredKeys() { // for testing + return CLASS_TO_OPENSEARCH_EXCEPTION_HANDLE_REGISTRY.keySet(); + } + } +} diff --git a/server/src/main/java/org/opensearch/action/ShardOperationFailedException.java b/libs/core/src/main/java/org/opensearch/action/ShardOperationFailedException.java similarity index 100% rename from server/src/main/java/org/opensearch/action/ShardOperationFailedException.java rename to libs/core/src/main/java/org/opensearch/action/ShardOperationFailedException.java diff --git a/libs/core/src/main/java/org/opensearch/action/package-info.java b/libs/core/src/main/java/org/opensearch/action/package-info.java new file mode 100644 index 0000000000000..a7b89f63a6217 --- /dev/null +++ b/libs/core/src/main/java/org/opensearch/action/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. + */ + +/** Core action module */ +package org.opensearch.action; diff --git a/server/src/main/java/org/opensearch/action/support/DefaultShardOperationFailedException.java b/libs/core/src/main/java/org/opensearch/action/support/DefaultShardOperationFailedException.java similarity index 97% rename from server/src/main/java/org/opensearch/action/support/DefaultShardOperationFailedException.java rename to libs/core/src/main/java/org/opensearch/action/support/DefaultShardOperationFailedException.java index 35334cbbad801..5a7e48a5efef7 100644 --- a/server/src/main/java/org/opensearch/action/support/DefaultShardOperationFailedException.java +++ b/libs/core/src/main/java/org/opensearch/action/support/DefaultShardOperationFailedException.java @@ -32,9 +32,8 @@ package org.opensearch.action.support; -import org.opensearch.BaseExceptionsHelper; -import org.opensearch.OpenSearchException; import org.opensearch.ExceptionsHelper; +import org.opensearch.OpenSearchException; import org.opensearch.action.ShardOperationFailedException; import org.opensearch.core.ParseField; import org.opensearch.common.io.stream.StreamInput; @@ -48,6 +47,7 @@ import java.io.IOException; import static org.opensearch.ExceptionsHelper.detailedMessage; +import static org.opensearch.OpenSearchException.generateThrowableXContent; import static org.opensearch.core.xcontent.ConstructingObjectParser.constructorArg; /** @@ -136,7 +136,7 @@ protected XContentBuilder innerToXContent(XContentBuilder builder, Params params builder.field("status", status.name()); if (reason != null) { builder.startObject("reason"); - BaseExceptionsHelper.generateThrowableXContent(builder, params, cause); + generateThrowableXContent(builder, params, cause); builder.endObject(); } return builder; diff --git a/libs/core/src/main/java/org/opensearch/action/support/package-info.java b/libs/core/src/main/java/org/opensearch/action/support/package-info.java new file mode 100644 index 0000000000000..6665904e2b318 --- /dev/null +++ b/libs/core/src/main/java/org/opensearch/action/support/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. + */ + +/** Support classes for core action module */ +package org.opensearch.action.support; diff --git a/server/src/main/java/org/opensearch/common/ParsingException.java b/libs/core/src/main/java/org/opensearch/common/ParsingException.java similarity index 100% rename from server/src/main/java/org/opensearch/common/ParsingException.java rename to libs/core/src/main/java/org/opensearch/common/ParsingException.java diff --git a/server/src/main/java/org/opensearch/common/bytes/AbstractBytesReference.java b/libs/core/src/main/java/org/opensearch/common/bytes/AbstractBytesReference.java similarity index 100% rename from server/src/main/java/org/opensearch/common/bytes/AbstractBytesReference.java rename to libs/core/src/main/java/org/opensearch/common/bytes/AbstractBytesReference.java diff --git a/server/src/main/java/org/opensearch/common/bytes/BytesArray.java b/libs/core/src/main/java/org/opensearch/common/bytes/BytesArray.java similarity index 100% rename from server/src/main/java/org/opensearch/common/bytes/BytesArray.java rename to libs/core/src/main/java/org/opensearch/common/bytes/BytesArray.java diff --git a/server/src/main/java/org/opensearch/common/bytes/BytesReference.java b/libs/core/src/main/java/org/opensearch/common/bytes/BytesReference.java similarity index 100% rename from server/src/main/java/org/opensearch/common/bytes/BytesReference.java rename to libs/core/src/main/java/org/opensearch/common/bytes/BytesReference.java diff --git a/server/src/main/java/org/opensearch/common/bytes/CompositeBytesReference.java b/libs/core/src/main/java/org/opensearch/common/bytes/CompositeBytesReference.java similarity index 100% rename from server/src/main/java/org/opensearch/common/bytes/CompositeBytesReference.java rename to libs/core/src/main/java/org/opensearch/common/bytes/CompositeBytesReference.java diff --git a/server/src/main/java/org/opensearch/common/bytes/PagedBytesReference.java b/libs/core/src/main/java/org/opensearch/common/bytes/PagedBytesReference.java similarity index 94% rename from server/src/main/java/org/opensearch/common/bytes/PagedBytesReference.java rename to libs/core/src/main/java/org/opensearch/common/bytes/PagedBytesReference.java index b7439853f11ba..9d432bc6fdd8c 100644 --- a/server/src/main/java/org/opensearch/common/bytes/PagedBytesReference.java +++ b/libs/core/src/main/java/org/opensearch/common/bytes/PagedBytesReference.java @@ -35,7 +35,6 @@ import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRefIterator; import org.opensearch.common.util.ByteArray; -import org.opensearch.common.util.PageCacheRecycler; import java.io.IOException; @@ -47,7 +46,7 @@ */ public class PagedBytesReference extends AbstractBytesReference { - private static final int PAGE_SIZE = PageCacheRecycler.BYTE_PAGE_SIZE; + public static final int PAGE_SIZE_IN_BYTES = 1 << 14; private final ByteArray byteArray; private final int offset; @@ -95,7 +94,7 @@ public final BytesRefIterator iterator() { // we calculate the initial fragment size here to ensure that if this reference is a slice we are still page aligned // across the entire iteration. The first page is smaller if our offset != 0 then we start in the middle of the page // otherwise we iterate full pages until we reach the last chunk which also might end within a page. - final int initialFragmentSize = offset != 0 ? PAGE_SIZE - (offset % PAGE_SIZE) : PAGE_SIZE; + final int initialFragmentSize = offset != 0 ? PAGE_SIZE_IN_BYTES - (offset % PAGE_SIZE_IN_BYTES) : PAGE_SIZE_IN_BYTES; return new BytesRefIterator() { int position = 0; int nextFragmentSize = Math.min(length, initialFragmentSize); @@ -109,7 +108,7 @@ public BytesRef next() throws IOException { assert materialized == false : "iteration should be page aligned but array got materialized"; position += nextFragmentSize; final int remaining = length - position; - nextFragmentSize = Math.min(remaining, PAGE_SIZE); + nextFragmentSize = Math.min(remaining, PAGE_SIZE_IN_BYTES); return slice; } else { assert nextFragmentSize == 0 : "fragmentSize expected [0] but was: [" + nextFragmentSize + "]"; diff --git a/libs/core/src/main/java/org/opensearch/common/bytes/package-info.java b/libs/core/src/main/java/org/opensearch/common/bytes/package-info.java new file mode 100644 index 0000000000000..0644188656496 --- /dev/null +++ b/libs/core/src/main/java/org/opensearch/common/bytes/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. + */ + +/** Classes for core Bytes module */ +package org.opensearch.common.bytes; diff --git a/server/src/main/java/org/opensearch/common/compress/NotXContentException.java b/libs/core/src/main/java/org/opensearch/common/compress/NotXContentException.java similarity index 100% rename from server/src/main/java/org/opensearch/common/compress/NotXContentException.java rename to libs/core/src/main/java/org/opensearch/common/compress/NotXContentException.java diff --git a/libs/core/src/main/java/org/opensearch/common/compress/package-info.java b/libs/core/src/main/java/org/opensearch/common/compress/package-info.java new file mode 100644 index 0000000000000..b3b2f2a2ec7ed --- /dev/null +++ b/libs/core/src/main/java/org/opensearch/common/compress/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. + */ + +/** Classes for core compress module */ +package org.opensearch.common.compress; diff --git a/libs/core/src/main/java/org/opensearch/common/io/package-info.java b/libs/core/src/main/java/org/opensearch/common/io/package-info.java new file mode 100644 index 0000000000000..0d57efe860d09 --- /dev/null +++ b/libs/core/src/main/java/org/opensearch/common/io/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. + */ + +/** Classes for core io module */ +package org.opensearch.common.io; diff --git a/server/src/main/java/org/opensearch/common/io/stream/BytesStream.java b/libs/core/src/main/java/org/opensearch/common/io/stream/BytesStream.java similarity index 100% rename from server/src/main/java/org/opensearch/common/io/stream/BytesStream.java rename to libs/core/src/main/java/org/opensearch/common/io/stream/BytesStream.java diff --git a/server/src/main/java/org/opensearch/common/io/stream/InputStreamStreamInput.java b/libs/core/src/main/java/org/opensearch/common/io/stream/InputStreamStreamInput.java similarity index 97% rename from server/src/main/java/org/opensearch/common/io/stream/InputStreamStreamInput.java rename to libs/core/src/main/java/org/opensearch/common/io/stream/InputStreamStreamInput.java index 991fd0fc09a9d..06e10c67cf972 100644 --- a/server/src/main/java/org/opensearch/common/io/stream/InputStreamStreamInput.java +++ b/libs/core/src/main/java/org/opensearch/common/io/stream/InputStreamStreamInput.java @@ -32,8 +32,6 @@ package org.opensearch.common.io.stream; -import org.opensearch.common.io.Streams; - import java.io.EOFException; import java.io.IOException; import java.io.InputStream; @@ -80,7 +78,7 @@ public byte readByte() throws IOException { @Override public void readBytes(byte[] b, int offset, int len) throws IOException { if (len < 0) throw new IndexOutOfBoundsException(); - final int read = Streams.readFully(is, b, offset, len); + final int read = is.readNBytes(b, offset, len); if (read != len) { throw new EOFException(); } diff --git a/server/src/main/java/org/opensearch/common/io/stream/NamedWriteable.java b/libs/core/src/main/java/org/opensearch/common/io/stream/NamedWriteable.java similarity index 100% rename from server/src/main/java/org/opensearch/common/io/stream/NamedWriteable.java rename to libs/core/src/main/java/org/opensearch/common/io/stream/NamedWriteable.java diff --git a/server/src/main/java/org/opensearch/common/io/stream/NamedWriteableRegistry.java b/libs/core/src/main/java/org/opensearch/common/io/stream/NamedWriteableRegistry.java similarity index 99% rename from server/src/main/java/org/opensearch/common/io/stream/NamedWriteableRegistry.java rename to libs/core/src/main/java/org/opensearch/common/io/stream/NamedWriteableRegistry.java index 0240d0a0a8bb3..985b2ad113f22 100644 --- a/server/src/main/java/org/opensearch/common/io/stream/NamedWriteableRegistry.java +++ b/libs/core/src/main/java/org/opensearch/common/io/stream/NamedWriteableRegistry.java @@ -92,6 +92,7 @@ public NamedWriteableRegistry(List entries) { Map, Map>> registry = new HashMap<>(); Map> readers = null; + @SuppressWarnings("rawtypes") Class currentCategory = null; for (Entry entry : entries) { if (currentCategory != entry.categoryClass) { diff --git a/server/src/main/java/org/opensearch/common/io/stream/NotSerializableExceptionWrapper.java b/libs/core/src/main/java/org/opensearch/common/io/stream/NotSerializableExceptionWrapper.java similarity index 93% rename from server/src/main/java/org/opensearch/common/io/stream/NotSerializableExceptionWrapper.java rename to libs/core/src/main/java/org/opensearch/common/io/stream/NotSerializableExceptionWrapper.java index 636df65b8c319..967957afc8c20 100644 --- a/server/src/main/java/org/opensearch/common/io/stream/NotSerializableExceptionWrapper.java +++ b/libs/core/src/main/java/org/opensearch/common/io/stream/NotSerializableExceptionWrapper.java @@ -32,9 +32,8 @@ package org.opensearch.common.io.stream; -import org.opensearch.BaseExceptionsHelper; -import org.opensearch.OpenSearchException; import org.opensearch.ExceptionsHelper; +import org.opensearch.OpenSearchException; import org.opensearch.rest.RestStatus; import java.io.IOException; @@ -54,8 +53,8 @@ public final class NotSerializableExceptionWrapper extends OpenSearchException { private final RestStatus status; public NotSerializableExceptionWrapper(Throwable other) { - super(BaseExceptionsHelper.getExceptionName(other) + ": " + other.getMessage(), other.getCause()); - this.name = BaseExceptionsHelper.getExceptionName(other); + super(OpenSearchException.getExceptionName(other) + ": " + other.getMessage(), other.getCause()); + this.name = OpenSearchException.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/libs/core/src/main/java/org/opensearch/common/io/stream/StreamInput.java similarity index 74% rename from server/src/main/java/org/opensearch/common/io/stream/StreamInput.java rename to libs/core/src/main/java/org/opensearch/common/io/stream/StreamInput.java index 7e0ef07c7e602..c6915ae1f45b0 100644 --- a/server/src/main/java/org/opensearch/common/io/stream/StreamInput.java +++ b/libs/core/src/main/java/org/opensearch/common/io/stream/StreamInput.java @@ -37,8 +37,10 @@ import org.apache.lucene.index.IndexFormatTooOldException; import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.LockObtainFailedException; +import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.BitUtil; import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.CharsRef; import org.opensearch.Build; import org.opensearch.OpenSearchException; import org.opensearch.Version; @@ -50,8 +52,6 @@ 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; import java.io.ByteArrayInputStream; @@ -59,6 +59,7 @@ import java.io.FileNotFoundException; import java.io.FilterInputStream; import java.io.IOException; +import java.io.InputStream; import java.math.BigInteger; import java.nio.file.AccessDeniedException; import java.nio.file.AtomicMoveNotSupportedException; @@ -82,6 +83,7 @@ import java.util.LinkedHashMap; import java.util.LinkedHashSet; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; @@ -102,7 +104,37 @@ * * @opensearch.internal */ -public abstract class StreamInput extends BaseStreamInput { +public abstract class StreamInput extends InputStream { + + private Version version = Version.CURRENT; + + /** + * The version of the node on the other side of this stream. + */ + public Version getVersion() { + return this.version; + } + + /** + * Set the version of the node on the other side of this stream. + */ + public void setVersion(Version version) { + this.version = version; + } + + /** + * Reads and returns a single byte. + */ + public abstract byte readByte() throws IOException; + + /** + * Reads a specified number of bytes into an array at the specified offset. + * + * @param b the array to read bytes into + * @param offset the offset in the array to start storing bytes + * @param len the number of bytes to read + */ + public abstract void readBytes(byte[] b, int offset, int len) throws IOException; /** * Reads a bytes reference from this stream, might hold an actual reference to the underlying @@ -145,6 +177,10 @@ public BytesRef readBytesRef() throws IOException { return readBytesRef(length); } + public void readFully(byte[] b) throws IOException { + readBytes(b, 0, b.length); + } + public BytesRef readBytesRef(int length) throws IOException { if (length == 0) { return new BytesRef(); @@ -158,6 +194,47 @@ public short readShort() throws IOException { return (short) (((readByte() & 0xFF) << 8) | (readByte() & 0xFF)); } + /** + * Reads four bytes and returns an int. + */ + public int readInt() throws IOException { + return ((readByte() & 0xFF) << 24) | ((readByte() & 0xFF) << 16) | ((readByte() & 0xFF) << 8) | (readByte() & 0xFF); + } + + /** + * Reads an int stored in variable-length format. Reads between one and + * five bytes. Smaller values take fewer bytes. Negative numbers + * will always use all 5 bytes and are therefore better serialized + * using {@link #readInt} + */ + public int readVInt() throws IOException { + byte b = readByte(); + int i = b & 0x7F; + if ((b & 0x80) == 0) { + return i; + } + b = readByte(); + i |= (b & 0x7F) << 7; + if ((b & 0x80) == 0) { + return i; + } + b = readByte(); + i |= (b & 0x7F) << 14; + if ((b & 0x80) == 0) { + return i; + } + b = readByte(); + i |= (b & 0x7F) << 21; + if ((b & 0x80) == 0) { + return i; + } + b = readByte(); + if ((b & 0x80) != 0) { + throw new IOException("Invalid vInt ((" + Integer.toHexString(b) + " & 0x7f) << 28) | " + Integer.toHexString(i)); + } + return i | ((b & 0x7F) << 28); + } + /** * Reads an optional {@link Integer}. */ @@ -282,6 +359,14 @@ public Text readText() throws IOException { return new Text(readBytesReference(length)); } + @Nullable + public String readOptionalString() throws IOException { + if (readBoolean()) { + return readString(); + } + return null; + } + @Nullable public SecureString readOptionalSecureString() throws IOException { SecureString value = null; @@ -305,6 +390,157 @@ public Float readOptionalFloat() throws IOException { return null; } + @Nullable + public Integer readOptionalVInt() throws IOException { + if (readBoolean()) { + return readVInt(); + } + return null; + } + + // Maximum char-count to de-serialize via the thread-local CharsRef buffer + private static final int SMALL_STRING_LIMIT = 1024; + + // Reusable bytes for deserializing strings + private static final ThreadLocal stringReadBuffer = ThreadLocal.withInitial(() -> new byte[1024]); + + // Thread-local buffer for smaller strings + private static final ThreadLocal smallSpare = ThreadLocal.withInitial(() -> new CharsRef(SMALL_STRING_LIMIT)); + + // Larger buffer used for long strings that can't fit into the thread-local buffer + // We don't use a CharsRefBuilder since we exactly know the size of the character array up front + // this prevents calling grow for every character since we don't need this + private CharsRef largeSpare; + + public String readString() throws IOException { + final int charCount = readArraySize(); + final CharsRef charsRef; + if (charCount > SMALL_STRING_LIMIT) { + if (largeSpare == null) { + largeSpare = new CharsRef(ArrayUtil.oversize(charCount, Character.BYTES)); + } else if (largeSpare.chars.length < charCount) { + // we don't use ArrayUtils.grow since there is no need to copy the array + largeSpare.chars = new char[ArrayUtil.oversize(charCount, Character.BYTES)]; + } + charsRef = largeSpare; + } else { + charsRef = smallSpare.get(); + } + charsRef.length = charCount; + int charsOffset = 0; + int offsetByteArray = 0; + int sizeByteArray = 0; + int missingFromPartial = 0; + final byte[] byteBuffer = stringReadBuffer.get(); + final char[] charBuffer = charsRef.chars; + for (; charsOffset < charCount;) { + final int charsLeft = charCount - charsOffset; + int bufferFree = byteBuffer.length - sizeByteArray; + // Determine the minimum amount of bytes that are left in the string + final int minRemainingBytes; + if (missingFromPartial > 0) { + // One byte for each remaining char except for the already partially read char + minRemainingBytes = missingFromPartial + charsLeft - 1; + missingFromPartial = 0; + } else { + // Each char has at least a single byte + minRemainingBytes = charsLeft; + } + final int toRead; + if (bufferFree < minRemainingBytes) { + // We don't have enough space left in the byte array to read as much as we'd like to so we free up as many bytes in the + // buffer by moving unused bytes that didn't make up a full char in the last iteration to the beginning of the buffer, + // if there are any + if (offsetByteArray > 0) { + sizeByteArray = sizeByteArray - offsetByteArray; + switch (sizeByteArray) { // We only have 0, 1 or 2 => no need to bother with a native call to System#arrayCopy + case 1: + byteBuffer[0] = byteBuffer[offsetByteArray]; + break; + case 2: + byteBuffer[0] = byteBuffer[offsetByteArray]; + byteBuffer[1] = byteBuffer[offsetByteArray + 1]; + break; + } + assert sizeByteArray <= 2 : "We never copy more than 2 bytes here since a char is 3 bytes max"; + toRead = Math.min(bufferFree + offsetByteArray, minRemainingBytes); + offsetByteArray = 0; + } else { + toRead = bufferFree; + } + } else { + toRead = minRemainingBytes; + } + readBytes(byteBuffer, sizeByteArray, toRead); + sizeByteArray += toRead; + // As long as we at least have three bytes buffered we don't need to do any bounds checking when getting the next char since we + // read 3 bytes per char/iteration at most + for (; offsetByteArray < sizeByteArray - 2; offsetByteArray++) { + final int c = byteBuffer[offsetByteArray] & 0xff; + switch (c >> 4) { + case 0: + case 1: + case 2: + case 3: + case 4: + case 5: + case 6: + case 7: + charBuffer[charsOffset++] = (char) c; + break; + case 12: + case 13: + charBuffer[charsOffset++] = (char) ((c & 0x1F) << 6 | byteBuffer[++offsetByteArray] & 0x3F); + break; + case 14: + charBuffer[charsOffset++] = (char) ((c & 0x0F) << 12 | (byteBuffer[++offsetByteArray] & 0x3F) << 6 + | (byteBuffer[++offsetByteArray] & 0x3F)); + break; + default: + throwOnBrokenChar(c); + } + } + // try to extract chars from remaining bytes with bounds checks for multi-byte chars + final int bufferedBytesRemaining = sizeByteArray - offsetByteArray; + for (int i = 0; i < bufferedBytesRemaining; i++) { + final int c = byteBuffer[offsetByteArray] & 0xff; + switch (c >> 4) { + case 0: + case 1: + case 2: + case 3: + case 4: + case 5: + case 6: + case 7: + charBuffer[charsOffset++] = (char) c; + offsetByteArray++; + break; + case 12: + case 13: + missingFromPartial = 2 - (bufferedBytesRemaining - i); + if (missingFromPartial == 0) { + offsetByteArray++; + charBuffer[charsOffset++] = (char) ((c & 0x1F) << 6 | byteBuffer[offsetByteArray++] & 0x3F); + } + ++i; + break; + case 14: + missingFromPartial = 3 - (bufferedBytesRemaining - i); + ++i; + break; + default: + throwOnBrokenChar(c); + } + } + } + return charsRef.toString(); + } + + private static void throwOnBrokenChar(int c) throws IOException { + throw new IOException("Invalid string; unexpected character: " + c + " hex: " + Integer.toHexString(c)); + } + public SecureString readSecureString() throws IOException { BytesReference bytesRef = readBytesReference(); byte[] bytes = BytesReference.toBytes(bytesRef); @@ -331,6 +567,43 @@ public final Double readOptionalDouble() throws IOException { return null; } + /** + * Reads a boolean. + */ + public final boolean readBoolean() throws IOException { + return readBoolean(readByte()); + } + + private boolean readBoolean(final byte value) { + if (value == 0) { + return false; + } else if (value == 1) { + return true; + } else { + final String message = String.format(Locale.ROOT, "unexpected byte [0x%02x]", value); + throw new IllegalStateException(message); + } + } + + @Nullable + public final Boolean readOptionalBoolean() throws IOException { + final byte value = readByte(); + if (value == 2) { + return null; + } else { + return readBoolean(value); + } + } + + /** + * Closes the stream to further operations. + */ + @Override + public abstract void close() throws IOException; + + @Override + public abstract int available() throws IOException; + public String[] readStringArray() throws IOException { int size = readArraySize(); if (size == 0) { @@ -409,7 +682,7 @@ public Map readMap() throws IOException { @Nullable public Object readGenericValue() throws IOException { byte type = readByte(); - BaseWriteable.Reader r = BaseWriteable.WriteableRegistry.getReader(type); + Writeable.Reader r = Writeable.WriteableRegistry.getReader(type); if (r != null) { return r.read(this); } @@ -489,7 +762,7 @@ public final Instant readOptionalInstant() throws IOException { return present ? readInstant() : null; } - @SuppressWarnings("unchecked") + @SuppressWarnings({ "rawtypes", "unchecked" }) private List readArrayList() throws IOException { int size = readArraySize(); if (size == 0) { @@ -521,6 +794,7 @@ private Object[] readArray() throws IOException { return list8; } + @SuppressWarnings({ "rawtypes", "unchecked" }) private Map readLinkedHashMap() throws IOException { int size9 = readArraySize(); if (size9 == 0) { @@ -533,6 +807,7 @@ private Map readLinkedHashMap() throws IOException { return map9; } + @SuppressWarnings({ "rawtypes", "unchecked" }) private Map readHashMap() throws IOException { int size10 = readArraySize(); if (size10 == 0) { @@ -873,7 +1148,7 @@ public C readOptionalNamedWriteable(Class category * @return the list of objects * @throws IOException if an I/O exception occurs reading the list */ - public List readList(final BaseWriteable.Reader reader) throws IOException { + public List readList(final Writeable.Reader reader) throws IOException { return readCollection(reader, ArrayList::new, Collections.emptyList()); } @@ -912,6 +1187,22 @@ public Set readSet(Writeable.Reader reader) throws IOException { return readCollection(reader, HashSet::new, Collections.emptySet()); } + /** + * Reads a collection of objects + */ + private > C readCollection(Writeable.Reader reader, IntFunction constructor, C empty) + throws IOException { + int count = readArraySize(); + if (count == 0) { + return empty; + } + C builder = constructor.apply(count); + for (int i = 0; i < count; i++) { + builder.add(reader.read(this)); + } + return builder; + } + /** * Reads a list of {@link NamedWriteable}s. If the returned list contains any entries it will be mutable. * If it is empty it might be immutable. @@ -967,11 +1258,29 @@ public static StreamInput wrap(byte[] bytes, int offset, int length) { return new InputStreamStreamInput(new ByteArrayInputStream(bytes, offset, length), length); } + /** + * Reads a vint via {@link #readVInt()} and applies basic checks to ensure the read array size is sane. + * This method uses {@link #ensureCanReadBytes(int)} to ensure this stream has enough bytes to read for the read array size. + */ + private int readArraySize() throws IOException { + final int arraySize = readVInt(); + if (arraySize > ArrayUtil.MAX_ARRAY_LENGTH) { + throw new IllegalStateException("array length must be <= to " + ArrayUtil.MAX_ARRAY_LENGTH + " but was: " + arraySize); + } + if (arraySize < 0) { + throw new NegativeArraySizeException("array size must be positive but was: " + arraySize); + } + // lets do a sanity check that if we are reading an array size that is bigger that the remaining bytes we can safely + // throw an exception instead of allocating the array based on the size. A simple corrutpted byte can make a node go OOM + // if the size is large and for perf reasons we allocate arrays ahead of time + ensureCanReadBytes(arraySize); + return arraySize; + } + /** * This method throws an {@link EOFException} if the given number of bytes can not be read from the this stream. This method might * be a no-op depending on the underlying implementation if the information of the remaining bytes is not present. */ - @Override protected abstract void ensureCanReadBytes(int length) throws EOFException; private static final TimeUnit[] TIME_UNITS = TimeUnit.values(); diff --git a/server/src/main/java/org/opensearch/common/io/stream/StreamOutput.java b/libs/core/src/main/java/org/opensearch/common/io/stream/StreamOutput.java similarity index 89% rename from server/src/main/java/org/opensearch/common/io/stream/StreamOutput.java rename to libs/core/src/main/java/org/opensearch/common/io/stream/StreamOutput.java index 8911e90aa7511..888c7530c4146 100644 --- a/server/src/main/java/org/opensearch/common/io/stream/StreamOutput.java +++ b/libs/core/src/main/java/org/opensearch/common/io/stream/StreamOutput.java @@ -43,23 +43,21 @@ import org.opensearch.Build; import org.opensearch.OpenSearchException; import org.opensearch.Version; -import org.opensearch.cluster.ClusterState; -import org.opensearch.cluster.metadata.Metadata; import org.opensearch.common.CharArrays; import org.opensearch.common.Nullable; import org.opensearch.common.bytes.BytesArray; import org.opensearch.common.bytes.BytesReference; +import org.opensearch.common.io.stream.Writeable.WriteableRegistry; import org.opensearch.common.io.stream.Writeable.Writer; import org.opensearch.common.settings.SecureString; import org.opensearch.common.text.Text; import org.opensearch.common.unit.TimeValue; -import org.opensearch.core.common.io.stream.BaseStreamOutput; -import org.opensearch.core.common.io.stream.BaseWriteable; import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; import java.io.EOFException; import java.io.FileNotFoundException; import java.io.IOException; +import java.io.OutputStream; import java.math.BigInteger; import java.nio.file.AccessDeniedException; import java.nio.file.AtomicMoveNotSupportedException; @@ -100,15 +98,30 @@ * * @opensearch.internal */ -public abstract class StreamOutput extends BaseStreamOutput { +public abstract class StreamOutput extends OutputStream { private static final int MAX_NESTED_EXCEPTION_LEVEL = 100; + private Version version = Version.CURRENT; private Set features = Collections.emptySet(); /** - * Test if the stream has the specified feature. Features are used when serializing {@link ClusterState.Custom} or - * {@link Metadata.Custom}; see also {@link ClusterState.FeatureAware}. + * The version of the node on the other side of this stream. + */ + public Version getVersion() { + return this.version; + } + + /** + * Set the version of the node on the other side of this stream. + */ + public void setVersion(Version version) { + this.version = version; + } + + /** + * Test if the stream has the specified feature. Features are used when serializing {@code ClusterState.Custom} or + * {@code Metadata.Custom}; see also {@code ClusterState.FeatureAware}. * * @param feature the feature to test * @return true if the stream has the specified feature @@ -139,6 +152,11 @@ public void seek(long position) throws IOException { throw new UnsupportedOperationException(); } + /** + * Writes a single byte. + */ + public abstract void writeByte(byte b) throws IOException; + /** * Writes an array of bytes. * @@ -158,6 +176,15 @@ public void writeBytes(byte[] b, int length) throws IOException { writeBytes(b, 0, length); } + /** + * Writes an array of bytes. + * + * @param b the bytes to write + * @param offset the offset in the byte array + * @param length the number of bytes to write + */ + public abstract void writeBytes(byte[] b, int offset, int length) throws IOException; + /** * Writes an array of bytes. * @@ -202,6 +229,8 @@ public void writeBytesRef(BytesRef bytes) throws IOException { write(bytes.bytes, bytes.offset, bytes.length); } + private static final ThreadLocal scratch = ThreadLocal.withInitial(() -> new byte[1024]); + public final void writeShort(short v) throws IOException { final byte[] buffer = scratch.get(); buffer[0] = (byte) (v >> 8); @@ -209,6 +238,49 @@ public final void writeShort(short v) throws IOException { writeBytes(buffer, 0, 2); } + /** + * Writes an int as four bytes. + */ + public void writeInt(int i) throws IOException { + final byte[] buffer = scratch.get(); + buffer[0] = (byte) (i >> 24); + buffer[1] = (byte) (i >> 16); + buffer[2] = (byte) (i >> 8); + buffer[3] = (byte) i; + writeBytes(buffer, 0, 4); + } + + /** + * Writes an int in a variable-length format. Writes between one and + * five bytes. Smaller values take fewer bytes. Negative numbers + * will always use all 5 bytes and are therefore better serialized + * using {@link #writeInt} + */ + public void writeVInt(int i) throws IOException { + /* + * Shortcut writing single byte because it is very, very common and + * can skip grabbing the scratch buffer. This is marginally slower + * than hand unrolling the entire encoding loop but hand unrolling + * the encoding loop blows out the method size so it can't be inlined. + * In that case benchmarks of the method itself are faster but + * benchmarks of methods that use this method are slower. + * This is philosophically in line with vint in general - it biases + * twoards being simple and fast for smaller numbers. + */ + if (Integer.numberOfLeadingZeros(i) >= 25) { + writeByte((byte) i); + return; + } + byte[] buffer = scratch.get(); + int index = 0; + do { + buffer[index++] = ((byte) ((i & 0x7f) | 0x80)); + i >>>= 7; + } while ((i & ~0x7F) != 0); + buffer[index++] = ((byte) i); + writeBytes(buffer, 0, index); + } + /** * Writes a long as eight bytes. */ @@ -250,7 +322,7 @@ public void writeOptionalVLong(@Nullable Long l) throws IOException { * Writes a long in a variable-length format without first checking if it is negative. Package private for testing. Use * {@link #writeVLong(long)} instead. */ - void writeVLongNoCheck(long i) throws IOException { + public void writeVLongNoCheck(long i) throws IOException { final byte[] buffer = scratch.get(); int index = 0; while ((i & ~0x7F) != 0) { @@ -312,6 +384,27 @@ public void writeOptionalSecureString(@Nullable SecureString secureStr) throws I } } + /** + * Writes an optional {@link Integer}. + */ + public void writeOptionalInt(@Nullable Integer integer) throws IOException { + if (integer == null) { + writeBoolean(false); + } else { + writeBoolean(true); + writeInt(integer); + } + } + + public void writeOptionalVInt(@Nullable Integer integer) throws IOException { + if (integer == null) { + writeBoolean(false); + } else { + writeBoolean(true); + writeVInt(integer); + } + } + public void writeOptionalFloat(@Nullable Float floatValue) throws IOException { if (floatValue == null) { writeBoolean(false); @@ -402,6 +495,39 @@ public final void writeBigInteger(BigInteger v) throws IOException { writeString(v.toString()); } + private static byte ZERO = 0; + private static byte ONE = 1; + private static byte TWO = 2; + + /** + * Writes a boolean. + */ + public void writeBoolean(boolean b) throws IOException { + writeByte(b ? ONE : ZERO); + } + + public void writeOptionalBoolean(@Nullable Boolean b) throws IOException { + if (b == null) { + writeByte(TWO); + } else { + writeBoolean(b); + } + } + + /** + * Forces any buffered output to be written. + */ + @Override + public abstract void flush() throws IOException; + + /** + * Closes this stream to further operations. + */ + @Override + public abstract void close() throws IOException; + + public abstract void reset() throws IOException; + @Override public void write(int b) throws IOException { writeByte((byte) b); @@ -532,10 +658,10 @@ public final void writeOptionalInstant(@Nullable Instant instant) throws IOExcep } } - private static final Map, BaseWriteable.Writer> WRITERS; + private static final Map, Writer> WRITERS; static { - Map, BaseWriteable.Writer> writers = new HashMap<>(); + Map, Writer> writers = new HashMap<>(); writers.put(String.class, (o, v) -> { o.writeByte((byte) 0); o.writeString((String) v); @@ -568,6 +694,7 @@ public final void writeOptionalInstant(@Nullable Instant instant) throws IOExcep }); writers.put(List.class, (o, v) -> { o.writeByte((byte) 7); + @SuppressWarnings("rawtypes") final List list = (List) v; o.writeVInt(list.size()); for (Object item : list) { @@ -659,7 +786,7 @@ public final void writeOptionalInstant(@Nullable Instant instant) throws IOExcep } private static Class getGenericType(Object value) { - Class registeredClass = Writeable.WriteableRegistry.getCustomClassFromInstance(value); + Class registeredClass = WriteableRegistry.getCustomClassFromInstance(value); if (registeredClass != null) { return registeredClass; } else if (value instanceof List) { @@ -689,7 +816,7 @@ public void writeGenericValue(@Nullable Object value) throws IOException { return; } final Class type = getGenericType(value); - BaseWriteable.Writer writer = BaseWriteable.WriteableRegistry.getWriter(type); + Writer writer = WriteableRegistry.getWriter(type); if (writer == null) { // fallback to this local hashmap // todo: move all writers to the registry @@ -977,7 +1104,7 @@ public void writeBuild(final Build build) throws IOException { writeString(build.getQualifiedVersion()); } - boolean failOnTooManyNestedExceptions(Throwable throwable) { + protected boolean failOnTooManyNestedExceptions(Throwable throwable) { throw new AssertionError("too many nested exceptions", throwable); } @@ -1022,7 +1149,7 @@ public void writeOptionalZoneId(@Nullable ZoneId timeZone) throws IOException { /** * Writes a collection to this stream. The corresponding collection can be read from a stream input using - * {@link StreamInput#readList(BaseWriteable.Reader)}. + * {@link StreamInput#readList(Writeable.Reader)}. * * @param collection the collection to write to this stream * @throws IOException if an I/O exception occurs writing the collection @@ -1053,7 +1180,7 @@ public void writeCollection(final Collection collection, final Writer /** * Writes a collection of a strings. The corresponding collection can be read from a stream input using - * {@link StreamInput#readList(BaseWriteable.Reader)}. + * {@link StreamInput#readList(Writeable.Reader)}. * * @param collection the collection of strings * @throws IOException if an I/O exception occurs writing the collection @@ -1064,7 +1191,7 @@ public void writeStringCollection(final Collection collection) throws IO /** * Writes an optional collection of a strings. The corresponding collection can be read from a stream input using - * {@link StreamInput#readList(BaseWriteable.Reader)}. + * {@link StreamInput#readList(Writeable.Reader)}. * * @param collection the collection of strings * @throws IOException if an I/O exception occurs writing the collection diff --git a/libs/core/src/main/java/org/opensearch/core/common/io/stream/BaseWriteable.java b/libs/core/src/main/java/org/opensearch/common/io/stream/Writeable.java similarity index 64% rename from libs/core/src/main/java/org/opensearch/core/common/io/stream/BaseWriteable.java rename to libs/core/src/main/java/org/opensearch/common/io/stream/Writeable.java index fbb8d4120f31d..735413d3642ec 100644 --- a/libs/core/src/main/java/org/opensearch/core/common/io/stream/BaseWriteable.java +++ b/libs/core/src/main/java/org/opensearch/common/io/stream/Writeable.java @@ -5,38 +5,63 @@ * this file be licensed under the Apache-2.0 license or a * compatible open source license. */ -package org.opensearch.core.common.io.stream; + +/* + * 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.common.io.stream; import java.io.IOException; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; /** - * Implementers can be written to a {@code StreamOutput} and read from a {@code StreamInput}. This allows them to be "thrown + * Implementers can be written to a {@linkplain StreamOutput} and read from a {@linkplain StreamInput}. This allows them to be "thrown * 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 */ -public interface BaseWriteable { +public interface Writeable { /** * A WriteableRegistry registers {@link Writer} methods for writing data types over a - * {@link BaseStreamOutput} channel and {@link Reader} methods for reading data from a - * {@link BaseStreamInput} channel. + * {@link StreamOutput} channel and {@link Reader} methods for reading data from a + * {@link StreamInput} channel. * * @opensearch.internal */ class WriteableRegistry { - private static final Map, Writer> WRITER_REGISTRY = new ConcurrentHashMap<>(); + private static final Map, Writer> WRITER_REGISTRY = new ConcurrentHashMap<>(); private static final Map, Class> WRITER_CUSTOM_CLASS_MAP = new ConcurrentHashMap<>(); - private static final Map> READER_REGISTRY = new ConcurrentHashMap<>(); + private static final Map> READER_REGISTRY = new ConcurrentHashMap<>(); /** * registers a streamable writer * * @opensearch.internal */ - public static > void registerWriter(final Class clazz, final W writer) { + public static > void registerWriter(final Class clazz, final W writer) { if (WRITER_REGISTRY.putIfAbsent(clazz, writer) != null) { throw new IllegalArgumentException("Streamable writer already registered for type [" + clazz.getName() + "]"); } @@ -47,7 +72,7 @@ class WriteableRegistry { * * @opensearch.internal */ - public static > void registerReader(final byte ordinal, final R reader) { + public static > void registerReader(final byte ordinal, final R reader) { if (READER_REGISTRY.putIfAbsent(ordinal, reader) != null) { throw new IllegalArgumentException("Streamable reader already registered for ordinal [" + (int) ordinal + "]"); } @@ -63,7 +88,7 @@ public static void registerClassAlias(final Class classInstance, final Class< * Returns the registered writer keyed by the class type */ @SuppressWarnings("unchecked") - public static > W getWriter(final Class clazz) { + public static > W getWriter(final Class clazz) { return (W) WRITER_REGISTRY.get(clazz); } @@ -71,7 +96,7 @@ public static void registerClassAlias(final Class classInstance, final Class< * Returns the ristered reader keyed by the unique ordinal */ @SuppressWarnings("unchecked") - public static > R getReader(final byte b) { + public static > R getReader(final byte b) { return (R) READER_REGISTRY.get(b); } @@ -92,18 +117,18 @@ public static Class getCustomClassFromInstance(final Object value) { } /** - * Write this into the {@linkplain BaseStreamOutput}. + * Write this into the {@linkplain StreamOutput}. */ - void writeTo(final S out) throws IOException; + void writeTo(StreamOutput out) throws IOException; /** - * Reference to a method that can write some object to a {@link BaseStreamOutput}. + * Reference to a method that can write some object to a {@link StreamOutput}. *

- * By convention this is a method from {@link BaseStreamOutput} itself (e.g., {@code StreamOutput#writeString}). If the value can be + * By convention this is a method from {@link StreamOutput} itself (e.g., {@code StreamOutput#writeString}). If the value can be * {@code null}, then the "optional" variant of methods should be used! *

- * Most classes should implement {@code Writeable} and the {@code Writeable#writeTo(BaseStreamOutput)} method should use - * {@link BaseStreamOutput} methods directly or this indirectly: + * Most classes should implement {@code Writeable} and the {@code Writeable#writeTo(StreamOutput)} method should use + * {@link StreamOutput} methods directly or this indirectly: *


      * public void writeTo(StreamOutput out) throws IOException {
      *     out.writeVInt(someValue);
@@ -112,7 +137,7 @@ public static Class getCustomClassFromInstance(final Object value) {
      * 
*/ @FunctionalInterface - interface Writer { + interface Writer { /** * Write {@code V}-type {@code value} to the {@code out}put stream. @@ -120,12 +145,12 @@ interface Writer { * @param out Output to write the {@code value} too * @param value The value to add */ - void write(final S out, V value) throws IOException; + void write(final StreamOutput out, V value) throws IOException; } /** * Reference to a method that can read some object from a stream. By convention this is a constructor that takes - * {@linkplain BaseStreamInput} as an argument for most classes and a static method for things like enums. Returning null from one of these + * {@linkplain StreamInput} as an argument for most classes and a static method for things like enums. Returning null from one of these * is always wrong - for that we use methods like {@code StreamInput#readOptionalWriteable(Reader)}. *

* As most classes will implement this via a constructor (or a static method in the case of enumerations), it's something that should @@ -138,13 +163,13 @@ interface Writer { * */ @FunctionalInterface - interface Reader { + interface Reader { /** * Read {@code V}-type value from a stream. * * @param in Input to read the value from */ - V read(final S in) throws IOException; + V read(final StreamInput in) throws IOException; } } diff --git a/libs/core/src/main/java/org/opensearch/core/common/io/stream/package-info.java b/libs/core/src/main/java/org/opensearch/common/io/stream/package-info.java similarity index 83% rename from libs/core/src/main/java/org/opensearch/core/common/io/stream/package-info.java rename to libs/core/src/main/java/org/opensearch/common/io/stream/package-info.java index 76d0842466b96..c332f90371f60 100644 --- a/libs/core/src/main/java/org/opensearch/core/common/io/stream/package-info.java +++ b/libs/core/src/main/java/org/opensearch/common/io/stream/package-info.java @@ -5,5 +5,6 @@ * this file be licensed under the Apache-2.0 license or a * compatible open source license. */ + /** Core transport stream classes */ -package org.opensearch.core.common.io.stream; +package org.opensearch.common.io.stream; diff --git a/libs/core/src/main/java/org/opensearch/common/package-info.java b/libs/core/src/main/java/org/opensearch/common/package-info.java new file mode 100644 index 0000000000000..211cde81a3b88 --- /dev/null +++ b/libs/core/src/main/java/org/opensearch/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.common; diff --git a/server/src/main/java/org/opensearch/common/settings/SecureString.java b/libs/core/src/main/java/org/opensearch/common/settings/SecureString.java similarity index 100% rename from server/src/main/java/org/opensearch/common/settings/SecureString.java rename to libs/core/src/main/java/org/opensearch/common/settings/SecureString.java diff --git a/libs/core/src/main/java/org/opensearch/common/settings/package-info.java b/libs/core/src/main/java/org/opensearch/common/settings/package-info.java new file mode 100644 index 0000000000000..10c1d5bab5fc1 --- /dev/null +++ b/libs/core/src/main/java/org/opensearch/common/settings/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. + */ + +/** Classes for core Settings module */ +package org.opensearch.common.settings; diff --git a/server/src/main/java/org/opensearch/common/text/Text.java b/libs/core/src/main/java/org/opensearch/common/text/Text.java similarity index 100% rename from server/src/main/java/org/opensearch/common/text/Text.java rename to libs/core/src/main/java/org/opensearch/common/text/Text.java diff --git a/libs/core/src/main/java/org/opensearch/common/text/package-info.java b/libs/core/src/main/java/org/opensearch/common/text/package-info.java new file mode 100644 index 0000000000000..d9ce2ebbc9340 --- /dev/null +++ b/libs/core/src/main/java/org/opensearch/common/text/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. + */ + +/** Classes for core text module */ +package org.opensearch.common.text; diff --git a/server/src/main/java/org/opensearch/common/util/BigArray.java b/libs/core/src/main/java/org/opensearch/common/util/BigArray.java similarity index 100% rename from server/src/main/java/org/opensearch/common/util/BigArray.java rename to libs/core/src/main/java/org/opensearch/common/util/BigArray.java diff --git a/server/src/main/java/org/opensearch/common/util/ByteArray.java b/libs/core/src/main/java/org/opensearch/common/util/ByteArray.java similarity index 100% rename from server/src/main/java/org/opensearch/common/util/ByteArray.java rename to libs/core/src/main/java/org/opensearch/common/util/ByteArray.java diff --git a/libs/core/src/main/java/org/opensearch/common/util/package-info.java b/libs/core/src/main/java/org/opensearch/common/util/package-info.java new file mode 100644 index 0000000000000..f106d27c21881 --- /dev/null +++ b/libs/core/src/main/java/org/opensearch/common/util/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. + */ + +/** Classes for core util module */ +package org.opensearch.common.util; diff --git a/server/src/main/java/org/opensearch/common/xcontent/XContentParserUtils.java b/libs/core/src/main/java/org/opensearch/common/xcontent/XContentParserUtils.java similarity index 100% rename from server/src/main/java/org/opensearch/common/xcontent/XContentParserUtils.java rename to libs/core/src/main/java/org/opensearch/common/xcontent/XContentParserUtils.java diff --git a/libs/core/src/main/java/org/opensearch/common/xcontent/package-info.java b/libs/core/src/main/java/org/opensearch/common/xcontent/package-info.java new file mode 100644 index 0000000000000..c4522c2ae6627 --- /dev/null +++ b/libs/core/src/main/java/org/opensearch/common/xcontent/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. + */ + +/** Base XContent Core Classes */ +package org.opensearch.common.xcontent; diff --git a/libs/core/src/main/java/org/opensearch/core/common/io/package-info.java b/libs/core/src/main/java/org/opensearch/core/common/io/package-info.java new file mode 100644 index 0000000000000..7f5318f53dd35 --- /dev/null +++ b/libs/core/src/main/java/org/opensearch/core/common/io/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. + */ + +/** Classes for core io module */ +package org.opensearch.core.common.io; diff --git a/libs/core/src/main/java/org/opensearch/core/common/io/stream/BaseStreamInput.java b/libs/core/src/main/java/org/opensearch/core/common/io/stream/BaseStreamInput.java deleted file mode 100644 index 178333d529cf6..0000000000000 --- a/libs/core/src/main/java/org/opensearch/core/common/io/stream/BaseStreamInput.java +++ /dev/null @@ -1,341 +0,0 @@ -/* - * 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.io.stream; - -import org.apache.lucene.util.ArrayUtil; -import org.apache.lucene.util.CharsRef; -import org.opensearch.Version; -import org.opensearch.common.Nullable; - -import java.io.EOFException; -import java.io.IOException; -import java.io.InputStream; -import java.util.Collection; -import java.util.Locale; -import java.util.function.IntFunction; - -/** - * Foundation class for reading core types off the transport stream - * - * todo: refactor {@code StreamInput} primitive readers to this class - * - * @opensearch.internal - */ -public abstract class BaseStreamInput extends InputStream { - // Maximum char-count to de-serialize via the thread-local CharsRef buffer - private static final int SMALL_STRING_LIMIT = 1024; - // Reusable bytes for deserializing strings - private static final ThreadLocal stringReadBuffer = ThreadLocal.withInitial(() -> new byte[1024]); - // Thread-local buffer for smaller strings - private static final ThreadLocal smallSpare = ThreadLocal.withInitial(() -> new CharsRef(SMALL_STRING_LIMIT)); - private Version version = Version.CURRENT; - // Larger buffer used for long strings that can't fit into the thread-local buffer - // We don't use a CharsRefBuilder since we exactly know the size of the character array up front - // this prevents calling grow for every character since we don't need this - private CharsRef largeSpare; - - /** - * The version of the node on the other side of this stream. - */ - public Version getVersion() { - return this.version; - } - - /** - * Set the version of the node on the other side of this stream. - */ - public void setVersion(Version version) { - this.version = version; - } - - /** - * Closes the stream to further operations. - */ - @Override - public abstract void close() throws IOException; - - @Override - public abstract int available() throws IOException; - - /** - * This method throws an {@link EOFException} if the given number of bytes can not be read from the this stream. This method might - * be a no-op depending on the underlying implementation if the information of the remaining bytes is not present. - */ - protected abstract void ensureCanReadBytes(int length) throws EOFException; - - /** - * Reads and returns a single byte. - */ - public abstract byte readByte() throws IOException; - - /** - * Reads a specified number of bytes into an array at the specified offset. - * - * @param b the array to read bytes into - * @param offset the offset in the array to start storing bytes - * @param len the number of bytes to read - */ - public abstract void readBytes(byte[] b, int offset, int len) throws IOException; - - public void readFully(byte[] b) throws IOException { - readBytes(b, 0, b.length); - } - - protected boolean readBoolean(final byte value) { - if (value == 0) { - return false; - } else if (value == 1) { - return true; - } else { - final String message = String.format(Locale.ROOT, "unexpected byte [0x%02x]", value); - throw new IllegalStateException(message); - } - } - - /** - * Reads a boolean. - */ - public final boolean readBoolean() throws IOException { - return readBoolean(readByte()); - } - - @Nullable - public final Boolean readOptionalBoolean() throws IOException { - final byte value = readByte(); - if (value == 2) { - return null; - } else { - return readBoolean(value); - } - } - - /** - * Reads four bytes and returns an int. - */ - public int readInt() throws IOException { - return ((readByte() & 0xFF) << 24) | ((readByte() & 0xFF) << 16) | ((readByte() & 0xFF) << 8) | (readByte() & 0xFF); - } - - /** - * Reads an int stored in variable-length format. Reads between one and - * five bytes. Smaller values take fewer bytes. Negative numbers - * will always use all 5 bytes and are therefore better serialized - * using {@link #readInt} - */ - public int readVInt() throws IOException { - byte b = readByte(); - int i = b & 0x7F; - if ((b & 0x80) == 0) { - return i; - } - b = readByte(); - i |= (b & 0x7F) << 7; - if ((b & 0x80) == 0) { - return i; - } - b = readByte(); - i |= (b & 0x7F) << 14; - if ((b & 0x80) == 0) { - return i; - } - b = readByte(); - i |= (b & 0x7F) << 21; - if ((b & 0x80) == 0) { - return i; - } - b = readByte(); - if ((b & 0x80) != 0) { - throw new IOException("Invalid vInt ((" + Integer.toHexString(b) + " & 0x7f) << 28) | " + Integer.toHexString(i)); - } - return i | ((b & 0x7F) << 28); - } - - @Nullable - public Integer readOptionalVInt() throws IOException { - if (readBoolean()) { - return readVInt(); - } - return null; - } - - public String readString() throws IOException { - final int charCount = readArraySize(); - final CharsRef charsRef; - if (charCount > SMALL_STRING_LIMIT) { - if (largeSpare == null) { - largeSpare = new CharsRef(ArrayUtil.oversize(charCount, Character.BYTES)); - } else if (largeSpare.chars.length < charCount) { - // we don't use ArrayUtils.grow since there is no need to copy the array - largeSpare.chars = new char[ArrayUtil.oversize(charCount, Character.BYTES)]; - } - charsRef = largeSpare; - } else { - charsRef = smallSpare.get(); - } - charsRef.length = charCount; - int charsOffset = 0; - int offsetByteArray = 0; - int sizeByteArray = 0; - int missingFromPartial = 0; - final byte[] byteBuffer = stringReadBuffer.get(); - final char[] charBuffer = charsRef.chars; - for (; charsOffset < charCount;) { - final int charsLeft = charCount - charsOffset; - int bufferFree = byteBuffer.length - sizeByteArray; - // Determine the minimum amount of bytes that are left in the string - final int minRemainingBytes; - if (missingFromPartial > 0) { - // One byte for each remaining char except for the already partially read char - minRemainingBytes = missingFromPartial + charsLeft - 1; - missingFromPartial = 0; - } else { - // Each char has at least a single byte - minRemainingBytes = charsLeft; - } - final int toRead; - if (bufferFree < minRemainingBytes) { - // We don't have enough space left in the byte array to read as much as we'd like to so we free up as many bytes in the - // buffer by moving unused bytes that didn't make up a full char in the last iteration to the beginning of the buffer, - // if there are any - if (offsetByteArray > 0) { - sizeByteArray = sizeByteArray - offsetByteArray; - switch (sizeByteArray) { // We only have 0, 1 or 2 => no need to bother with a native call to System#arrayCopy - case 1: - byteBuffer[0] = byteBuffer[offsetByteArray]; - break; - case 2: - byteBuffer[0] = byteBuffer[offsetByteArray]; - byteBuffer[1] = byteBuffer[offsetByteArray + 1]; - break; - } - assert sizeByteArray <= 2 : "We never copy more than 2 bytes here since a char is 3 bytes max"; - toRead = Math.min(bufferFree + offsetByteArray, minRemainingBytes); - offsetByteArray = 0; - } else { - toRead = bufferFree; - } - } else { - toRead = minRemainingBytes; - } - readBytes(byteBuffer, sizeByteArray, toRead); - sizeByteArray += toRead; - // As long as we at least have three bytes buffered we don't need to do any bounds checking when getting the next char since we - // read 3 bytes per char/iteration at most - for (; offsetByteArray < sizeByteArray - 2; offsetByteArray++) { - final int c = byteBuffer[offsetByteArray] & 0xff; - switch (c >> 4) { - case 0: - case 1: - case 2: - case 3: - case 4: - case 5: - case 6: - case 7: - charBuffer[charsOffset++] = (char) c; - break; - case 12: - case 13: - charBuffer[charsOffset++] = (char) ((c & 0x1F) << 6 | byteBuffer[++offsetByteArray] & 0x3F); - break; - case 14: - charBuffer[charsOffset++] = (char) ((c & 0x0F) << 12 | (byteBuffer[++offsetByteArray] & 0x3F) << 6 - | (byteBuffer[++offsetByteArray] & 0x3F)); - break; - default: - BaseStreamInput.throwOnBrokenChar(c); - } - } - // try to extract chars from remaining bytes with bounds checks for multi-byte chars - final int bufferedBytesRemaining = sizeByteArray - offsetByteArray; - for (int i = 0; i < bufferedBytesRemaining; i++) { - final int c = byteBuffer[offsetByteArray] & 0xff; - switch (c >> 4) { - case 0: - case 1: - case 2: - case 3: - case 4: - case 5: - case 6: - case 7: - charBuffer[charsOffset++] = (char) c; - offsetByteArray++; - break; - case 12: - case 13: - missingFromPartial = 2 - (bufferedBytesRemaining - i); - if (missingFromPartial == 0) { - offsetByteArray++; - charBuffer[charsOffset++] = (char) ((c & 0x1F) << 6 | byteBuffer[offsetByteArray++] & 0x3F); - } - ++i; - break; - case 14: - missingFromPartial = 3 - (bufferedBytesRemaining - i); - ++i; - break; - default: - BaseStreamInput.throwOnBrokenChar(c); - } - } - } - return charsRef.toString(); - } - - @Nullable - public String readOptionalString() throws IOException { - if (readBoolean()) { - return readString(); - } - return null; - } - - private static void throwOnBrokenChar(int c) throws IOException { - throw new IOException("Invalid string; unexpected character: " + c + " hex: " + Integer.toHexString(c)); - } - - /** - * Reads a vint via {@link #readVInt()} and applies basic checks to ensure the read array size is sane. - * This method uses {@link #ensureCanReadBytes(int)} to ensure this stream has enough bytes to read for the read array size. - */ - protected int readArraySize() throws IOException { - final int arraySize = readVInt(); - if (arraySize > ArrayUtil.MAX_ARRAY_LENGTH) { - throw new IllegalStateException("array length must be <= to " + ArrayUtil.MAX_ARRAY_LENGTH + " but was: " + arraySize); - } - if (arraySize < 0) { - throw new NegativeArraySizeException("array size must be positive but was: " + arraySize); - } - // lets do a sanity check that if we are reading an array size that is bigger that the remaining bytes we can safely - // throw an exception instead of allocating the array based on the size. A simple corrutpted byte can make a node go OOM - // if the size is large and for perf reasons we allocate arrays ahead of time - ensureCanReadBytes(arraySize); - return arraySize; - } - - /** - * Reads a collection of objects - */ - @SuppressWarnings("unchecked") - protected > C readCollection( - BaseWriteable.Reader reader, - IntFunction constructor, - C empty - ) throws IOException { - int count = readArraySize(); - if (count == 0) { - return empty; - } - C builder = constructor.apply(count); - for (int i = 0; i < count; i++) { - builder.add(reader.read((S) this)); - } - return builder; - } -} diff --git a/libs/core/src/main/java/org/opensearch/core/common/io/stream/BaseStreamOutput.java b/libs/core/src/main/java/org/opensearch/core/common/io/stream/BaseStreamOutput.java deleted file mode 100644 index e2d4edf0a6023..0000000000000 --- a/libs/core/src/main/java/org/opensearch/core/common/io/stream/BaseStreamOutput.java +++ /dev/null @@ -1,150 +0,0 @@ -/* - * 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.io.stream; - -import org.opensearch.Version; -import org.opensearch.common.Nullable; - -import java.io.IOException; -import java.io.OutputStream; - -/** - * Foundation class for writing core types over the transport stream - * - * todo: refactor {@code StreamOutput} primitive writers to this class - * - * @opensearch.internal - */ -public abstract class BaseStreamOutput extends OutputStream { - protected static final ThreadLocal scratch = ThreadLocal.withInitial(() -> new byte[1024]); - private static byte ZERO = 0; - private static byte ONE = 1; - private static byte TWO = 2; - protected Version version = Version.CURRENT; - - /** - * The version of the node on the other side of this stream. - */ - public Version getVersion() { - return this.version; - } - - /** - * Set the version of the node on the other side of this stream. - */ - public void setVersion(Version version) { - this.version = version; - } - - /** - * Closes this stream to further operations. - */ - @Override - public abstract void close() throws IOException; - - /** - * Forces any buffered output to be written. - */ - @Override - public abstract void flush() throws IOException; - - public abstract void reset() throws IOException; - - /** - * Writes a single byte. - */ - public abstract void writeByte(byte b) throws IOException; - - /** - * Writes an array of bytes. - * - * @param b the bytes to write - * @param offset the offset in the byte array - * @param length the number of bytes to write - */ - public abstract void writeBytes(byte[] b, int offset, int length) throws IOException; - - /** - * Writes a boolean. - */ - public void writeBoolean(boolean b) throws IOException { - writeByte(b ? ONE : ZERO); - } - - public void writeOptionalBoolean(@Nullable Boolean b) throws IOException { - if (b == null) { - writeByte(TWO); - } else { - writeBoolean(b); - } - } - - /** - * Writes an int as four bytes. - */ - public void writeInt(int i) throws IOException { - final byte[] buffer = scratch.get(); - buffer[0] = (byte) (i >> 24); - buffer[1] = (byte) (i >> 16); - buffer[2] = (byte) (i >> 8); - buffer[3] = (byte) i; - writeBytes(buffer, 0, 4); - } - - /** - * Writes an optional {@link Integer}. - */ - public void writeOptionalInt(@Nullable Integer integer) throws IOException { - if (integer == null) { - writeBoolean(false); - } else { - writeBoolean(true); - writeInt(integer); - } - } - - /** - * Writes an int in a variable-length format. Writes between one and - * five bytes. Smaller values take fewer bytes. Negative numbers - * will always use all 5 bytes and are therefore better serialized - * using {@link #writeInt} - */ - public void writeVInt(int i) throws IOException { - /* - * Shortcut writing single byte because it is very, very common and - * can skip grabbing the scratch buffer. This is marginally slower - * than hand unrolling the entire encoding loop but hand unrolling - * the encoding loop blows out the method size so it can't be inlined. - * In that case benchmarks of the method itself are faster but - * benchmarks of methods that use this method are slower. - * This is philosophically in line with vint in general - it biases - * twoards being simple and fast for smaller numbers. - */ - if (Integer.numberOfLeadingZeros(i) >= 25) { - writeByte((byte) i); - return; - } - byte[] buffer = scratch.get(); - int index = 0; - do { - buffer[index++] = ((byte) ((i & 0x7f) | 0x80)); - i >>>= 7; - } while ((i & ~0x7F) != 0); - buffer[index++] = ((byte) i); - writeBytes(buffer, 0, index); - } - - public void writeOptionalVInt(@Nullable Integer integer) throws IOException { - if (integer == null) { - writeBoolean(false); - } else { - writeBoolean(true); - writeVInt(integer); - } - } -} diff --git a/server/src/main/java/org/opensearch/index/Index.java b/libs/core/src/main/java/org/opensearch/index/Index.java similarity index 100% rename from server/src/main/java/org/opensearch/index/Index.java rename to libs/core/src/main/java/org/opensearch/index/Index.java diff --git a/libs/core/src/main/java/org/opensearch/index/package-info.java b/libs/core/src/main/java/org/opensearch/index/package-info.java new file mode 100644 index 0000000000000..bcb93e8c78856 --- /dev/null +++ b/libs/core/src/main/java/org/opensearch/index/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. + */ + +/** Classes for core index module */ +package org.opensearch.index; diff --git a/server/src/main/java/org/opensearch/index/shard/ShardId.java b/libs/core/src/main/java/org/opensearch/index/shard/ShardId.java similarity index 96% rename from server/src/main/java/org/opensearch/index/shard/ShardId.java rename to libs/core/src/main/java/org/opensearch/index/shard/ShardId.java index 9f226cd2df76a..ac24130ed90c9 100644 --- a/server/src/main/java/org/opensearch/index/shard/ShardId.java +++ b/libs/core/src/main/java/org/opensearch/index/shard/ShardId.java @@ -32,7 +32,7 @@ package org.opensearch.index.shard; -import org.opensearch.cluster.metadata.IndexMetadata; +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; @@ -109,7 +109,7 @@ public static ShardId fromString(String shardIdString) { } String indexName = shardIdString.substring(1, splitPosition); int shardId = Integer.parseInt(shardIdString.substring(splitPosition + 2, shardIdString.length() - 1)); - return new ShardId(new Index(indexName, IndexMetadata.INDEX_UUID_NA_VALUE), shardId); + return new ShardId(new Index(indexName, Strings.UNKNOWN_UUID_VALUE), shardId); } @Override diff --git a/libs/core/src/main/java/org/opensearch/index/shard/package-info.java b/libs/core/src/main/java/org/opensearch/index/shard/package-info.java new file mode 100644 index 0000000000000..3ae1d95494bd6 --- /dev/null +++ b/libs/core/src/main/java/org/opensearch/index/shard/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. + */ + +/** Classes for core index/shard module */ +package org.opensearch.index.shard; diff --git a/server/src/main/java/org/opensearch/index/snapshots/IndexShardSnapshotException.java b/libs/core/src/main/java/org/opensearch/index/snapshots/IndexShardSnapshotException.java similarity index 100% rename from server/src/main/java/org/opensearch/index/snapshots/IndexShardSnapshotException.java rename to libs/core/src/main/java/org/opensearch/index/snapshots/IndexShardSnapshotException.java diff --git a/server/src/main/java/org/opensearch/index/snapshots/IndexShardSnapshotFailedException.java b/libs/core/src/main/java/org/opensearch/index/snapshots/IndexShardSnapshotFailedException.java similarity index 100% rename from server/src/main/java/org/opensearch/index/snapshots/IndexShardSnapshotFailedException.java rename to libs/core/src/main/java/org/opensearch/index/snapshots/IndexShardSnapshotFailedException.java diff --git a/libs/core/src/main/java/org/opensearch/index/snapshots/package-info.java b/libs/core/src/main/java/org/opensearch/index/snapshots/package-info.java new file mode 100644 index 0000000000000..7102459987dbf --- /dev/null +++ b/libs/core/src/main/java/org/opensearch/index/snapshots/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. + */ + +/** Classes for core snapshots module */ +package org.opensearch.index.snapshots; diff --git a/server/src/main/java/org/opensearch/rest/RestStatus.java b/libs/core/src/main/java/org/opensearch/rest/RestStatus.java similarity index 100% rename from server/src/main/java/org/opensearch/rest/RestStatus.java rename to libs/core/src/main/java/org/opensearch/rest/RestStatus.java diff --git a/libs/core/src/main/java/org/opensearch/rest/package-info.java b/libs/core/src/main/java/org/opensearch/rest/package-info.java new file mode 100644 index 0000000000000..a629540e3858c --- /dev/null +++ b/libs/core/src/main/java/org/opensearch/rest/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. + */ + +/** Classes for core REST module */ +package org.opensearch.rest; 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 a39f66f571d0a..0bf41f37c45a7 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.BaseExceptionsHelper; +import org.opensearch.ExceptionsHelper; 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( - BaseExceptionsHelper.detailedMessage(e), + ExceptionsHelper.detailedMessage(e), containsString("[geotile_grid] precision doesn't support values of type: VALUE_BOOLEAN") ); } 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 5b6102d1a3ad4..449e89cdc5f3c 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 @@ -32,7 +32,7 @@ package org.opensearch.script.mustache; -import org.opensearch.BaseOpenSearchException; +import org.opensearch.OpenSearchException; import org.opensearch.action.ActionResponse; import org.opensearch.action.search.MultiSearchResponse; import org.opensearch.common.Nullable; @@ -166,7 +166,7 @@ public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params par for (Item item : items) { if (item.isFailure()) { builder.startObject(); - BaseOpenSearchException.generateFailureXContent(builder, params, item.getFailure(), true); + OpenSearchException.generateFailureXContent(builder, params, item.getFailure(), true); builder.endObject(); } else { item.getResponse().toXContent(builder, params); 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 35ceb60dec7ed..6eb974c77a5f3 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.BaseOpenSearchException; +import org.opensearch.OpenSearchException; 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()); - BaseOpenSearchException[] rootCauses = BaseOpenSearchException.guessRootCauses(response.getFailures().get("broken_query")); + OpenSearchException[] rootCauses = OpenSearchException.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/RankEvalResponse.java b/modules/rank-eval/src/main/java/org/opensearch/index/rankeval/RankEvalResponse.java index 58a940e1618da..180069ab91bd6 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,7 +32,6 @@ package org.opensearch.index.rankeval; -import org.opensearch.BaseOpenSearchException; import org.opensearch.OpenSearchException; import org.opensearch.action.ActionResponse; import org.opensearch.core.ParseField; @@ -137,7 +136,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.startObject("failures"); for (String key : failures.keySet()) { builder.startObject(key); - BaseOpenSearchException.generateFailureXContent(builder, params, failures.get(key), true); + OpenSearchException.generateFailureXContent(builder, params, failures.get(key), true); builder.endObject(); } builder.endObject(); diff --git a/modules/reindex/src/main/java/org/opensearch/index/reindex/remote/RemoteResponseParsers.java b/modules/reindex/src/main/java/org/opensearch/index/reindex/remote/RemoteResponseParsers.java index b1e4d53c90d92..5a4674e146585 100644 --- a/modules/reindex/src/main/java/org/opensearch/index/reindex/remote/RemoteResponseParsers.java +++ b/modules/reindex/src/main/java/org/opensearch/index/reindex/remote/RemoteResponseParsers.java @@ -35,9 +35,9 @@ import org.apache.lucene.search.TotalHits; import org.opensearch.LegacyESVersion; import org.opensearch.Version; +import org.opensearch.common.bytes.BytesReference; import org.opensearch.core.ParseField; import org.opensearch.common.ParsingException; -import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.collect.Tuple; import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; import org.opensearch.core.xcontent.ConstructingObjectParser; 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 e11ad3b5013c1..61336156a83d0 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,7 +33,6 @@ 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; @@ -216,7 +215,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(BaseExceptionsHelper.stackTrace(e), containsString(OpenSearchRejectedExecutionException.class.getSimpleName())); + assertThat(ExceptionsHelper.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/qa/smoke-test-http/src/test/java/org/opensearch/http/DetailedErrorsEnabledIT.java b/qa/smoke-test-http/src/test/java/org/opensearch/http/DetailedErrorsEnabledIT.java index e2ccf86d31dbf..76f801c75d866 100644 --- a/qa/smoke-test-http/src/test/java/org/opensearch/http/DetailedErrorsEnabledIT.java +++ b/qa/smoke-test-http/src/test/java/org/opensearch/http/DetailedErrorsEnabledIT.java @@ -58,7 +58,7 @@ public void testThatErrorTraceWorksByDefault() throws IOException, ParseExceptio Response response = e.getResponse(); assertThat(response.getHeader("Content-Type"), containsString("application/json")); assertThat(EntityUtils.toString(response.getEntity()), - containsString("\"stack_trace\":\"[Validation Failed: 1: index / indices is missing;]; " + + containsString("\"stack_trace\":\"OpenSearchException[Validation Failed: 1: index / indices is missing;]; " + "nested: ActionRequestValidationException[Validation Failed: 1:")); } diff --git a/server/src/internalClusterTest/java/org/opensearch/action/RejectionActionIT.java b/server/src/internalClusterTest/java/org/opensearch/action/RejectionActionIT.java index f2f55b238ab7c..bda24b48b7f10 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.BaseExceptionsHelper; +import org.opensearch.ExceptionsHelper; 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 = BaseExceptionsHelper.unwrapCause(t); + Throwable unwrap = ExceptionsHelper.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/search/TransportSearchIT.java b/server/src/internalClusterTest/java/org/opensearch/action/search/TransportSearchIT.java index 895d7ebea88b6..8cd7bdeb0c822 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.BaseExceptionsHelper; +import org.opensearch.ExceptionsHelper; import org.opensearch.action.ActionListener; import org.opensearch.action.admin.cluster.node.stats.NodeStats; import org.opensearch.action.admin.cluster.node.stats.NodesStatsRequest; @@ -415,7 +415,7 @@ public void onFailure(Exception e) { SearchPhaseExecutionException.class, () -> client.prepareSearch("test").addAggregation(new TestAggregationBuilder("test")).get() ); - assertThat(BaseExceptionsHelper.unwrapCause(exc).getCause().getMessage(), containsString("")); + assertThat(ExceptionsHelper.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(BaseExceptionsHelper.unwrapCause(exc).getCause().getMessage(), containsString("")); + assertThat(ExceptionsHelper.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(BaseExceptionsHelper.unwrapCause(exc).getCause().getMessage(), containsString("boom")); + assertThat(ExceptionsHelper.unwrapCause(exc).getCause().getMessage(), containsString("boom")); } assertBusy(() -> assertThat(requestBreakerUsed(), equalTo(0L))); } diff --git a/server/src/internalClusterTest/java/org/opensearch/blocks/SimpleBlocksIT.java b/server/src/internalClusterTest/java/org/opensearch/blocks/SimpleBlocksIT.java index 872da7bb12b8e..8ede3e25b2e1a 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.BaseExceptionsHelper; +import org.opensearch.ExceptionsHelper; 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 = BaseExceptionsHelper.unwrapCause(t); + Throwable cause = ExceptionsHelper.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 = BaseExceptionsHelper.unwrapCause(t); + Throwable cause = ExceptionsHelper.unwrapCause(t); if (cause instanceof ClusterBlockException) { ClusterBlockException e = (ClusterBlockException) cause; assertThat(e.blocks(), hasSize(1)); 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 6ef3848cec9bb..2dc6b2085b866 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.BaseExceptionsHelper; +import org.opensearch.ExceptionsHelper; 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 = BaseExceptionsHelper.unwrapCause(bulkItemResponse.getFailure().getCause()); + Throwable cause = ExceptionsHelper.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 995f11bcd79c6..28bd5a6ae252d 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.BaseExceptionsHelper; +import org.opensearch.ExceptionsHelper; 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 = BaseExceptionsHelper.unwrapCause(throwable); + final Throwable t = ExceptionsHelper.unwrapCause(throwable); if (t instanceof ClusterBlockException) { ClusterBlockException clusterBlockException = (ClusterBlockException) t; assertThat(clusterBlockException.blocks(), hasSize(1)); 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 60a7da4f4912a..b73b7722f9728 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.BaseOpenSearchException; +import org.opensearch.OpenSearchException; 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 BaseOpenSearchException); + assertTrue(nestedException instanceof OpenSearchException); 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 6f97b8ef40ffe..617c5745c9bba 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.BaseOpenSearchException; +import org.opensearch.OpenSearchException; 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) { - BaseOpenSearchException[] rootCauses = e.guessRootCauses(); + OpenSearchException[] rootCauses = e.guessRootCauses(); if (rootCauses.length == 1) { - BaseOpenSearchException rootCause = rootCauses[0]; + OpenSearchException 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 e56c2b3c713bd..dae788abe0d10 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.BaseOpenSearchException; +import org.opensearch.OpenSearchException; 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) { - BaseOpenSearchException[] rootCauses = e.guessRootCauses(); + OpenSearchException[] rootCauses = e.guessRootCauses(); if (rootCauses.length == 1) { - BaseOpenSearchException rootCause = rootCauses[0]; + OpenSearchException 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 96b256dc81ce1..fa8e823545b36 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,7 +31,6 @@ 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; @@ -421,9 +420,9 @@ public void testSingleValuedFieldOrderedByIllegalAgg() throws Exception { .get(); fail("Expected an exception"); } catch (SearchPhaseExecutionException e) { - BaseOpenSearchException[] rootCauses = e.guessRootCauses(); + OpenSearchException[] rootCauses = e.guessRootCauses(); if (rootCauses.length == 1) { - BaseOpenSearchException rootCause = rootCauses[0]; + OpenSearchException 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 eb6153493323c..406c57d044259 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.BaseExceptionsHelper; +import org.opensearch.ExceptionsHelper; 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 = BaseExceptionsHelper.unwrapCause(e); + Throwable cause = ExceptionsHelper.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 fc4a2908ae563..85fe794b05fc6 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.BaseExceptionsHelper; +import org.opensearch.ExceptionsHelper; 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 = BaseExceptionsHelper.unwrapCause(ex); + Throwable cause = ExceptionsHelper.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 4d7fa05ba5043..1da079781dc63 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.BaseExceptionsHelper; +import org.opensearch.ExceptionsHelper; 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 = BaseExceptionsHelper.unwrapCause(e); + Throwable cause = ExceptionsHelper.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 = BaseExceptionsHelper.unwrapCause(e); + Throwable cause = ExceptionsHelper.unwrapCause(e); if (cause == null) { throw e; } else if (cause instanceof SearchPhaseExecutionException) { diff --git a/server/src/internalClusterTest/java/org/opensearch/snapshots/SharedClusterSnapshotRestoreIT.java b/server/src/internalClusterTest/java/org/opensearch/snapshots/SharedClusterSnapshotRestoreIT.java index 767fd5ea8fde3..9736aba92f3d1 100644 --- a/server/src/internalClusterTest/java/org/opensearch/snapshots/SharedClusterSnapshotRestoreIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/snapshots/SharedClusterSnapshotRestoreIT.java @@ -34,7 +34,7 @@ import org.apache.lucene.util.BytesRef; -import org.opensearch.BaseExceptionsHelper; +import org.opensearch.ExceptionsHelper; import org.opensearch.OpenSearchException; import org.opensearch.Version; import org.opensearch.action.ActionFuture; @@ -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(BaseExceptionsHelper.stackTrace(ex), containsString("Random IOException")); + assertThat(ExceptionsHelper.stackTrace(ex), containsString("Random IOException")); } logger.info("--> snapshot with no I/O failures"); diff --git a/server/src/main/java/org/opensearch/OpenSearchException.java b/server/src/main/java/org/opensearch/OpenSearchServerException.java similarity index 67% rename from server/src/main/java/org/opensearch/OpenSearchException.java rename to server/src/main/java/org/opensearch/OpenSearchServerException.java index 5d258b338f57e..825197c07b977 100644 --- a/server/src/main/java/org/opensearch/OpenSearchException.java +++ b/server/src/main/java/org/opensearch/OpenSearchServerException.java @@ -6,93 +6,37 @@ * 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.action.support.replication.ReplicationOperation; -import org.opensearch.cluster.action.shard.ShardStateAction; -import org.opensearch.cluster.routing.NodeWeighedAwayException; -import org.opensearch.cluster.routing.PreferenceBasedSearchNotAllowedException; -import org.opensearch.cluster.routing.UnsupportedWeightedRoutingStateException; -import org.opensearch.cluster.service.ClusterManagerThrottlingException; -import org.opensearch.common.CheckedFunction; -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.core.xcontent.XContentBuilder; -import org.opensearch.core.xcontent.XContentParser; -import org.opensearch.index.Index; -import org.opensearch.index.shard.ShardId; -import org.opensearch.rest.RestStatus; -import org.opensearch.search.aggregations.MultiBucketConsumerService; -import org.opensearch.search.pipeline.SearchPipelineProcessingException; -import org.opensearch.snapshots.SnapshotInUseDeletionException; -import org.opensearch.transport.TcpTransport; +import org.opensearch.index.snapshots.IndexShardSnapshotException; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import static org.opensearch.BaseOpenSearchException.OpenSearchExceptionHandleRegistry.registerExceptionHandle; +import static org.opensearch.OpenSearchException.OpenSearchExceptionHandle; +import static org.opensearch.OpenSearchException.OpenSearchExceptionHandleRegistry.registerExceptionHandle; +import static org.opensearch.OpenSearchException.UNKNOWN_VERSION_ADDED; import static org.opensearch.Version.V_2_1_0; import static org.opensearch.Version.V_2_4_0; import static org.opensearch.Version.V_2_5_0; import static org.opensearch.Version.V_2_6_0; import static org.opensearch.Version.V_2_7_0; import static org.opensearch.Version.V_3_0_0; -import static org.opensearch.cluster.metadata.IndexMetadata.INDEX_UUID_NA_VALUE; -import static org.opensearch.common.xcontent.XContentParserUtils.ensureExpectedToken; -import static org.opensearch.common.xcontent.XContentParserUtils.ensureFieldName; /** - * A base class for all opensearch exceptions. + * Utility class to register server exceptions * * @opensearch.internal */ -public class OpenSearchException extends BaseOpenSearchException implements Writeable { +public final class OpenSearchServerException { + + private OpenSearchServerException() { + // no ctor: + } /** * Setting a higher base exception id to avoid conflicts. */ private static final int CUSTOM_ELASTICSEARCH_EXCEPTIONS_BASE_ID = 10000; - static { - registerExceptionHandle( - new OpenSearchExceptionHandle( - org.opensearch.index.snapshots.IndexShardSnapshotFailedException.class, - org.opensearch.index.snapshots.IndexShardSnapshotFailedException::new, - 0, - UNKNOWN_VERSION_ADDED - ) - ); + public static void registerExceptions() { registerExceptionHandle( new OpenSearchExceptionHandle( org.opensearch.search.dfs.DfsPhaseExecutionException.class, @@ -384,14 +328,6 @@ public class OpenSearchException extends BaseOpenSearchException implements Writ UNKNOWN_VERSION_ADDED ) ); - registerExceptionHandle( - new OpenSearchExceptionHandle( - org.opensearch.common.ParsingException.class, - org.opensearch.common.ParsingException::new, - 40, - UNKNOWN_VERSION_ADDED - ) - ); registerExceptionHandle( new OpenSearchExceptionHandle( org.opensearch.index.shard.IndexShardClosedException.class, @@ -518,14 +454,6 @@ public class OpenSearchException extends BaseOpenSearchException implements Writ // 59 used to be OpenSearchRejectedExecutionException // 60 used to be for EarlyTerminationException // 61 used to be for RoutingValidationException - registerExceptionHandle( - new OpenSearchExceptionHandle( - org.opensearch.common.io.stream.NotSerializableExceptionWrapper.class, - org.opensearch.common.io.stream.NotSerializableExceptionWrapper::new, - 62, - UNKNOWN_VERSION_ADDED - ) - ); registerExceptionHandle( new OpenSearchExceptionHandle( org.opensearch.indices.AliasFilterParsingException.class, @@ -560,7 +488,12 @@ public class OpenSearchException extends BaseOpenSearchException implements Writ ) ); registerExceptionHandle( - new OpenSearchExceptionHandle(OpenSearchException.class, OpenSearchException::new, 68, UNKNOWN_VERSION_ADDED) + new OpenSearchExceptionHandle( + org.opensearch.OpenSearchException.class, + org.opensearch.OpenSearchException::new, + 68, + UNKNOWN_VERSION_ADDED + ) ); registerExceptionHandle( new OpenSearchExceptionHandle( @@ -751,12 +684,7 @@ public class OpenSearchException extends BaseOpenSearchException implements Writ ) ); registerExceptionHandle( - new OpenSearchExceptionHandle( - org.opensearch.index.snapshots.IndexShardSnapshotException.class, - org.opensearch.index.snapshots.IndexShardSnapshotException::new, - 98, - UNKNOWN_VERSION_ADDED - ) + new OpenSearchExceptionHandle(IndexShardSnapshotException.class, IndexShardSnapshotException::new, 98, UNKNOWN_VERSION_ADDED) ); registerExceptionHandle( new OpenSearchExceptionHandle( @@ -889,8 +817,8 @@ public class OpenSearchException extends BaseOpenSearchException implements Writ ); registerExceptionHandle( new OpenSearchExceptionHandle( - ReplicationOperation.RetryOnPrimaryException.class, - ReplicationOperation.RetryOnPrimaryException::new, + org.opensearch.action.support.replication.ReplicationOperation.RetryOnPrimaryException.class, + org.opensearch.action.support.replication.ReplicationOperation.RetryOnPrimaryException::new, 117, UNKNOWN_VERSION_ADDED ) @@ -939,8 +867,8 @@ public class OpenSearchException extends BaseOpenSearchException implements Writ // 124 used to be Script.ScriptParseException registerExceptionHandle( new OpenSearchExceptionHandle( - TcpTransport.HttpRequestOnTransportException.class, - TcpTransport.HttpRequestOnTransportException::new, + org.opensearch.transport.TcpTransport.HttpRequestOnTransportException.class, + org.opensearch.transport.TcpTransport.HttpRequestOnTransportException::new, 125, UNKNOWN_VERSION_ADDED ) @@ -1045,8 +973,8 @@ public class OpenSearchException extends BaseOpenSearchException implements Writ ); registerExceptionHandle( new OpenSearchExceptionHandle( - ShardStateAction.NoLongerPrimaryShardException.class, - ShardStateAction.NoLongerPrimaryShardException::new, + org.opensearch.cluster.action.shard.ShardStateAction.NoLongerPrimaryShardException.class, + org.opensearch.cluster.action.shard.ShardStateAction.NoLongerPrimaryShardException::new, 142, UNKNOWN_VERSION_ADDED ) @@ -1094,8 +1022,8 @@ public class OpenSearchException extends BaseOpenSearchException implements Writ // 148 was UnknownNamedObjectException registerExceptionHandle( new OpenSearchExceptionHandle( - MultiBucketConsumerService.TooManyBucketsException.class, - MultiBucketConsumerService.TooManyBucketsException::new, + org.opensearch.search.aggregations.MultiBucketConsumerService.TooManyBucketsException.class, + org.opensearch.search.aggregations.MultiBucketConsumerService.TooManyBucketsException::new, 149, UNKNOWN_VERSION_ADDED ) @@ -1222,39 +1150,51 @@ public class OpenSearchException extends BaseOpenSearchException implements Writ ); registerExceptionHandle( new OpenSearchExceptionHandle( - ClusterManagerThrottlingException.class, - ClusterManagerThrottlingException::new, + org.opensearch.cluster.service.ClusterManagerThrottlingException.class, + org.opensearch.cluster.service.ClusterManagerThrottlingException::new, 165, Version.V_2_5_0 ) ); registerExceptionHandle( new OpenSearchExceptionHandle( - SnapshotInUseDeletionException.class, - SnapshotInUseDeletionException::new, + org.opensearch.snapshots.SnapshotInUseDeletionException.class, + org.opensearch.snapshots.SnapshotInUseDeletionException::new, 166, UNKNOWN_VERSION_ADDED ) ); registerExceptionHandle( new OpenSearchExceptionHandle( - UnsupportedWeightedRoutingStateException.class, - UnsupportedWeightedRoutingStateException::new, + org.opensearch.cluster.routing.UnsupportedWeightedRoutingStateException.class, + org.opensearch.cluster.routing.UnsupportedWeightedRoutingStateException::new, 167, V_2_5_0 ) ); registerExceptionHandle( new OpenSearchExceptionHandle( - PreferenceBasedSearchNotAllowedException.class, - PreferenceBasedSearchNotAllowedException::new, + org.opensearch.cluster.routing.PreferenceBasedSearchNotAllowedException.class, + org.opensearch.cluster.routing.PreferenceBasedSearchNotAllowedException::new, 168, V_2_6_0 ) ); - registerExceptionHandle(new OpenSearchExceptionHandle(NodeWeighedAwayException.class, NodeWeighedAwayException::new, 169, V_2_6_0)); registerExceptionHandle( - new OpenSearchExceptionHandle(SearchPipelineProcessingException.class, SearchPipelineProcessingException::new, 170, V_2_7_0) + new OpenSearchExceptionHandle( + org.opensearch.cluster.routing.NodeWeighedAwayException.class, + org.opensearch.cluster.routing.NodeWeighedAwayException::new, + 169, + V_2_6_0 + ) + ); + registerExceptionHandle( + new OpenSearchExceptionHandle( + org.opensearch.search.pipeline.SearchPipelineProcessingException.class, + org.opensearch.search.pipeline.SearchPipelineProcessingException::new, + 170, + V_2_7_0 + ) ); registerExceptionHandle( new OpenSearchExceptionHandle( @@ -1265,358 +1205,4 @@ public class OpenSearchException extends BaseOpenSearchException implements Writ ) ); } - - /** - * Construct a OpenSearchException with the specified cause exception. - */ - public OpenSearchException(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 OpenSearchException(String msg, Object... args) { - super(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 OpenSearchException(String msg, Throwable cause, Object... args) { - super(msg, cause, args); - } - - public OpenSearchException(StreamInput in) throws IOException { - super(in.readOptionalString(), in.readException()); - readStackTrace(this, in); - headers.putAll(in.readMapOfLists(StreamInput::readString, StreamInput::readString)); - metadata.putAll(in.readMapOfLists(StreamInput::readString, StreamInput::readString)); - } - - /** - * Returns the rest status code associated with this exception. - */ - public RestStatus status() { - Throwable cause = unwrapCause(); - if (cause == this) { - return RestStatus.INTERNAL_SERVER_ERROR; - } else { - return ExceptionsHelper.status(cause); - } - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeOptionalString(this.getMessage()); - out.writeException(this.getCause()); - writeStackTraces(this, out, StreamOutput::writeException); - out.writeMapOfLists(headers, StreamOutput::writeString, StreamOutput::writeString); - out.writeMapOfLists(metadata, StreamOutput::writeString, StreamOutput::writeString); - } - - /** - * Returns true iff the given class is a registered for an exception to be read. - */ - public static boolean isRegistered(final Class exception, Version version) { - return OpenSearchExceptionHandleRegistry.isRegistered(exception, version); - } - - static Set> getRegisteredKeys() { // for testing - return OpenSearchExceptionHandleRegistry.getRegisteredKeys(); - } - - /** - * Returns the serialization id the given exception. - */ - public static int getId(final Class exception) { - return OpenSearchExceptionHandleRegistry.getId(exception); - } - - /** - * Generate a {@link OpenSearchException} from a {@link XContentParser}. This does not - * return the original exception type (ie NodeClosedException for example) but just wraps - * the type, the reason and the cause of the exception. It also recursively parses the - * tree structure of the cause, returning it as a tree structure of {@link OpenSearchException} - * instances. - */ - public static OpenSearchException fromXContent(XContentParser parser) throws IOException { - XContentParser.Token token = parser.nextToken(); - ensureExpectedToken(XContentParser.Token.FIELD_NAME, token, parser); - return innerFromXContent(parser, false); - } - - public static OpenSearchException innerFromXContent(XContentParser parser, boolean parseRootCauses) throws IOException { - XContentParser.Token token = parser.currentToken(); - ensureExpectedToken(XContentParser.Token.FIELD_NAME, token, parser); - - String type = null, reason = null, stack = null; - OpenSearchException cause = null; - Map> metadata = new HashMap<>(); - Map> headers = new HashMap<>(); - List rootCauses = new ArrayList<>(); - List suppressed = new ArrayList<>(); - - for (; token == XContentParser.Token.FIELD_NAME; token = parser.nextToken()) { - String currentFieldName = parser.currentName(); - token = parser.nextToken(); - - if (token.isValue()) { - if (BaseExceptionsHelper.TYPE.equals(currentFieldName)) { - type = parser.text(); - } else if (BaseExceptionsHelper.REASON.equals(currentFieldName)) { - reason = parser.text(); - } 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 (BaseExceptionsHelper.CAUSED_BY.equals(currentFieldName)) { - cause = fromXContent(parser); - } else if (BaseExceptionsHelper.HEADER.equals(currentFieldName)) { - while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) { - if (token == XContentParser.Token.FIELD_NAME) { - currentFieldName = parser.currentName(); - } else { - List values = headers.getOrDefault(currentFieldName, new ArrayList<>()); - if (token == XContentParser.Token.VALUE_STRING) { - values.add(parser.text()); - } else if (token == XContentParser.Token.START_ARRAY) { - while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { - if (token == XContentParser.Token.VALUE_STRING) { - values.add(parser.text()); - } else { - parser.skipChildren(); - } - } - } else if (token == XContentParser.Token.START_OBJECT) { - parser.skipChildren(); - } - headers.put(currentFieldName, values); - } - } - } else { - // Any additional metadata object added by the metadataToXContent method is ignored - // and skipped, so that the parser does not fail on unknown fields. The parser only - // support metadata key-pairs and metadata arrays of values. - parser.skipChildren(); - } - } else if (token == XContentParser.Token.START_ARRAY) { - if (parseRootCauses && ROOT_CAUSE.equals(currentFieldName)) { - while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { - rootCauses.add(fromXContent(parser)); - } - } else if (BaseExceptionsHelper.SUPPRESSED.match(currentFieldName, parser.getDeprecationHandler())) { - while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { - suppressed.add(fromXContent(parser)); - } - } else { - // Parse the array and add each item to the corresponding list of metadata. - // Arrays of objects are not supported yet and just ignored and skipped. - List values = new ArrayList<>(); - while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) { - if (token == XContentParser.Token.VALUE_STRING) { - values.add(parser.text()); - } else { - parser.skipChildren(); - } - } - if (values.size() > 0) { - if (metadata.containsKey(currentFieldName)) { - values.addAll(metadata.get(currentFieldName)); - } - metadata.put(currentFieldName, values); - } - } - } - } - - OpenSearchException e = new OpenSearchException(buildMessage(type, reason, stack), cause); - for (Map.Entry> entry : metadata.entrySet()) { - // subclasses can print out additional metadata through the metadataToXContent method. Simple key-value pairs will be - // parsed back and become part of this metadata set, while objects and arrays are not supported when parsing back. - // Those key-value pairs become part of the metadata set and inherit the "opensearch." prefix as that is currently required - // 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(BaseExceptionsHelper.OPENSEARCH_PREFIX_KEY + entry.getKey(), entry.getValue()); - } - for (Map.Entry> header : headers.entrySet()) { - e.addHeader(header.getKey(), header.getValue()); - } - - // Adds root causes as suppressed exception. This way they are not lost - // after parsing and can be retrieved using getSuppressed() method. - for (OpenSearchException rootCause : rootCauses) { - e.addSuppressed(rootCause); - } - for (OpenSearchException s : suppressed) { - e.addSuppressed(s); - } - return e; - } - - /** - * Parses the output of {@link #generateFailureXContent(XContentBuilder, Params, Exception, boolean)} - */ - public static OpenSearchException failureFromXContent(XContentParser parser) throws IOException { - XContentParser.Token token = parser.currentToken(); - ensureFieldName(parser, token, ERROR); - - token = parser.nextToken(); - if (token.isValue()) { - return new OpenSearchException(buildMessage("exception", parser.text(), null)); - } - - ensureExpectedToken(XContentParser.Token.START_OBJECT, token, parser); - token = parser.nextToken(); - - // Root causes are parsed in the innerFromXContent() and are added as suppressed exceptions. - return innerFromXContent(parser, true); - } - - @Override - public String toString() { - StringBuilder builder = new StringBuilder(); - if (metadata.containsKey(INDEX_METADATA_KEY)) { - builder.append(getIndex()); - if (metadata.containsKey(SHARD_METADATA_KEY)) { - builder.append('[').append(getShardId()).append(']'); - } - builder.append(' '); - } - return builder.append(BaseExceptionsHelper.detailedMessage(this).trim()).toString(); - } - - /** - * Deserializes stacktrace elements as well as suppressed exceptions from the given output stream and - * adds it to the given exception. - */ - public static T readStackTrace(T throwable, StreamInput in) throws IOException { - throwable.setStackTrace(in.readArray(i -> { - final String declaringClasss = i.readString(); - final String fileName = i.readOptionalString(); - final String methodName = i.readString(); - final int lineNumber = i.readVInt(); - return new StackTraceElement(declaringClasss, methodName, fileName, lineNumber); - }, StackTraceElement[]::new)); - - int numSuppressed = in.readVInt(); - for (int i = 0; i < numSuppressed; i++) { - throwable.addSuppressed(in.readException()); - } - return throwable; - } - - /** - * Serializes the given exceptions stacktrace elements as well as it's suppressed exceptions to the given output stream. - */ - public static T writeStackTraces(T throwable, StreamOutput out, Writer exceptionWriter) - throws IOException { - out.writeArray((o, v) -> { - o.writeString(v.getClassName()); - o.writeOptionalString(v.getFileName()); - o.writeString(v.getMethodName()); - o.writeVInt(v.getLineNumber()); - }, throwable.getStackTrace()); - out.writeArray(exceptionWriter, throwable.getSuppressed()); - return throwable; - } - - /** - * This is the list of Exceptions OpenSearch can throw over the wire or save into a corruption marker. Each value in the enum is a - * single exception tying the Class to an id for use of the encode side and the id back to a constructor for use on the decode side. As - * such its ok if the exceptions to change names so long as their constructor can still read the exception. Each exception is listed - * in id order below. If you want to remove an exception leave a tombstone comment and mark the id as null in - * ExceptionSerializationTests.testIds.ids. - */ - protected static class OpenSearchExceptionHandle extends BaseOpenSearchExceptionHandle { - OpenSearchExceptionHandle( - final Class exceptionClass, - final CheckedFunction constructor, - final int id, - final Version versionAdded - ) { - super(exceptionClass, constructor, id, versionAdded); - OpenSearchExceptionHandleRegistry.registerExceptionHandle(this); - } - } - - /** - * Returns an array of all registered handle IDs. These are the IDs for every registered - * exception. - * - * @return an array of all registered handle IDs - */ - static int[] ids() { - return OpenSearchExceptionHandleRegistry.ids().stream().mapToInt(i -> i).toArray(); - } - - /** - * Returns an array of all registered pairs of handle IDs and exception classes. These pairs are - * provided for every registered exception. - * - * @return an array of all registered pairs of handle IDs and exception classes - */ - static Tuple>[] classes() { - final Tuple>[] ts = OpenSearchExceptionHandleRegistry.handles() - .stream() - .map(h -> Tuple.tuple(h.id, h.exceptionClass)) - .toArray(Tuple[]::new); - return ts; - } - - public Index getIndex() { - List index = getMetadata(INDEX_METADATA_KEY); - if (index != null && index.isEmpty() == false) { - List index_uuid = getMetadata(INDEX_METADATA_KEY_UUID); - return new Index(index.get(0), index_uuid.get(0)); - } - - return null; - } - - public ShardId getShardId() { - List shard = getMetadata(SHARD_METADATA_KEY); - if (shard != null && shard.isEmpty() == false) { - return new ShardId(getIndex(), Integer.parseInt(shard.get(0))); - } - return null; - } - - public void setIndex(Index index) { - if (index != null) { - addMetadata(INDEX_METADATA_KEY, index.getName()); - addMetadata(INDEX_METADATA_KEY_UUID, index.getUUID()); - } - } - - public void setIndex(String index) { - if (index != null) { - setIndex(new Index(index, INDEX_UUID_NA_VALUE)); - } - } - - public void setShard(ShardId shardId) { - if (shardId != null) { - setIndex(shardId.getIndex()); - addMetadata(SHARD_METADATA_KEY, Integer.toString(shardId.id())); - } - } - } diff --git a/server/src/main/java/org/opensearch/action/TaskOperationFailure.java b/server/src/main/java/org/opensearch/action/TaskOperationFailure.java index e1cf77fb3d530..25e1ab1bb3d6f 100644 --- a/server/src/main/java/org/opensearch/action/TaskOperationFailure.java +++ b/server/src/main/java/org/opensearch/action/TaskOperationFailure.java @@ -32,9 +32,8 @@ package org.opensearch.action; -import org.opensearch.BaseExceptionsHelper; -import org.opensearch.OpenSearchException; import org.opensearch.ExceptionsHelper; +import org.opensearch.OpenSearchException; import org.opensearch.core.ParseField; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; @@ -145,7 +144,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (reason != null) { builder.field(REASON); builder.startObject(); - BaseExceptionsHelper.generateThrowableXContent(builder, params, reason); + OpenSearchException.generateThrowableXContent(builder, params, reason); builder.endObject(); } return builder; 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 874f6c9f13ac7..f6d9a5c7b6453 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.BaseExceptionsHelper; +import org.opensearch.OpenSearchException; 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"); - BaseExceptionsHelper.generateThrowableXContent(builder, params, e); + OpenSearchException.generateThrowableXContent(builder, params, e); builder.endObject(); } builder.endObject(); 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 b7b589606561b..b806e78cd93ff 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 @@ -31,7 +31,7 @@ package org.opensearch.action.admin.indices.close; -import org.opensearch.BaseOpenSearchException; +import org.opensearch.OpenSearchException; 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 builder.field("closed", false); if (exception != null) { builder.startObject("exception"); - BaseOpenSearchException.generateFailureXContent(builder, params, exception, true); + OpenSearchException.generateFailureXContent(builder, params, exception, true); builder.endObject(); } else { builder.startObject("failedShards"); 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 6d31dfb98b7c8..6d89170fefd7c 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.BaseOpenSearchException; +import org.opensearch.OpenSearchException; 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"); - BaseOpenSearchException.generateFailureXContent(builder, params, exception, true); + OpenSearchException.generateFailureXContent(builder, params, exception, true); builder.endObject(); } else { builder.startArray("failed_shards"); 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 6d1525c6b671b..01321bb09323d 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,7 +33,7 @@ package org.opensearch.action.admin.indices.shards; import com.carrotsearch.hppc.cursors.IntObjectCursor; -import org.opensearch.BaseExceptionsHelper; +import org.opensearch.OpenSearchException; import org.opensearch.action.ActionResponse; import org.opensearch.action.support.DefaultShardOperationFailedException; import org.opensearch.cluster.node.DiscoveryNode; @@ -202,7 +202,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(Fields.ALLOCATED, allocationStatus.value()); if (storeException != null) { builder.startObject(Fields.STORE_EXCEPTION); - BaseExceptionsHelper.generateThrowableXContent(builder, params, storeException); + OpenSearchException.generateThrowableXContent(builder, params, storeException); builder.endObject(); } return builder; 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 e0bc2ae5d0e18..c712a18e85013 100644 --- a/server/src/main/java/org/opensearch/action/bulk/BulkItemResponse.java +++ b/server/src/main/java/org/opensearch/action/bulk/BulkItemResponse.java @@ -32,7 +32,6 @@ package org.opensearch.action.bulk; -import org.opensearch.BaseExceptionsHelper; import org.opensearch.ExceptionsHelper; import org.opensearch.OpenSearchException; import org.opensearch.Version; @@ -95,7 +94,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(_ID, failure.getId()); builder.field(STATUS, failure.getStatus().getStatus()); builder.startObject(ERROR); - BaseExceptionsHelper.generateThrowableXContent(builder, params, failure.getCause()); + OpenSearchException.generateThrowableXContent(builder, params, failure.getCause()); builder.endObject(); } builder.endObject(); @@ -357,7 +356,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(ID_FIELD, id); } builder.startObject(CAUSE_FIELD); - BaseExceptionsHelper.generateThrowableXContent(builder, params, cause); + OpenSearchException.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/TransportBulkAction.java b/server/src/main/java/org/opensearch/action/bulk/TransportBulkAction.java index fe9d0f21e01b2..ed407550b8b2f 100644 --- a/server/src/main/java/org/opensearch/action/bulk/TransportBulkAction.java +++ b/server/src/main/java/org/opensearch/action/bulk/TransportBulkAction.java @@ -35,7 +35,7 @@ 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.ExceptionsHelper; import org.opensearch.core.Assertions; import org.opensearch.OpenSearchParseException; import org.opensearch.ResourceAlreadyExistsException; @@ -325,7 +325,7 @@ protected void doRun() { @Override public void onFailure(Exception e) { - if (!(BaseExceptionsHelper.unwrapCause(e) instanceof ResourceAlreadyExistsException)) { + if (!(ExceptionsHelper.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 ff9226e5e6317..cbb30714ee8e1 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.BaseExceptionsHelper; +import org.opensearch.ExceptionsHelper; 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 BaseExceptionsHelper.unwrapCause(e) instanceof VersionConflictEngineException; + return ExceptionsHelper.unwrapCause(e) instanceof VersionConflictEngineException; } /** 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 3e2aa1316a7cc..88f78d2ca6815 100644 --- a/server/src/main/java/org/opensearch/action/get/MultiGetResponse.java +++ b/server/src/main/java/org/opensearch/action/get/MultiGetResponse.java @@ -32,7 +32,6 @@ package org.opensearch.action.get; -import org.opensearch.BaseOpenSearchException; import org.opensearch.OpenSearchException; import org.opensearch.Version; import org.opensearch.action.ActionResponse; @@ -131,7 +130,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.startObject(); builder.field(INDEX.getPreferredName(), index); builder.field(ID.getPreferredName(), id); - BaseOpenSearchException.generateFailureXContent(builder, params, exception, true); + OpenSearchException.generateFailureXContent(builder, params, exception, true); builder.endObject(); return builder; } 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 0fdd340c420b3..db88214cff0ec 100644 --- a/server/src/main/java/org/opensearch/action/ingest/SimulateDocumentBaseResult.java +++ b/server/src/main/java/org/opensearch/action/ingest/SimulateDocumentBaseResult.java @@ -31,7 +31,6 @@ package org.opensearch.action.ingest; -import org.opensearch.BaseOpenSearchException; import org.opensearch.OpenSearchException; import org.opensearch.core.ParseField; import org.opensearch.common.io.stream.StreamInput; @@ -126,7 +125,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (failure == null) { ingestDocument.toXContent(builder, params); } else { - BaseOpenSearchException.generateFailureXContent(builder, params, failure, true); + OpenSearchException.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 badddd5152bc9..509705fe856e8 100644 --- a/server/src/main/java/org/opensearch/action/ingest/SimulateProcessorResult.java +++ b/server/src/main/java/org/opensearch/action/ingest/SimulateProcessorResult.java @@ -31,7 +31,6 @@ package org.opensearch.action.ingest; -import org.opensearch.BaseOpenSearchException; import org.opensearch.OpenSearchException; import org.opensearch.core.ParseField; import org.opensearch.common.collect.Tuple; @@ -270,10 +269,10 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (failure != null && ingestDocument != null) { builder.startObject(IGNORED_ERROR_FIELD); - BaseOpenSearchException.generateFailureXContent(builder, params, failure, true); + OpenSearchException.generateFailureXContent(builder, params, failure, true); builder.endObject(); } else if (failure != null) { - BaseOpenSearchException.generateFailureXContent(builder, params, failure, true); + OpenSearchException.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 89869582910e5..969e0edbbc9d6 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 - : BaseOpenSearchException.guessRootCauses(shardSearchFailures[0].getCause())[0]; + : OpenSearchException.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 = BaseOpenSearchException.guessRootCauses(shardSearchFailures[0].getCause())[0]; + Throwable cause = OpenSearchException.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/MultiSearchResponse.java b/server/src/main/java/org/opensearch/action/search/MultiSearchResponse.java index d3c4dcab79d7f..b6c30b6be00c9 100644 --- a/server/src/main/java/org/opensearch/action/search/MultiSearchResponse.java +++ b/server/src/main/java/org/opensearch/action/search/MultiSearchResponse.java @@ -32,9 +32,8 @@ package org.opensearch.action.search; -import org.opensearch.BaseOpenSearchException; -import org.opensearch.OpenSearchException; import org.opensearch.ExceptionsHelper; +import org.opensearch.OpenSearchException; import org.opensearch.action.ActionResponse; import org.opensearch.common.Nullable; import org.opensearch.common.Strings; @@ -192,7 +191,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws for (Item item : items) { builder.startObject(); if (item.isFailure()) { - BaseOpenSearchException.generateFailureXContent(builder, params, item.getFailure(), true); + OpenSearchException.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/SearchPhaseExecutionException.java b/server/src/main/java/org/opensearch/action/search/SearchPhaseExecutionException.java index 891d9a1e2e924..65c876cbd7c61 100644 --- a/server/src/main/java/org/opensearch/action/search/SearchPhaseExecutionException.java +++ b/server/src/main/java/org/opensearch/action/search/SearchPhaseExecutionException.java @@ -32,8 +32,6 @@ 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; @@ -123,7 +121,7 @@ public Throwable getCause() { Throwable cause = super.getCause(); if (cause == null) { // fall back to guessed root cause - for (BaseOpenSearchException rootCause : guessRootCauses()) { + for (OpenSearchException rootCause : guessRootCauses()) { return rootCause; } } @@ -161,14 +159,14 @@ protected void metadataToXContent(XContentBuilder builder, Params params) throws @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - Throwable ex = BaseExceptionsHelper.unwrapCause(this); + Throwable ex = ExceptionsHelper.unwrapCause(this); if (ex != this) { - BaseExceptionsHelper.generateThrowableXContent(builder, params, this); + OpenSearchException.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") - BaseExceptionsHelper.innerToXContent( + OpenSearchException.innerToXContent( builder, params, this, @@ -183,14 +181,14 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws } @Override - public BaseOpenSearchException[] guessRootCauses() { + public OpenSearchException[] guessRootCauses() { ShardOperationFailedException[] failures = ExceptionsHelper.groupBy(shardFailures); - List rootCauses = new ArrayList<>(failures.length); + List rootCauses = new ArrayList<>(failures.length); for (ShardOperationFailedException failure : failures) { - BaseOpenSearchException[] guessRootCauses = BaseOpenSearchException.guessRootCauses(failure.getCause()); + OpenSearchException[] guessRootCauses = OpenSearchException.guessRootCauses(failure.getCause()); rootCauses.addAll(Arrays.asList(guessRootCauses)); } - return rootCauses.toArray(new BaseOpenSearchException[0]); + return rootCauses.toArray(new OpenSearchException[0]); } @Override 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 c820b7ada427a..891f7f3119541 100644 --- a/server/src/main/java/org/opensearch/action/search/ShardSearchFailure.java +++ b/server/src/main/java/org/opensearch/action/search/ShardSearchFailure.java @@ -32,7 +32,6 @@ package org.opensearch.action.search; -import org.opensearch.BaseExceptionsHelper; import org.opensearch.ExceptionsHelper; import org.opensearch.OpenSearchException; import org.opensearch.action.OriginalIndices; @@ -86,14 +85,14 @@ public ShardSearchFailure(Exception e) { } public ShardSearchFailure(Exception e, @Nullable SearchShardTarget shardTarget) { - this(e, BaseExceptionsHelper.unwrapCause(e), shardTarget); + this(e, ExceptionsHelper.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(), - BaseExceptionsHelper.detailedMessage(e), + ExceptionsHelper.detailedMessage(e), ExceptionsHelper.status(unwrappedCause), unwrappedCause ); @@ -120,7 +119,7 @@ public String toString() { + "], reason [" + reason + "], cause [" - + (cause == null ? "_na" : BaseExceptionsHelper.stackTrace(cause)) + + (cause == null ? "_na" : ExceptionsHelper.stackTrace(cause)) + "]"; } @@ -148,7 +147,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws } builder.field(REASON_FIELD); builder.startObject(); - BaseExceptionsHelper.generateThrowableXContent(builder, params, cause); + OpenSearchException.generateThrowableXContent(builder, params, cause); builder.endObject(); } builder.endObject(); 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 62059ca3f0754..03e7509b3b8e3 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.BaseExceptionsHelper; +import org.opensearch.ExceptionsHelper; 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 = BaseExceptionsHelper.unwrapCause(e); + final Throwable actual = ExceptionsHelper.unwrapCause(e); return (actual instanceof ShardNotFoundException || actual instanceof IndexNotFoundException || actual instanceof IllegalIndexShardStateException 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 3cec823422eb2..a7c7a799883a6 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,9 +34,8 @@ 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.core.Assertions; import org.opensearch.OpenSearchException; import org.opensearch.action.ActionListener; import org.opensearch.action.UnavailableShardsException; @@ -334,7 +333,7 @@ public void onFinished() { @Override public boolean shouldRetry(Exception e) { - final Throwable cause = BaseExceptionsHelper.unwrapCause(e); + final Throwable cause = ExceptionsHelper.unwrapCause(e); return cause instanceof CircuitBreakingException || cause instanceof OpenSearchRejectedExecutionException || cause instanceof ConnectTransportException; @@ -359,7 +358,7 @@ private void updateCheckPoints(ShardRouting shard, LongSupplier localCheckpointS } private void onNoLongerPrimary(Exception failure) { - final Throwable cause = BaseExceptionsHelper.unwrapCause(failure); + final Throwable cause = ExceptionsHelper.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 1b3b72d45b5b5..6f8f93ccc0ce3 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.BaseExceptionsHelper; +import org.opensearch.ExceptionsHelper; 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(), BaseExceptionsHelper.detailedMessage(cause), status, cause); + super(shardId.getIndexName(), shardId.getId(), ExceptionsHelper.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(); - BaseExceptionsHelper.generateThrowableXContent(builder, params, cause); + OpenSearchException.generateThrowableXContent(builder, params, cause); builder.endObject(); builder.field(STATUS, status); builder.field(PRIMARY, primary); 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 f1e05592731a5..ff84ceb22e043 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.BaseOpenSearchException; +import org.opensearch.OpenSearchException; 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()); - BaseOpenSearchException.generateFailureXContent(builder, params, failure.getCause(), true); + OpenSearchException.generateFailureXContent(builder, params, failure.getCause(), true); builder.endObject(); } else { TermVectorsResponse getResponse = response.getResponse(); 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 853690fa1e242..002c5fd3b89db 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,7 +35,6 @@ 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; @@ -630,7 +629,7 @@ public String toString() { components.add("primary term [" + primaryTerm + "]"); components.add("message [" + message + "]"); if (failure != null) { - components.add("failure [" + BaseExceptionsHelper.detailedMessage(failure) + "]"); + components.add("failure [" + ExceptionsHelper.detailedMessage(failure) + "]"); } components.add("markAsStale [" + markAsStale + "]"); return String.join(", ", components); 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 4e48441c7a39a..45fdf7ca641de 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.BaseExceptionsHelper; +import org.opensearch.ExceptionsHelper; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.routing.allocation.RoutingAllocation; @@ -390,7 +390,7 @@ public String getDetails() { if (message == null) { return null; } - return message + (failure == null ? "" : ", failure " + BaseExceptionsHelper.detailedMessage(failure)); + return message + (failure == null ? "" : ", failure " + ExceptionsHelper.detailedMessage(failure)); } /** 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 d08dbba3a249d..61ca0f50e93b9 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.BaseExceptionsHelper; +import org.opensearch.ExceptionsHelper; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.common.Nullable; @@ -62,7 +62,7 @@ public String toString() { + ", message [" + message + "], failure [" - + (failure == null ? "null" : BaseExceptionsHelper.detailedMessage(failure)) + + (failure == null ? "null" : ExceptionsHelper.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 ab1d56cbf7a9e..d50a474838a6b 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.BaseExceptionsHelper; +import org.opensearch.OpenSearchException; 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"); - BaseExceptionsHelper.generateThrowableXContent(builder, params, storeException); + OpenSearchException.generateThrowableXContent(builder, params, storeException); builder.endObject(); } } diff --git a/server/src/main/java/org/opensearch/common/Strings.java b/server/src/main/java/org/opensearch/common/Strings.java index 0bec840a15f40..15a4b17252450 100644 --- a/server/src/main/java/org/opensearch/common/Strings.java +++ b/server/src/main/java/org/opensearch/common/Strings.java @@ -33,7 +33,7 @@ package org.opensearch.common; import org.apache.lucene.util.BytesRefBuilder; -import org.opensearch.BaseExceptionsHelper; +import org.opensearch.ExceptionsHelper; import org.opensearch.OpenSearchException; import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.util.CollectionUtils; @@ -280,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", BaseExceptionsHelper.stackTrace(e)); + builder.field("stack_trace", ExceptionsHelper.stackTrace(e)); builder.endObject(); return toString(builder); } catch (IOException e2) { diff --git a/server/src/main/java/org/opensearch/common/io/Streams.java b/server/src/main/java/org/opensearch/common/io/Streams.java index 00e9c84e4c2aa..6e44f18b7f4d5 100644 --- a/server/src/main/java/org/opensearch/common/io/Streams.java +++ b/server/src/main/java/org/opensearch/common/io/Streams.java @@ -162,36 +162,9 @@ public static String copyToString(Reader in) throws IOException { return out.toString(); } - public static int readFully(Reader reader, char[] dest) throws IOException { - return readFully(reader, dest, 0, dest.length); - } - - public static int readFully(Reader reader, char[] dest, int offset, int len) throws IOException { - int read = 0; - while (read < len) { - final int r = reader.read(dest, offset + read, len - read); - if (r == -1) { - break; - } - read += r; - } - return read; - } - + @Deprecated public static int readFully(InputStream reader, byte[] dest) throws IOException { - return readFully(reader, dest, 0, dest.length); - } - - public static int readFully(InputStream reader, byte[] dest, int offset, int len) throws IOException { - int read = 0; - while (read < len) { - final int r = reader.read(dest, offset + read, len - read); - if (r == -1) { - break; - } - read += r; - } - return read; + return reader.readNBytes(dest, 0, dest.length); } /** diff --git a/server/src/main/java/org/opensearch/common/io/stream/Streamables.java b/server/src/main/java/org/opensearch/common/io/stream/Streamables.java index abaaa020cfd5b..e594247be708e 100644 --- a/server/src/main/java/org/opensearch/common/io/stream/Streamables.java +++ b/server/src/main/java/org/opensearch/common/io/stream/Streamables.java @@ -12,8 +12,7 @@ import org.joda.time.ReadableInstant; import org.opensearch.common.geo.GeoPoint; import org.opensearch.common.time.DateUtils; -import org.opensearch.core.common.io.stream.BaseWriteable.WriteableRegistry; -import org.opensearch.core.common.io.stream.BaseWriteable; +import org.opensearch.common.io.stream.Writeable.WriteableRegistry; import org.opensearch.script.JodaCompatibleZonedDateTime; import java.time.Instant; @@ -47,7 +46,7 @@ public static void registerStreamables() { */ private static void registerWriters() { /** {@link ReadableInstant} */ - WriteableRegistry.>registerWriter(ReadableInstant.class, (o, v) -> { + WriteableRegistry.registerWriter(ReadableInstant.class, (o, v) -> { o.writeByte((byte) 13); final ReadableInstant instant = (ReadableInstant) v; o.writeString(instant.getZone().getID()); @@ -55,7 +54,7 @@ private static void registerWriters() { }); WriteableRegistry.registerClassAlias(ReadableInstant.class, ReadableInstant.class); /** {@link JodaCompatibleZonedDateTime} */ - WriteableRegistry.>registerWriter(JodaCompatibleZonedDateTime.class, (o, v) -> { + WriteableRegistry.registerWriter(JodaCompatibleZonedDateTime.class, (o, v) -> { // write the joda compatibility datetime as joda datetime o.writeByte((byte) 13); final JodaCompatibleZonedDateTime zonedDateTime = (JodaCompatibleZonedDateTime) v; @@ -65,7 +64,7 @@ private static void registerWriters() { o.writeLong(zonedDateTime.toInstant().toEpochMilli()); }); /** {@link GeoPoint} */ - BaseWriteable.WriteableRegistry.>registerWriter(GeoPoint.class, (o, v) -> { + WriteableRegistry.registerWriter(GeoPoint.class, (o, v) -> { o.writeByte((byte) 22); ((GeoPoint) v).writeTo(o); }); @@ -78,12 +77,12 @@ private static void registerWriters() { */ private static void registerReaders() { /** {@link JodaCompatibleZonedDateTime */ - WriteableRegistry.>registerReader(Byte.valueOf((byte) 13), (i) -> { + WriteableRegistry.registerReader(Byte.valueOf((byte) 13), (i) -> { final ZoneId zoneId = DateUtils.dateTimeZoneToZoneId(DateTimeZone.forID(i.readString())); long millis = i.readLong(); return new JodaCompatibleZonedDateTime(Instant.ofEpochMilli(millis), zoneId); }); /** {@link GeoPoint} */ - WriteableRegistry.>registerReader(Byte.valueOf((byte) 22), GeoPoint::new); + WriteableRegistry.registerReader(Byte.valueOf((byte) 22), GeoPoint::new); } } diff --git a/server/src/main/java/org/opensearch/common/io/stream/Writeable.java b/server/src/main/java/org/opensearch/common/io/stream/Writeable.java deleted file mode 100644 index c04cd7977fdc0..0000000000000 --- a/server/src/main/java/org/opensearch/common/io/stream/Writeable.java +++ /dev/null @@ -1,88 +0,0 @@ -/* - * 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.common.io.stream; - -import org.opensearch.core.common.io.stream.BaseWriteable; - -import java.io.IOException; - -/** - * Implementers can be written to a {@linkplain StreamOutput} and read from a {@linkplain StreamInput}. This allows them to be "thrown - * 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 - */ -public interface Writeable extends BaseWriteable { - - /** - * Write this into the {@linkplain StreamOutput}. - */ - void writeTo(StreamOutput out) throws IOException; - - /** - * Reference to a method that can write some object to a {@link StreamOutput}. - *

- * By convention this is a method from {@link StreamOutput} itself (e.g., {@link StreamOutput#writeString}). If the value can be - * {@code null}, then the "optional" variant of methods should be used! - *

- * Most classes should implement {@link Writeable} and the {@link Writeable#writeTo(StreamOutput)} method should use - * {@link StreamOutput} methods directly or this indirectly: - *


-     * public void writeTo(StreamOutput out) throws IOException {
-     *     out.writeVInt(someValue);
-     *     out.writeMapOfLists(someMap, StreamOutput::writeString, StreamOutput::writeString);
-     * }
-     * 
- */ - @FunctionalInterface - interface Writer extends BaseWriteable.Writer {} - - /** - * Reference to a method that can read some object from a stream. By convention this is a constructor that takes - * {@linkplain StreamInput} as an argument for most classes and a static method for things like enums. Returning null from one of these - * is always wrong - for that we use methods like {@link StreamInput#readOptionalWriteable(Reader)}. - *

- * As most classes will implement this via a constructor (or a static method in the case of enumerations), it's something that should - * look like: - *


-     * public MyClass(final StreamInput in) throws IOException {
-     *     this.someValue = in.readVInt();
-     *     this.someMap = in.readMapOfLists(StreamInput::readString, StreamInput::readString);
-     * }
-     * 
- */ - @FunctionalInterface - interface Reader extends BaseWriteable.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 13a9551d9e421..125eab9512be8 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.BaseExceptionsHelper; +import org.opensearch.OpenSearchException; 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 = BaseExceptionsHelper.getExceptionName(e) + ":[" + e.getMessage() + "]"; + String message = OpenSearchException.getExceptionName(e) + ":[" + e.getMessage() + "]"; return Queries.newMatchNoDocsQuery("failed [" + field + "] query, caused by " + message); } diff --git a/server/src/main/java/org/opensearch/common/util/PageCacheRecycler.java b/server/src/main/java/org/opensearch/common/util/PageCacheRecycler.java index 6d786e85bab1c..429dce6a69e75 100644 --- a/server/src/main/java/org/opensearch/common/util/PageCacheRecycler.java +++ b/server/src/main/java/org/opensearch/common/util/PageCacheRecycler.java @@ -33,6 +33,7 @@ package org.opensearch.common.util; import org.apache.lucene.util.RamUsageEstimator; +import org.opensearch.common.bytes.PagedBytesReference; import org.opensearch.common.recycler.AbstractRecyclerC; import org.opensearch.common.recycler.Recycler; import org.opensearch.common.settings.Setting; @@ -94,7 +95,7 @@ public class PageCacheRecycler { ); /** Page size in bytes: 16KB */ - public static final int PAGE_SIZE_IN_BYTES = 1 << 14; + public static final int PAGE_SIZE_IN_BYTES = PagedBytesReference.PAGE_SIZE_IN_BYTES; public static final int OBJECT_PAGE_SIZE = PAGE_SIZE_IN_BYTES / RamUsageEstimator.NUM_BYTES_OBJECT_REF; public static final int LONG_PAGE_SIZE = PAGE_SIZE_IN_BYTES / Long.BYTES; public static final int INT_PAGE_SIZE = PAGE_SIZE_IN_BYTES / Integer.BYTES; diff --git a/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java b/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java index 35272d9f54dc6..e4df2e604c320 100644 --- a/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java +++ b/server/src/main/java/org/opensearch/gateway/AsyncShardFetch.java @@ -33,7 +33,7 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; -import org.opensearch.BaseExceptionsHelper; +import org.opensearch.ExceptionsHelper; import org.opensearch.OpenSearchTimeoutException; import org.opensearch.action.ActionListener; import org.opensearch.action.FailedNodeException; @@ -251,7 +251,7 @@ protected synchronized void processAsyncFetch(List responses, List searchLookup) { throw new IllegalArgumentException("Fielddata is not supported on field [" + name() + "] of type [" + typeName() + "]"); @@ -411,7 +412,7 @@ public Relation isFieldWithinQuery( /** @throws IllegalArgumentException if the fielddata is not supported on this type. * An IllegalArgumentException is needed in order to return an http error 400 - * when this error occurs in a request. see: {@link org.opensearch.ExceptionsHelper#status} + * when this error occurs in a request. see: {@link ExceptionsHelper#status} **/ protected final void failIfNoDocValues() { if (hasDocValues() == false) { 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 cc5cae168e7b6..ab711d3a0ea00 100644 --- a/server/src/main/java/org/opensearch/index/query/MoreLikeThisQueryBuilder.java +++ b/server/src/main/java/org/opensearch/index/query/MoreLikeThisQueryBuilder.java @@ -37,7 +37,6 @@ 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; @@ -449,7 +448,7 @@ public String toString() { toXContent(builder, EMPTY_PARAMS); return Strings.toString(builder); } catch (Exception e) { - return "{ \"error\" : \"" + BaseExceptionsHelper.detailedMessage(e) + "\"}"; + return "{ \"error\" : \"" + ExceptionsHelper.detailedMessage(e) + "\"}"; } } 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 be9cade4925eb..2fe0fe0259c33 100644 --- a/server/src/main/java/org/opensearch/index/reindex/BulkByScrollTask.java +++ b/server/src/main/java/org/opensearch/index/reindex/BulkByScrollTask.java @@ -32,7 +32,6 @@ package org.opensearch.index.reindex; -import org.opensearch.BaseExceptionsHelper; import org.opensearch.OpenSearchException; import org.opensearch.common.Nullable; import org.opensearch.core.ParseField; @@ -1000,7 +999,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws status.toXContent(builder, params); } else { builder.startObject(); - BaseExceptionsHelper.generateThrowableXContent(builder, params, exception); + OpenSearchException.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 ffd4bfb86afa5..90f7796bf1fdc 100644 --- a/server/src/main/java/org/opensearch/index/reindex/ReindexRequest.java +++ b/server/src/main/java/org/opensearch/index/reindex/ReindexRequest.java @@ -36,8 +36,8 @@ import org.opensearch.action.CompositeIndicesRequest; import org.opensearch.action.index.IndexRequest; import org.opensearch.action.search.SearchRequest; -import org.opensearch.core.ParseField; import org.opensearch.common.bytes.BytesReference; +import org.opensearch.core.ParseField; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.logging.DeprecationLogger; 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 8628bd84da35a..81560de78b336 100644 --- a/server/src/main/java/org/opensearch/index/reindex/ScrollableHitSource.java +++ b/server/src/main/java/org/opensearch/index/reindex/ScrollableHitSource.java @@ -33,8 +33,8 @@ package org.opensearch.index.reindex; import org.apache.logging.log4j.Logger; -import org.opensearch.BaseExceptionsHelper; import org.opensearch.ExceptionsHelper; +import org.opensearch.OpenSearchException; import org.opensearch.action.ActionListener; import org.opensearch.action.bulk.BackoffPolicy; import org.opensearch.action.bulk.BulkItemResponse; @@ -489,7 +489,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.field(REASON_FIELD); { builder.startObject(); - BaseExceptionsHelper.generateThrowableXContent(builder, params, reason); + OpenSearchException.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 02397bc356539..a81dc96ff1145 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.BaseExceptionsHelper; +import org.opensearch.ExceptionsHelper; import org.opensearch.action.ActionListener; import org.opensearch.action.StepListener; import org.opensearch.cluster.metadata.IndexMetadata; @@ -500,7 +500,7 @@ private void internalRecoverFromStore(IndexShard indexShard) throws IndexShardRe files = Arrays.toString(store.directory().listAll()); } catch (Exception inner) { inner.addSuppressed(e); - files += " (failure=" + BaseExceptionsHelper.detailedMessage(inner) + ")"; + files += " (failure=" + ExceptionsHelper.detailedMessage(inner) + ")"; } if (indexShouldExists) { throw new IndexShardRecoveryException( diff --git a/server/src/main/java/org/opensearch/index/store/Store.java b/server/src/main/java/org/opensearch/index/store/Store.java index 2c0d5decebba8..90832b4c77756 100644 --- a/server/src/main/java/org/opensearch/index/store/Store.java +++ b/server/src/main/java/org/opensearch/index/store/Store.java @@ -67,7 +67,6 @@ import org.opensearch.common.Nullable; import org.opensearch.common.UUIDs; import org.opensearch.common.bytes.BytesReference; -import org.opensearch.common.io.Streams; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; @@ -1240,7 +1239,7 @@ public static void hashFile(BytesRefBuilder fileHash, InputStream in, long size) final int len = (int) Math.min(1024 * 1024, size); // for safety we limit this to 1MB fileHash.grow(len); fileHash.setLength(len); - final int readBytes = Streams.readFully(in, fileHash.bytes(), 0, len); + final int readBytes = in.readNBytes(fileHash.bytes(), 0, len); assert readBytes == len : Integer.toString(readBytes) + " != " + Integer.toString(len); assert fileHash.length() == len : Integer.toString(fileHash.length()) + " != " + Integer.toString(len); } 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 a08d3182fa156..17af529a700d9 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.BaseExceptionsHelper; +import org.opensearch.ExceptionsHelper; import org.opensearch.OpenSearchException; import org.opensearch.OpenSearchTimeoutException; import org.opensearch.action.ActionListener; @@ -679,7 +679,7 @@ private void onException(Exception e) { e ); } - Throwable cause = BaseExceptionsHelper.unwrapCause(e); + Throwable cause = ExceptionsHelper.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); @@ -690,7 +690,7 @@ private void onException(Exception e) { cause = cause.getCause(); } // do it twice, in case we have double transport exception - cause = BaseExceptionsHelper.unwrapCause(cause); + cause = ExceptionsHelper.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 f7d75b356d6a3..77ffd75d0148a 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.BaseExceptionsHelper; +import org.opensearch.ExceptionsHelper; import org.opensearch.action.ActionListener; import org.opensearch.action.ActionListenerResponseHandler; import org.opensearch.action.support.RetryableAction; @@ -127,10 +127,10 @@ private static boolean retryableException(Exception e) { if (e instanceof ConnectTransportException) { return true; } else if (e instanceof SendRequestTransportException) { - final Throwable cause = BaseExceptionsHelper.unwrapCause(e); + final Throwable cause = ExceptionsHelper.unwrapCause(e); return cause instanceof ConnectTransportException; } else if (e instanceof RemoteTransportException) { - final Throwable cause = BaseExceptionsHelper.unwrapCause(e); + final Throwable cause = ExceptionsHelper.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 2931caa353bc6..a7e0c0ec887ab 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.BaseExceptionsHelper; +import org.opensearch.ExceptionsHelper; 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 = BaseExceptionsHelper.unwrapCause(e); + Throwable cause = ExceptionsHelper.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 415fb91874416..4d75ff4896706 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.BaseExceptionsHelper; +import org.opensearch.ExceptionsHelper; 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() + "[" + BaseExceptionsHelper.stackTrace(e) + "]"); + cancellableThreads.cancel("failed" + description() + "[" + ExceptionsHelper.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/rest/BytesRestResponse.java b/server/src/main/java/org/opensearch/rest/BytesRestResponse.java index a5af9226ad831..3bb094bcd4ba0 100644 --- a/server/src/main/java/org/opensearch/rest/BytesRestResponse.java +++ b/server/src/main/java/org/opensearch/rest/BytesRestResponse.java @@ -36,11 +36,9 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.logging.log4j.util.Supplier; -import org.opensearch.BaseExceptionsHelper; -import org.opensearch.BaseOpenSearchException; +import org.opensearch.ExceptionsHelper; import org.opensearch.OpenSearchException; import org.opensearch.OpenSearchStatusException; -import org.opensearch.ExceptionsHelper; import org.opensearch.common.bytes.BytesArray; import org.opensearch.common.bytes.BytesReference; import org.opensearch.core.xcontent.ToXContent; @@ -113,8 +111,8 @@ public BytesRestResponse(RestChannel channel, Exception e) throws IOException { public BytesRestResponse(RestChannel channel, RestStatus status, Exception e) throws IOException { ToXContent.Params params = paramsFromRequest(channel.request()); if (params.paramAsBoolean( - BaseExceptionsHelper.REST_EXCEPTION_SKIP_STACK_TRACE, - BaseExceptionsHelper.REST_EXCEPTION_SKIP_STACK_TRACE_DEFAULT + OpenSearchException.REST_EXCEPTION_SKIP_STACK_TRACE, + OpenSearchException.REST_EXCEPTION_SKIP_STACK_TRACE_DEFAULT ) && e != null) { // log exception only if it is not returned in the response Supplier messageSupplier = () -> new ParameterizedMessage( @@ -156,12 +154,9 @@ public RestStatus status() { private ToXContent.Params paramsFromRequest(RestRequest restRequest) { ToXContent.Params params = restRequest; - if (params.paramAsBoolean("error_trace", !BaseExceptionsHelper.REST_EXCEPTION_SKIP_STACK_TRACE_DEFAULT) + if (params.paramAsBoolean("error_trace", OpenSearchException.REST_EXCEPTION_SKIP_STACK_TRACE_DEFAULT == false) && false == skipStackTrace()) { - params = new ToXContent.DelegatingMapParams( - singletonMap(BaseExceptionsHelper.REST_EXCEPTION_SKIP_STACK_TRACE, "false"), - params - ); + params = new ToXContent.DelegatingMapParams(singletonMap(OpenSearchException.REST_EXCEPTION_SKIP_STACK_TRACE, "false"), params); } return params; } @@ -173,7 +168,7 @@ protected boolean skipStackTrace() { private void build(XContentBuilder builder, ToXContent.Params params, RestStatus status, boolean detailedErrorsEnabled, Exception e) throws IOException { builder.startObject(); - BaseOpenSearchException.generateFailureXContent(builder, params, e, detailedErrorsEnabled); + OpenSearchException.generateFailureXContent(builder, params, e, detailedErrorsEnabled); builder.field(STATUS, status.getStatus()); builder.endObject(); } diff --git a/server/src/main/java/org/opensearch/script/Script.java b/server/src/main/java/org/opensearch/script/Script.java index 6580bfe8326f6..7f6c41575ccb8 100644 --- a/server/src/main/java/org/opensearch/script/Script.java +++ b/server/src/main/java/org/opensearch/script/Script.java @@ -33,10 +33,10 @@ package org.opensearch.script; import org.opensearch.OpenSearchParseException; +import org.opensearch.common.bytes.BytesReference; import org.opensearch.core.ParseField; 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.io.stream.Writeable; 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 dc75088d4c85b..afd46f1a8164c 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.BaseExceptionsHelper; +import org.opensearch.ExceptionsHelper; 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\" : \"" + BaseExceptionsHelper.detailedMessage(e) + "\"}"; + return "{ \"error\" : \"" + ExceptionsHelper.detailedMessage(e) + "\"}"; } } diff --git a/server/src/main/java/org/opensearch/tasks/TaskCancellationService.java b/server/src/main/java/org/opensearch/tasks/TaskCancellationService.java index 4368b5fe0ba3a..ebce26c4bbfbc 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.BaseExceptionsHelper; +import org.opensearch.ExceptionsHelper; import org.opensearch.OpenSearchSecurityException; import org.opensearch.action.ActionListener; import org.opensearch.action.StepListener; @@ -156,7 +156,7 @@ public void handleResponse(TransportResponse.Empty response) { @Override public void handleException(TransportException exp) { - assert BaseExceptionsHelper.unwrapCause(exp) instanceof OpenSearchSecurityException == false; + assert ExceptionsHelper.unwrapCause(exp) instanceof OpenSearchSecurityException == false; logger.warn("Cannot send ban for tasks with the parent [{}] to the node [{}]", taskId, node); groupedListener.onFailure(exp); } @@ -172,7 +172,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 BaseExceptionsHelper.unwrapCause(exp) instanceof OpenSearchSecurityException == false; + assert ExceptionsHelper.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/TaskManager.java b/server/src/main/java/org/opensearch/tasks/TaskManager.java index 443a6c0853884..f243cf392bbb8 100644 --- a/server/src/main/java/org/opensearch/tasks/TaskManager.java +++ b/server/src/main/java/org/opensearch/tasks/TaskManager.java @@ -37,7 +37,6 @@ 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; @@ -370,7 +369,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 {}", BaseExceptionsHelper.detailedMessage(error)), ex); + logger.warn(() -> new ParameterizedMessage("couldn't store error {}", ExceptionsHelper.detailedMessage(error)), ex); listener.onFailure(ex); return; } @@ -382,7 +381,7 @@ public void onResponse(Void aVoid) { @Override public void onFailure(Exception e) { - logger.warn(() -> new ParameterizedMessage("couldn't store error {}", BaseExceptionsHelper.detailedMessage(error)), e); + logger.warn(() -> new ParameterizedMessage("couldn't store error {}", ExceptionsHelper.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 11ea9a9a82861..2387de173c924 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.BaseExceptionsHelper; +import org.opensearch.OpenSearchException; 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(); - BaseExceptionsHelper.generateThrowableXContent(builder, ToXContent.EMPTY_PARAMS, error); + OpenSearchException.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 329fc55431bc8..1feb115cb585a 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.BaseExceptionsHelper; +import org.opensearch.ExceptionsHelper; 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 (BaseExceptionsHelper.unwrapCause(e) instanceof ResourceAlreadyExistsException) { + if (ExceptionsHelper.unwrapCause(e) instanceof ResourceAlreadyExistsException) { // we have the index, do it try { doStoreResult(taskResult, listener); diff --git a/server/src/main/java/org/opensearch/transport/TransportService.java b/server/src/main/java/org/opensearch/transport/TransportService.java index cb324035bcd62..f41112d69196b 100644 --- a/server/src/main/java/org/opensearch/transport/TransportService.java +++ b/server/src/main/java/org/opensearch/transport/TransportService.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.OpenSearchServerException; import org.opensearch.Version; import org.opensearch.action.ActionListener; import org.opensearch.action.ActionListenerResponseHandler; @@ -168,6 +169,7 @@ public void close() {} * over the {@link StreamOutput} and {@link StreamInput} wire */ Streamables.registerStreamables(); + OpenSearchServerException.registerExceptions(); } /** does nothing. easy way to ensure class is loaded so the above static block is called to register the streamables */ diff --git a/server/src/test/java/org/opensearch/ExceptionSerializationTests.java b/server/src/test/java/org/opensearch/ExceptionSerializationTests.java index a01ffc71b1675..8b0d6faeb1e61 100644 --- a/server/src/test/java/org/opensearch/ExceptionSerializationTests.java +++ b/server/src/test/java/org/opensearch/ExceptionSerializationTests.java @@ -161,7 +161,8 @@ public void testExceptionRegistration() throws ClassNotFoundException, IOExcepti final Set> hasDedicatedWrite = new HashSet<>(); final Set> registered = new HashSet<>(); final String path = "/org/opensearch"; - final Path startPath = PathUtils.get(OpenSearchException.class.getProtectionDomain().getCodeSource().getLocation().toURI()) + final Path coreLibStartPath = PathUtils.get(OpenSearchException.class.getProtectionDomain().getCodeSource().getLocation().toURI()); + final Path startPath = PathUtils.get(OpenSearchServerException.class.getProtectionDomain().getCodeSource().getLocation().toURI()) .resolve("org") .resolve("opensearch"); final Set ignore = Sets.newHashSet( @@ -243,6 +244,9 @@ public FileVisitResult postVisitDirectory(Path dir, IOException exc) throws IOEx } }; + // walk the core library start path + Files.walkFileTree(coreLibStartPath, visitor); + // walk the server module start path Files.walkFileTree(startPath, visitor); final Path testStartPath = PathUtils.get(ExceptionSerializationTests.class.getResource(path).toURI()); Files.walkFileTree(testStartPath, visitor); @@ -893,7 +897,7 @@ public void testIds() { } } - for (final Tuple> tuple : OpenSearchException.classes()) { + for (final Tuple> tuple : OpenSearchException.classes()) { assertNotNull(tuple.v1()); assertNotNull( tuple.v2().getName() + " not found in ExceptionSerializationTests.testIds. Please add it.", diff --git a/server/src/test/java/org/opensearch/ExceptionsHelperTests.java b/server/src/test/java/org/opensearch/ExceptionsHelperTests.java index 395c5ec14c8a3..4c65eadf46e89 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(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")); + 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")); } public void testGroupBy() { diff --git a/server/src/test/java/org/opensearch/OpenSearchExceptionTests.java b/server/src/test/java/org/opensearch/OpenSearchExceptionTests.java index d584e9130e20e..a0bd8202abbbe 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"))) ); - BaseOpenSearchException[] rootCauses = exception.guessRootCauses(); + OpenSearchException[] rootCauses = exception.guessRootCauses(); assertEquals(rootCauses.length, 1); - assertEquals(BaseExceptionsHelper.getExceptionName(rootCauses[0]), "index_not_found_exception"); + assertEquals(OpenSearchException.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 = BaseOpenSearchException.guessRootCauses(randomBoolean() ? new RemoteTransportException("remoteboom", ex) : ex); + rootCauses = OpenSearchException.guessRootCauses(randomBoolean() ? new RemoteTransportException("remoteboom", ex) : ex); } - assertEquals("parsing_exception", BaseExceptionsHelper.getExceptionName(rootCauses[0])); + assertEquals("parsing_exception", OpenSearchException.getExceptionName(rootCauses[0])); assertEquals("foobar", rootCauses[0].getMessage()); OpenSearchException oneLevel = new OpenSearchException("foo", new RuntimeException("foobar")); rootCauses = oneLevel.guessRootCauses(); - assertEquals("exception", BaseExceptionsHelper.getExceptionName(rootCauses[0])); + assertEquals("exception", OpenSearchException.getExceptionName(rootCauses[0])); assertEquals("foo", rootCauses[0].getMessage()); } { @@ -175,32 +175,32 @@ public void testGuessRootCause() { "all shards failed", new ShardSearchFailure[] { failure, failure1, failure2 } ); - final BaseOpenSearchException[] rootCauses = ex.guessRootCauses(); + final OpenSearchException[] rootCauses = ex.guessRootCauses(); assertEquals(rootCauses.length, 2); - assertEquals(BaseExceptionsHelper.getExceptionName(rootCauses[0]), "parsing_exception"); + assertEquals(OpenSearchException.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", BaseExceptionsHelper.getExceptionName(rootCauses[1])); - assertEquals("foo1", rootCauses[1].getIndexName()); + assertEquals("query_shard_exception", OpenSearchException.getExceptionName(rootCauses[1])); + assertEquals("foo1", rootCauses[1].getIndex().getName()); assertEquals("foobar", rootCauses[1].getMessage()); } { - final BaseOpenSearchException[] foobars = BaseOpenSearchException.guessRootCauses(new IllegalArgumentException("foobar")); + final OpenSearchException[] foobars = OpenSearchException.guessRootCauses(new IllegalArgumentException("foobar")); assertEquals(foobars.length, 1); - assertThat(foobars[0], instanceOf(BaseOpenSearchException.class)); + assertThat(foobars[0], instanceOf(OpenSearchException.class)); assertEquals("foobar", foobars[0].getMessage()); assertEquals(IllegalArgumentException.class, foobars[0].getCause().getClass()); assertEquals("illegal_argument_exception", foobars[0].getExceptionName()); } { - final BaseOpenSearchException[] foobars = BaseOpenSearchException.guessRootCauses( + final OpenSearchException[] foobars = OpenSearchException.guessRootCauses( new RemoteTransportException("abc", new IllegalArgumentException("foobar")) ); assertEquals(foobars.length, 1); - assertThat(foobars[0], instanceOf(BaseOpenSearchException.class)); + assertThat(foobars[0], instanceOf(OpenSearchException.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 BaseOpenSearchException[] causes = BaseOpenSearchException.guessRootCauses(outer); + final OpenSearchException[] causes = OpenSearchException.guessRootCauses(outer); assertEquals(causes.length, 1); - assertThat(causes[0], instanceOf(BaseOpenSearchException.class)); + assertThat(causes[0], instanceOf(OpenSearchException.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 BaseOpenSearchException[] causes = BaseOpenSearchException.guessRootCauses(outer); + final OpenSearchException[] causes = OpenSearchException.guessRootCauses(outer); assertEquals(causes.length, 1); - assertThat(causes[0], instanceOf(BaseOpenSearchException.class)); + assertThat(causes[0], instanceOf(OpenSearchException.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(BaseExceptionsHelper.REST_EXCEPTION_SKIP_STACK_TRACE, "false") + Collections.singletonMap(OpenSearchException.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) -> { - BaseExceptionsHelper.generateThrowableXContent(builder, params, ex); + OpenSearchException.generateThrowableXContent(builder, params, ex); return builder; }); @@ -740,7 +740,7 @@ public void testThrowableToAndFromXContent() throws IOException { } BytesReference throwableBytes = toShuffledXContent((builder, params) -> { - BaseExceptionsHelper.generateThrowableXContent(builder, params, throwable); + OpenSearchException.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(); - BaseExceptionsHelper.generateThrowableXContent(builder, ToXContent.EMPTY_PARAMS, throwable); + OpenSearchException.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() - BaseOpenSearchException.generateFailureXContent(builder, params, null, randomBoolean()); + OpenSearchException.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) -> { - BaseOpenSearchException.generateFailureXContent(builder, params, failure, false); + OpenSearchException.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 = BaseExceptionsHelper.summaryMessage(failure); - assertEquals(BaseOpenSearchException.buildMessage("exception", reason, null), parsedFailure.getMessage()); + String reason = ExceptionsHelper.summaryMessage(failure); + assertEquals(OpenSearchException.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) -> { - BaseOpenSearchException.generateFailureXContent(builder, params, finalFailure, true); + OpenSearchException.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) -> { - BaseExceptionsHelper.generateThrowableXContent(builder, params, e); + OpenSearchException.generateThrowableXContent(builder, params, e); return builder; }, expectedJson); } diff --git a/server/src/test/java/org/opensearch/action/bulk/BulkItemResponseTests.java b/server/src/test/java/org/opensearch/action/bulk/BulkItemResponseTests.java index 88c6b466059bc..3e5a7d5ad5242 100644 --- a/server/src/test/java/org/opensearch/action/bulk/BulkItemResponseTests.java +++ b/server/src/test/java/org/opensearch/action/bulk/BulkItemResponseTests.java @@ -32,8 +32,8 @@ package org.opensearch.action.bulk; -import org.opensearch.OpenSearchException; import org.opensearch.ExceptionsHelper; +import org.opensearch.OpenSearchException; import org.opensearch.action.DocWriteRequest; import org.opensearch.action.DocWriteResponse; import org.opensearch.action.bulk.BulkItemResponse.Failure; diff --git a/server/src/test/java/org/opensearch/action/bulk/BulkResponseTests.java b/server/src/test/java/org/opensearch/action/bulk/BulkResponseTests.java index ef930b61f890b..aca2419e2d325 100644 --- a/server/src/test/java/org/opensearch/action/bulk/BulkResponseTests.java +++ b/server/src/test/java/org/opensearch/action/bulk/BulkResponseTests.java @@ -32,8 +32,8 @@ package org.opensearch.action.bulk; -import org.opensearch.OpenSearchException; import org.opensearch.ExceptionsHelper; +import org.opensearch.OpenSearchException; import org.opensearch.action.DocWriteRequest; import org.opensearch.action.DocWriteResponse; import org.opensearch.action.delete.DeleteResponseTests; diff --git a/server/src/test/java/org/opensearch/action/search/MultiSearchRequestTests.java b/server/src/test/java/org/opensearch/action/search/MultiSearchRequestTests.java index e65101838884a..5795d182e622e 100644 --- a/server/src/test/java/org/opensearch/action/search/MultiSearchRequestTests.java +++ b/server/src/test/java/org/opensearch/action/search/MultiSearchRequestTests.java @@ -36,10 +36,10 @@ import org.opensearch.action.support.IndicesOptions; import org.opensearch.common.CheckedBiConsumer; import org.opensearch.common.CheckedRunnable; +import org.opensearch.common.bytes.BytesReference; import org.opensearch.core.ParseField; import org.opensearch.common.Strings; import org.opensearch.common.bytes.BytesArray; -import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.io.stream.NamedWriteableRegistry; import org.opensearch.common.logging.DeprecationLogger; import org.opensearch.common.unit.TimeValue; diff --git a/server/src/test/java/org/opensearch/client/AbstractClientHeadersTestCase.java b/server/src/test/java/org/opensearch/client/AbstractClientHeadersTestCase.java index f40227beb59b7..92a88aa7940ee 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.BaseExceptionsHelper; +import org.opensearch.ExceptionsHelper; 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: " + BaseExceptionsHelper.stackTrace(t)); + fail("Exception cause unwrapping ran for 10 levels: " + ExceptionsHelper.stackTrace(t)); return null; } result = result.getCause(); diff --git a/server/src/test/java/org/opensearch/common/io/stream/BytesStreamsTests.java b/server/src/test/java/org/opensearch/common/io/stream/BytesStreamsTests.java index 0af60b8065588..111354f2b14db 100644 --- a/server/src/test/java/org/opensearch/common/io/stream/BytesStreamsTests.java +++ b/server/src/test/java/org/opensearch/common/io/stream/BytesStreamsTests.java @@ -931,7 +931,7 @@ public void testWriteCircularReferenceException() throws IOException { BytesStreamOutput prodOut = new BytesStreamOutput() { @Override - boolean failOnTooManyNestedExceptions(Throwable throwable) { + public boolean failOnTooManyNestedExceptions(Throwable throwable) { assertThat(throwable, sameInstance(rootEx)); return true; } diff --git a/server/src/test/java/org/opensearch/rest/BytesRestResponseTests.java b/server/src/test/java/org/opensearch/rest/BytesRestResponseTests.java index 0e58e8ce719f6..a5d70886aa651 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.BaseExceptionsHelper; +import org.opensearch.ExceptionsHelper; import org.opensearch.OpenSearchStatusException; import org.opensearch.OpenSearchException; import org.opensearch.ResourceAlreadyExistsException; @@ -130,7 +130,7 @@ public void testErrorTrace() throws Exception { String text = response.content().utf8ToString(); assertThat(text, containsString("\"type\":\"unknown_exception\",\"reason\":\"an error occurred reading data\"")); assertThat(text, containsString("{\"type\":\"file_not_found_exception\"")); - assertThat(text, containsString("\"stack_trace\":\"[an error occurred reading data]")); + assertThat(text, containsString("\"stack_trace\":\"OpenSearchException[an error occurred reading data]")); } public void testGuessRootCause() throws IOException { @@ -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 = BaseExceptionsHelper.stackTrace(ex); + String stackTrace = ExceptionsHelper.stackTrace(ex); assertTrue(stackTrace.contains("Caused by: ParsingException[foobar]")); } 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 801f057d8888d..88628cd44c721 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.BaseExceptionsHelper; +import org.opensearch.ExceptionsHelper; 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 = BaseExceptionsHelper.unwrapCause(e); + Throwable cause = ExceptionsHelper.unwrapCause(e); if (cause == null) { throw e; } else if (cause instanceof SearchPhaseExecutionException) { diff --git a/server/src/test/java/org/opensearch/transport/TransportActionProxyTests.java b/server/src/test/java/org/opensearch/transport/TransportActionProxyTests.java index 64125863c3c5f..5190495a6253e 100644 --- a/server/src/test/java/org/opensearch/transport/TransportActionProxyTests.java +++ b/server/src/test/java/org/opensearch/transport/TransportActionProxyTests.java @@ -31,7 +31,7 @@ package org.opensearch.transport; -import org.opensearch.BaseExceptionsHelper; +import org.opensearch.ExceptionsHelper; import org.opensearch.OpenSearchException; import org.opensearch.Version; import org.opensearch.cluster.node.DiscoveryNode; @@ -196,7 +196,7 @@ public void handleResponse(SimpleTestResponse response) { @Override public void handleException(TransportException exp) { try { - Throwable cause = BaseExceptionsHelper.unwrapCause(exp); + Throwable cause = ExceptionsHelper.unwrapCause(exp); assertEquals("greetings from TS_C", cause.getMessage()); } finally { latch.countDown(); 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 4474aae1f0631..caa5b90016740 100644 --- a/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java +++ b/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java @@ -41,7 +41,7 @@ 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.ExceptionsHelper; import org.opensearch.OpenSearchException; import org.opensearch.action.ActionListener; import org.opensearch.action.DocWriteResponse; @@ -1611,7 +1611,7 @@ public void indexRandom(boolean forceRefresh, boolean dummyDocuments, boolean ma } final List actualErrors = new ArrayList<>(); for (Tuple tuple : errors) { - Throwable t = BaseExceptionsHelper.unwrapCause(tuple.v2()); + Throwable t = ExceptionsHelper.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/transport/AbstractSimpleTransportTestCase.java b/test/framework/src/main/java/org/opensearch/transport/AbstractSimpleTransportTestCase.java index 94e66a34d6cea..fed14d6bbc1ba 100644 --- a/test/framework/src/main/java/org/opensearch/transport/AbstractSimpleTransportTestCase.java +++ b/test/framework/src/main/java/org/opensearch/transport/AbstractSimpleTransportTestCase.java @@ -39,7 +39,6 @@ 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 +557,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(BaseExceptionsHelper.unwrapCause(e.getCause()).getMessage(), equalTo("simulated")); + assertThat(ExceptionsHelper.unwrapCause(e.getCause()).getMessage(), equalTo("simulated")); } // use assert busy as callbacks are called on a different thread @@ -577,7 +576,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(BaseExceptionsHelper.unwrapCause(e.getCause()).getMessage(), equalTo("simulated")); + assertThat(ExceptionsHelper.unwrapCause(e.getCause()).getMessage(), equalTo("simulated")); } // use assert busy as callbacks are called on a different thread @@ -597,7 +596,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(BaseExceptionsHelper.unwrapCause(e.getCause()).getMessage(), equalTo("simulated")); + assertThat(ExceptionsHelper.unwrapCause(e.getCause()).getMessage(), equalTo("simulated")); } // use assert busy as callbacks are called on a different thread @@ -1651,7 +1650,7 @@ public void handleResponse(StringMessageResponse response) { @Override public void handleException(TransportException exp) { - Throwable cause = BaseExceptionsHelper.unwrapCause(exp); + Throwable cause = ExceptionsHelper.unwrapCause(exp); assertThat(cause, instanceOf(ConnectTransportException.class)); assertThat(((ConnectTransportException) cause).node(), equalTo(nodeA)); } @@ -1662,7 +1661,7 @@ public void handleException(TransportException exp) { res.txGet(); fail("exception should be thrown"); } catch (Exception e) { - Throwable cause = BaseExceptionsHelper.unwrapCause(e); + Throwable cause = ExceptionsHelper.unwrapCause(e); assertThat(cause, instanceOf(ConnectTransportException.class)); assertThat(((ConnectTransportException) cause).node(), equalTo(nodeA)); } From 5dce570c85abb96507effd7391f999d850013f6e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 19 Jun 2023 13:51:17 -0700 Subject: [PATCH 043/109] Bump commons-io:commons-io from 2.12.0 to 2.13.0 in /plugins/discovery-azure-classic (#8140) * Bump commons-io:commons-io in /plugins/discovery-azure-classic Bumps commons-io:commons-io from 2.12.0 to 2.13.0. --- updated-dependencies: - dependency-name: commons-io:commons-io dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] * Updating SHAs Signed-off-by: dependabot[bot] * Added changelog entry Signed-off-by: Kartik Ganesh --------- Signed-off-by: dependabot[bot] Signed-off-by: Kartik Ganesh Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: dependabot[bot] Co-authored-by: Kartik Ganesh --- CHANGELOG.md | 3 ++- plugins/discovery-azure-classic/build.gradle | 2 +- .../licenses/commons-io-2.12.0.jar.sha1 | 1 - .../licenses/commons-io-2.13.0.jar.sha1 | 1 + 4 files changed, 4 insertions(+), 3 deletions(-) delete mode 100644 plugins/discovery-azure-classic/licenses/commons-io-2.12.0.jar.sha1 create mode 100644 plugins/discovery-azure-classic/licenses/commons-io-2.13.0.jar.sha1 diff --git a/CHANGELOG.md b/CHANGELOG.md index 30f8ef55cc92b..c93cf160d6416 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -119,6 +119,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `com.netflix.nebula:gradle-info-plugin` from 12.1.3 to 12.1.4 (#8139) - Bump `com.azure:azure-storage-common` from 12.21.1 to 12.21.2 (#8142) - Bump `com.networknt:json-schema-validator` from 1.0.83 to 1.0.84 (#8141) +- Bump `commons-io:commons-io` from 2.12.0 to 2.13.0 in /plugins/discovery-azure-classic ([#8140](https://github.com/opensearch-project/OpenSearch/pull/8140)) ### Changed - Replace jboss-annotations-api_1.2_spec with jakarta.annotation-api ([#7836](https://github.com/opensearch-project/OpenSearch/pull/7836)) @@ -139,4 +140,4 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), ### Security [Unreleased 3.0]: https://github.com/opensearch-project/OpenSearch/compare/2.x...HEAD -[Unreleased 2.x]: https://github.com/opensearch-project/OpenSearch/compare/2.8...2.x \ No newline at end of file +[Unreleased 2.x]: https://github.com/opensearch-project/OpenSearch/compare/2.8...2.x diff --git a/plugins/discovery-azure-classic/build.gradle b/plugins/discovery-azure-classic/build.gradle index 00953141b51e1..1fed446016647 100644 --- a/plugins/discovery-azure-classic/build.gradle +++ b/plugins/discovery-azure-classic/build.gradle @@ -53,7 +53,7 @@ dependencies { api "org.apache.logging.log4j:log4j-1.2-api:${versions.log4j}" api "commons-codec:commons-codec:${versions.commonscodec}" api "commons-lang:commons-lang:2.6" - api "commons-io:commons-io:2.12.0" + api "commons-io:commons-io:2.13.0" api 'javax.mail:mail:1.4.7' api 'javax.inject:javax.inject:1' api "com.sun.jersey:jersey-client:${versions.jersey}" diff --git a/plugins/discovery-azure-classic/licenses/commons-io-2.12.0.jar.sha1 b/plugins/discovery-azure-classic/licenses/commons-io-2.12.0.jar.sha1 deleted file mode 100644 index 5225b130fb817..0000000000000 --- a/plugins/discovery-azure-classic/licenses/commons-io-2.12.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -e5e3eb2ff05b494287f51476bc715161412c525f \ No newline at end of file diff --git a/plugins/discovery-azure-classic/licenses/commons-io-2.13.0.jar.sha1 b/plugins/discovery-azure-classic/licenses/commons-io-2.13.0.jar.sha1 new file mode 100644 index 0000000000000..c165136eb5822 --- /dev/null +++ b/plugins/discovery-azure-classic/licenses/commons-io-2.13.0.jar.sha1 @@ -0,0 +1 @@ +8bb2bc9b4df17e2411533a0708a69f983bf5e83b \ No newline at end of file From a72edf88ef87e4302bec7b502014919ac6d9d054 Mon Sep 17 00:00:00 2001 From: Andrew Ross Date: Mon, 19 Jun 2023 14:10:46 -0700 Subject: [PATCH 044/109] Fix changelog divergence with 2.x (#8152) Multiple changes were incorrectly placed in the "Unreleased 3.0" section on main and then backported. This brings the changelog on main in line with 2.x for changes that have been backported. Signed-off-by: Andrew Ross --- CHANGELOG.md | 56 +++++++++++++++++++++++----------------------------- 1 file changed, 25 insertions(+), 31 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index c93cf160d6416..30fc9be4f57ec 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -10,8 +10,6 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Allow mmap to use new JDK-19 preview APIs in Apache Lucene 9.4+ ([#5151](https://github.com/opensearch-project/OpenSearch/pull/5151)) - Add events correlation engine plugin ([#6854](https://github.com/opensearch-project/OpenSearch/issues/6854)) - Add support for ignoring missing Javadoc on generated code using annotation ([#7604](https://github.com/opensearch-project/OpenSearch/pull/7604)) -- Implement concurrent aggregations support without profile option ([#7514](https://github.com/opensearch-project/OpenSearch/pull/7514)) -- Add dynamic index and cluster setting for concurrent segment search ([#7956](https://github.com/opensearch-project/OpenSearch/pull/7956)) ### Dependencies - Bump `log4j-core` from 2.18.0 to 2.19.0 @@ -19,41 +17,24 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `avro` from 1.11.0 to 1.11.1 - Bump `woodstox-core` from 6.3.0 to 6.3.1 - Bump `xmlbeans` from 5.1.0 to 5.1.1 ([#4354](https://github.com/opensearch-project/OpenSearch/pull/4354)) -- Bump `azure-storage-common` from 12.18.0 to 12.18.1 ([#4164](https://github.com/opensearch-project/OpenSearch/pull/4664)) - Bump `reactor-netty-core` from 1.0.19 to 1.0.22 ([#4447](https://github.com/opensearch-project/OpenSearch/pull/4447)) - Bump `reactive-streams` from 1.0.3 to 1.0.4 ([#4488](https://github.com/opensearch-project/OpenSearch/pull/4488)) -- Bump `reactor-core` from 3.4.23 to 3.5.1 ([#5604](https://github.com/opensearch-project/OpenSearch/pull/5604)) - Bump `jempbox` from 1.8.16 to 1.8.17 ([#4550](https://github.com/opensearch-project/OpenSearch/pull/4550)) -- Bump `spock-core` from 2.1-groovy-3.0 to 2.3-groovy-3.0 ([#5315](https://github.com/opensearch-project/OpenSearch/pull/5315)) - Update to Gradle 7.6 and JDK-19 ([#4973](https://github.com/opensearch-project/OpenSearch/pull/4973)) - Update Apache Lucene to 9.5.0-snapshot-d5cef1c ([#5570](https://github.com/opensearch-project/OpenSearch/pull/5570)) - Bump `maven-model` from 3.6.2 to 3.8.6 ([#5599](https://github.com/opensearch-project/OpenSearch/pull/5599)) - Bump `maxmind-db` from 2.1.0 to 3.0.0 ([#5601](https://github.com/opensearch-project/OpenSearch/pull/5601)) - Bump `wiremock-jre8-standalone` from 2.33.2 to 2.35.0 - Bump `gson` from 2.10 to 2.10.1 -- Bump `json-schema-validator` from 1.0.73 to 1.0.76 -- Bump `joni` from 2.1.44 to 2.1.45 -- Bump `org.jruby.joni:joni` from 2.1.45 to 2.1.48 - Bump `com.google.code.gson:gson` from 2.10 to 2.10.1 - Bump `com.maxmind.geoip2:geoip2` from 4.0.0 to 4.0.1 -- Bump `com.networknt:json-schema-validator` from 1.0.76 to 1.0.78 - Bump `com.avast.gradle:gradle-docker-compose-plugin` from 0.16.11 to 0.16.12 - Bump `org.apache.commons:commons-compress` from 1.22 to 1.23.0 - Bump `org.apache.commons:commons-configuration2` from 2.8.0 to 2.9.0 - Bump `com.netflix.nebula:nebula-publishing-plugin` from 19.2.0 to 20.3.0 -- Bump `com.diffplug.spotless` from 6.17.0 to 6.18.0 - Bump `io.opencensus:opencensus-api` from 0.18.0 to 0.31.1 ([#7291](https://github.com/opensearch-project/OpenSearch/pull/7291)) -- Add `org.reactivestreams` 1.0.4 ([7372](https://github.com/opensearch-project/OpenSearch/pull/7372/)) -- Add `com.github.luben:zstd-jni` version 1.5.5-3 ([#2996](https://github.com/opensearch-project/OpenSearch/pull/2996)) - OpenJDK Update (April 2023 Patch releases) ([#7344](https://github.com/opensearch-project/OpenSearch/pull/7344) -- Bump `com.amazonaws` 1.12.270 to `software.amazon.awssdk` 2.20.55 ([7372](https://github.com/opensearch-project/OpenSearch/pull/7372/)) -- Bump `com.networknt:json-schema-validator` from 1.0.81 to 1.0.83 ([7933](https://github.com/opensearch-project/OpenSearch/pull/7933/)) - Bump `com.google.http-client:google-http-client:1.43.2` from 1.42.0 to 1.43.2 ([7928](https://github.com/opensearch-project/OpenSearch/pull/7928))) -- Bump `com.azure:azure-storage-blob` from 12.21.1 to 12.22.2 ([7930](https://github.com/opensearch-project/OpenSearch/pull/7930)) -- Bump `io.projectreactor:reactor-core` from 3.5.1 to 3.5.6 in /plugins/repository-azure ([#8016](https://github.com/opensearch-project/OpenSearch/pull/8016)) -- Bump `com.google.guava:guava` from 32.0.0-jre to 32.0.1-jre in /distribution/tools/upgrade-cli ([#8011](https://github.com/opensearch-project/OpenSearch/pull/8011)) -- Bump `com.google.guava:guava` from 32.0.0-jre to 32.0.1-jre in /distribution/tools/plugin-cli ([#8012](https://github.com/opensearch-project/OpenSearch/pull/8012)) -- Bump all `com.google.guava:guava` to 32.0.1-jre ([#8107](https://github.com/opensearch-project/OpenSearch/pull/8107)) ### Changed - [CCR] Add getHistoryOperationsFromTranslog method to fetch the history snapshot from translogs ([#3948](https://github.com/opensearch-project/OpenSearch/pull/3948)) @@ -62,12 +43,6 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Change http code on create index API with bad input raising NotXContentException from 500 to 400 ([#4773](https://github.com/opensearch-project/OpenSearch/pull/4773)) - Change http code for DecommissioningFailedException from 500 to 400 ([#5283](https://github.com/opensearch-project/OpenSearch/pull/5283)) - Improve summary error message for invalid setting updates ([#4792](https://github.com/opensearch-project/OpenSearch/pull/4792)) -- Reduce memory copy in zstd compression ([#7681](https://github.com/opensearch-project/OpenSearch/pull/7681)) -- Add ZSTD compression for snapshotting ([#2996](https://github.com/opensearch-project/OpenSearch/pull/2996)) -- Change `com.amazonaws.sdk.ec2MetadataServiceEndpointOverride` to `aws.ec2MetadataServiceEndpoint` ([7372](https://github.com/opensearch-project/OpenSearch/pull/7372/)) -- Change `com.amazonaws.sdk.stsEndpointOverride` to `aws.stsEndpointOverride` ([7372](https://github.com/opensearch-project/OpenSearch/pull/7372/)) -- Align range and default value for deletes_pct_allowed in merge policy ([#7730](https://github.com/opensearch-project/OpenSearch/pull/7730)) -- Rename QueryPhase actors like Suggest, Rescore to be processors rather than phase ([#8025](https://github.com/opensearch-project/OpenSearch/pull/8025)) ### Deprecated @@ -86,7 +61,6 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Remove LegacyESVersion.V_7_10_ Constants ([#5018](https://github.com/opensearch-project/OpenSearch/pull/5018)) - Remove Version.V_1_ Constants ([#5021](https://github.com/opensearch-project/OpenSearch/pull/5021)) - Remove custom Map, List and Set collection classes ([#6871](https://github.com/opensearch-project/OpenSearch/pull/6871)) -- Remove COMPRESSOR variable from CompressorFactory and use DEFLATE_COMPRESSOR instead ([7907](https://github.com/opensearch-project/OpenSearch/pull/7907)) ### Fixed - Fix 'org.apache.hc.core5.http.ParseException: Invalid protocol version' under JDK 16+ ([#4827](https://github.com/opensearch-project/OpenSearch/pull/4827)) @@ -95,7 +69,6 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Replaces ZipInputStream with ZipFile to fix Zip Slip vulnerability ([#7230](https://github.com/opensearch-project/OpenSearch/pull/7230)) - Add missing validation/parsing of SearchBackpressureMode of SearchBackpressureSettings ([#7541](https://github.com/opensearch-project/OpenSearch/pull/7541)) - Fix mapping char_filter when mapping a hashtag ([#7591](https://github.com/opensearch-project/OpenSearch/pull/7591)) -- With only GlobalAggregation in request causes unnecessary wrapping with MultiCollector ([#8125](https://github.com/opensearch-project/OpenSearch/pull/8125)) ### Security @@ -104,38 +77,59 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Add task cancellation monitoring service ([#7642](https://github.com/opensearch-project/OpenSearch/pull/7642)) - Add TokenManager Interface ([#7452](https://github.com/opensearch-project/OpenSearch/pull/7452)) - Add Remote store as a segment replication source ([#7653](https://github.com/opensearch-project/OpenSearch/pull/7653)) +- Implement concurrent aggregations support without profile option ([#7514](https://github.com/opensearch-project/OpenSearch/pull/7514)) +- Add dynamic index and cluster setting for concurrent segment search ([#7956](https://github.com/opensearch-project/OpenSearch/pull/7956)) - Add descending order search optimization through reverse segment read. ([#7967](https://github.com/opensearch-project/OpenSearch/pull/7967)) -- Make remote cluster connection setup in async ([#8038](https://github.com/opensearch-project/OpenSearch/pull/8038)) - Update components of segrep backpressure to support remote store. ([#8020](https://github.com/opensearch-project/OpenSearch/pull/8020)) +- Make remote cluster connection setup in async ([#8038](https://github.com/opensearch-project/OpenSearch/pull/8038)) ### Dependencies +- Bump `com.azure:azure-storage-common` from 12.21.0 to 12.21.1 (#7566, #7814) +- Bump `com.google.guava:guava` from 30.1.1-jre to 32.0.0-jre (#7565, #7811, #7807, #7808) +- Bump `net.minidev:json-smart` from 2.4.10 to 2.4.11 (#7660, #7812) +- Bump `org.gradle.test-retry` from 1.5.2 to 1.5.3 (#7810) +- Bump `com.diffplug.spotless` from 6.17.0 to 6.18.0 (#7896) - Bump `jackson` from 2.15.1 to 2.15.2 ([#7897](https://github.com/opensearch-project/OpenSearch/pull/7897)) +- Add `com.github.luben:zstd-jni` version 1.5.5-3 ([#2996](https://github.com/opensearch-project/OpenSearch/pull/2996)) - Bump `netty` from 4.1.91.Final to 4.1.93.Final ([#7901](https://github.com/opensearch-project/OpenSearch/pull/7901)) +- Bump `com.amazonaws` 1.12.270 to `software.amazon.awssdk` 2.20.55 ([7372](https://github.com/opensearch-project/OpenSearch/pull/7372/)) +- Add `org.reactivestreams` 1.0.4 ([7372](https://github.com/opensearch-project/OpenSearch/pull/7372/)) +- Bump `com.networknt:json-schema-validator` from 1.0.81 to 1.0.83 ([7968](https://github.com/opensearch-project/OpenSearch/pull/7968)) - Bump `com.netflix.nebula:gradle-extra-configurations-plugin` from 9.0.0 to 10.0.0 in /buildSrc ([#7068](https://github.com/opensearch-project/OpenSearch/pull/7068)) - Bump `com.google.guava:guava` from 32.0.0-jre to 32.0.1-jre (#8009) - Bump `commons-io:commons-io` from 2.12.0 to 2.13.0 (#8014, #8013, #8010) - Bump `com.diffplug.spotless` from 6.18.0 to 6.19.0 (#8007) +- Bump `'com.azure:azure-storage-blob` to 12.22.2 from 12.21.1 ([#8043](https://github.com/opensearch-project/OpenSearch/pull/8043)) - Bump `org.jruby.joni:joni` from 2.1.48 to 2.2.1 (#8015) -- Bump `com.netflix.nebula:gradle-info-plugin` from 12.1.3 to 12.1.4 (#8139) -- Bump `com.azure:azure-storage-common` from 12.21.1 to 12.21.2 (#8142) +- Bump `com.google.guava:guava` from 32.0.0-jre to 32.0.1-jre ([#8011](https://github.com/opensearch-project/OpenSearch/pull/8011), [#8012](https://github.com/opensearch-project/OpenSearch/pull/8012), [#8107](https://github.com/opensearch-project/OpenSearch/pull/8107)) +- Bump `io.projectreactor:reactor-core` from 3.4.18 to 3.5.6 in /plugins/repository-azure ([#8016](https://github.com/opensearch-project/OpenSearch/pull/8016)) +- Bump `spock-core` from 2.1-groovy-3.0 to 2.3-groovy-3.0 ([#8122](https://github.com/opensearch-project/OpenSearch/pull/8122)) - Bump `com.networknt:json-schema-validator` from 1.0.83 to 1.0.84 (#8141) - Bump `commons-io:commons-io` from 2.12.0 to 2.13.0 in /plugins/discovery-azure-classic ([#8140](https://github.com/opensearch-project/OpenSearch/pull/8140)) ### Changed - Replace jboss-annotations-api_1.2_spec with jakarta.annotation-api ([#7836](https://github.com/opensearch-project/OpenSearch/pull/7836)) +- Reduce memory copy in zstd compression ([#7681](https://github.com/opensearch-project/OpenSearch/pull/7681)) - Add min, max, average and thread info to resource stats in tasks API ([#7673](https://github.com/opensearch-project/OpenSearch/pull/7673)) +- Add ZSTD compression for snapshotting ([#2996](https://github.com/opensearch-project/OpenSearch/pull/2996)) +- Change `com.amazonaws.sdk.ec2MetadataServiceEndpointOverride` to `aws.ec2MetadataServiceEndpoint` ([7372](https://github.com/opensearch-project/OpenSearch/pull/7372/)) +- Change `com.amazonaws.sdk.stsEndpointOverride` to `aws.stsEndpointOverride` ([7372](https://github.com/opensearch-project/OpenSearch/pull/7372/)) +- Add new query profile collector fields with concurrent search execution ([#7898](https://github.com/opensearch-project/OpenSearch/pull/7898)) +- Align range and default value for deletes_pct_allowed in merge policy ([#7730](https://github.com/opensearch-project/OpenSearch/pull/7730)) +- Rename QueryPhase actors like Suggest, Rescore to be processors rather than phase ([#8025](https://github.com/opensearch-project/OpenSearch/pull/8025)) - Compress and cache cluster state during validate join request ([#7321](https://github.com/opensearch-project/OpenSearch/pull/7321)) - [Snapshot Interop] Add Changes in Create Snapshot Flow for remote store interoperability. ([#7118](https://github.com/opensearch-project/OpenSearch/pull/7118)) -- Add new query profile collector fields with concurrent search execution ([#7898](https://github.com/opensearch-project/OpenSearch/pull/7898)) - Allow insecure string settings to warn-log usage and advise to migration of a newer secure variant ([#5496](https://github.com/opensearch-project/OpenSearch/pull/5496)) ### Deprecated ### Removed +- Remove COMPRESSOR variable from CompressorFactory and use DEFLATE_COMPRESSOR instead ([7907](https://github.com/opensearch-project/OpenSearch/pull/7907)) ### Fixed - Fixing error: adding a new/forgotten parameter to the configuration for checking the config on startup in plugins/repository-s3 #7924 - Enforce 512 byte document ID limit in bulk updates ([#8039](https://github.com/opensearch-project/OpenSearch/pull/8039)) +- With only GlobalAggregation in request causes unnecessary wrapping with MultiCollector ([#8125](https://github.com/opensearch-project/OpenSearch/pull/8125)) ### Security From e6fd76ccd5f02b20ef308aab8aa2472cc1340de4 Mon Sep 17 00:00:00 2001 From: Kartik Ganesh Date: Tue, 20 Jun 2023 12:53:41 -0700 Subject: [PATCH 045/109] Minor fixes to CHANGELOG (#8179) This fixes some minor differences with the 2.x changelog Signed-off-by: Kartik Ganesh --- CHANGELOG.md | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 30fc9be4f57ec..32071699eae59 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -68,7 +68,6 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Support OpenSSL Provider with default Netty allocator ([#5460](https://github.com/opensearch-project/OpenSearch/pull/5460)) - Replaces ZipInputStream with ZipFile to fix Zip Slip vulnerability ([#7230](https://github.com/opensearch-project/OpenSearch/pull/7230)) - Add missing validation/parsing of SearchBackpressureMode of SearchBackpressureSettings ([#7541](https://github.com/opensearch-project/OpenSearch/pull/7541)) -- Fix mapping char_filter when mapping a hashtag ([#7591](https://github.com/opensearch-project/OpenSearch/pull/7591)) ### Security @@ -105,6 +104,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `io.projectreactor:reactor-core` from 3.4.18 to 3.5.6 in /plugins/repository-azure ([#8016](https://github.com/opensearch-project/OpenSearch/pull/8016)) - Bump `spock-core` from 2.1-groovy-3.0 to 2.3-groovy-3.0 ([#8122](https://github.com/opensearch-project/OpenSearch/pull/8122)) - Bump `com.networknt:json-schema-validator` from 1.0.83 to 1.0.84 (#8141) +- Bump `com.netflix.nebula:gradle-info-plugin` from 12.1.3 to 12.1.4 (#8139) - Bump `commons-io:commons-io` from 2.12.0 to 2.13.0 in /plugins/discovery-azure-classic ([#8140](https://github.com/opensearch-project/OpenSearch/pull/8140)) ### Changed @@ -124,12 +124,13 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), ### Deprecated ### Removed -- Remove COMPRESSOR variable from CompressorFactory and use DEFLATE_COMPRESSOR instead ([7907](https://github.com/opensearch-project/OpenSearch/pull/7907)) +- Remove `COMPRESSOR` variable from `CompressorFactory` and use `DEFLATE_COMPRESSOR` instead ([7907](https://github.com/opensearch-project/OpenSearch/pull/7907)) ### Fixed -- Fixing error: adding a new/forgotten parameter to the configuration for checking the config on startup in plugins/repository-s3 #7924 +- Fixing error: adding a new/forgotten parameter to the configuration for checking the config on startup in plugins/repository-s3 #7924 - Enforce 512 byte document ID limit in bulk updates ([#8039](https://github.com/opensearch-project/OpenSearch/pull/8039)) - With only GlobalAggregation in request causes unnecessary wrapping with MultiCollector ([#8125](https://github.com/opensearch-project/OpenSearch/pull/8125)) +- Fix mapping char_filter when mapping a hashtag ([#7591](https://github.com/opensearch-project/OpenSearch/pull/7591)) ### Security From 40d758387bd9227314a64a73035260829bfc3c10 Mon Sep 17 00:00:00 2001 From: Darshit Chanpura <35282393+DarshitChanpura@users.noreply.github.com> Date: Tue, 20 Jun 2023 17:30:28 -0400 Subject: [PATCH 046/109] [Enhancement] Pass localNode info to all plugins on node start (#7919) * Updates Node start to now pass localNode info so that it is available directly to a plugin instead of hacky implementation Signed-off-by: Darshit Chanpura * Adds this PR to changelog Signed-off-by: Darshit Chanpura * Makes onNodeStarted call plugin agnostic Signed-off-by: Darshit Chanpura * Marking onNodeStarted() as deprecated Signed-off-by: Darshit Chanpura * Adds a test to verify the behaviour of new signature of onNodeStarted Signed-off-by: Darshit Chanpura * Updates test to also check for nodeId Signed-off-by: Darshit Chanpura * Adds instructions to use the overloaded implementation of onNodeStarted and adds a test to check that current plugins are not affect with this implementation Signed-off-by: Darshit Chanpura --------- Signed-off-by: Darshit Chanpura Signed-off-by: Darshit Chanpura <35282393+DarshitChanpura@users.noreply.github.com> --- CHANGELOG.md | 1 + .../main/java/org/opensearch/node/Node.java | 2 +- .../org/opensearch/plugins/ClusterPlugin.java | 13 ++++ .../plugins/ClusterPluginTests.java | 78 +++++++++++++++++++ 4 files changed, 93 insertions(+), 1 deletion(-) create mode 100644 server/src/test/java/org/opensearch/plugins/ClusterPluginTests.java diff --git a/CHANGELOG.md b/CHANGELOG.md index 32071699eae59..961bc2978d4e1 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -43,6 +43,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Change http code on create index API with bad input raising NotXContentException from 500 to 400 ([#4773](https://github.com/opensearch-project/OpenSearch/pull/4773)) - Change http code for DecommissioningFailedException from 500 to 400 ([#5283](https://github.com/opensearch-project/OpenSearch/pull/5283)) - Improve summary error message for invalid setting updates ([#4792](https://github.com/opensearch-project/OpenSearch/pull/4792)) +- Pass localNode info to all plugins on node start ([#7919](https://github.com/opensearch-project/OpenSearch/pull/7919)) ### Deprecated diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index c51eff57de68c..808c054de4969 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -1356,7 +1356,7 @@ public void onTimeout(TimeValue timeout) { logger.info("started"); - pluginsService.filterPlugins(ClusterPlugin.class).forEach(ClusterPlugin::onNodeStarted); + pluginsService.filterPlugins(ClusterPlugin.class).forEach(plugin -> plugin.onNodeStarted(clusterService.localNode())); return this; } diff --git a/server/src/main/java/org/opensearch/plugins/ClusterPlugin.java b/server/src/main/java/org/opensearch/plugins/ClusterPlugin.java index b0ed91dd4c468..c2e147b86d17f 100644 --- a/server/src/main/java/org/opensearch/plugins/ClusterPlugin.java +++ b/server/src/main/java/org/opensearch/plugins/ClusterPlugin.java @@ -32,6 +32,7 @@ package org.opensearch.plugins; +import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.routing.allocation.ExistingShardsAllocator; import org.opensearch.cluster.routing.allocation.allocator.ShardsAllocator; import org.opensearch.cluster.routing.allocation.decider.AllocationDecider; @@ -86,7 +87,19 @@ default Map getExistingShardsAllocators() { /** * Called when the node is started + * + * DEPRECATED: Use {@link #onNodeStarted(DiscoveryNode)} for newer implementations. */ + @Deprecated default void onNodeStarted() {} + /** + * Called when node is started. DiscoveryNode argument is passed to allow referring localNode value inside plugin + * + * @param localNode local Node info + */ + default void onNodeStarted(DiscoveryNode localNode) { + onNodeStarted(); + } + } diff --git a/server/src/test/java/org/opensearch/plugins/ClusterPluginTests.java b/server/src/test/java/org/opensearch/plugins/ClusterPluginTests.java new file mode 100644 index 0000000000000..c90d4e2789e82 --- /dev/null +++ b/server/src/test/java/org/opensearch/plugins/ClusterPluginTests.java @@ -0,0 +1,78 @@ +/* + * 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.plugins; + +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.test.OpenSearchSingleNodeTestCase; + +import java.util.Collection; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +public class ClusterPluginTests extends OpenSearchSingleNodeTestCase { + + @Override + protected Collection> getPlugins() { + return pluginList(DummyClusterPlugin.class, DummyClusterPlugin2.class); + } + + public void testOnNodeStarted_shouldContainLocalNodeInfo() { + + DiscoveryNode localNode = DummyClusterPlugin.getLocalNode(); + + assertTrue(localNode != null); + // TODO Figure out if there is a way to check ephemeralId + assertTrue(localNode.getId().equals(node().getNodeEnvironment().nodeId())); + } + + public void testOnNodeStarted_shouldCallDeprecatedMethod() { + DummyClusterPlugin2 dummyClusterPlugin2 = mock(DummyClusterPlugin2.class); + dummyClusterPlugin2.onNodeStarted(); + verify(dummyClusterPlugin2, times(1)).onNodeStarted(); + + DiscoveryNode localNode = DummyClusterPlugin2.getLocalNode(); + assertTrue(localNode != null); + } + +} + +final class DummyClusterPlugin extends Plugin implements ClusterPlugin { + + private static volatile DiscoveryNode localNode; + + public DummyClusterPlugin() {} + + @Override + public void onNodeStarted(DiscoveryNode localNode) { + DummyClusterPlugin.localNode = localNode; + } + + public static DiscoveryNode getLocalNode() { + return localNode; + } +} + +class DummyClusterPlugin2 extends Plugin implements ClusterPlugin { + + private static volatile DiscoveryNode localNode; + + public DummyClusterPlugin2() {} + + @Override + public void onNodeStarted() { + localNode = mock(DiscoveryNode.class); + } + + public static DiscoveryNode getLocalNode() { + return localNode; + } + +} From 1f6a3bca68149eb263a1287f3aeed767226813ad Mon Sep 17 00:00:00 2001 From: Daniel Widdis Date: Tue, 20 Jun 2023 14:40:42 -0700 Subject: [PATCH 047/109] Consume all params and content on extension Rest Request exception (#8096) Signed-off-by: Daniel Widdis --- .../opensearch/rest/extensions/RestSendToExtensionAction.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/server/src/main/java/org/opensearch/rest/extensions/RestSendToExtensionAction.java b/server/src/main/java/org/opensearch/rest/extensions/RestSendToExtensionAction.java index 8c4e3f4b42412..51ff74b1869a0 100644 --- a/server/src/main/java/org/opensearch/rest/extensions/RestSendToExtensionAction.java +++ b/server/src/main/java/org/opensearch/rest/extensions/RestSendToExtensionAction.java @@ -219,6 +219,10 @@ public void handleResponse(RestExecuteOnExtensionResponse response) { @Override public void handleException(TransportException exp) { logger.debug("REST request failed", exp); + // On failure the original request params and content aren't consumed + // which gives misleading error messages, so we just consume them here + request.params().keySet().stream().forEach(p -> request.param(p)); + request.content(); inProgressFuture.completeExceptionally(exp); } From 68c1c86897a59fcffa77bfecc06641326e5e5199 Mon Sep 17 00:00:00 2001 From: Andriy Redko Date: Tue, 20 Jun 2023 18:02:25 -0400 Subject: [PATCH 048/109] Bump 'mockito' from 5.2.0 to 5.4.0 (#8181) Signed-off-by: Andriy Redko --- CHANGELOG.md | 1 + buildSrc/version.properties | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 961bc2978d4e1..cb67e56c7845f 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -107,6 +107,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `com.networknt:json-schema-validator` from 1.0.83 to 1.0.84 (#8141) - Bump `com.netflix.nebula:gradle-info-plugin` from 12.1.3 to 12.1.4 (#8139) - Bump `commons-io:commons-io` from 2.12.0 to 2.13.0 in /plugins/discovery-azure-classic ([#8140](https://github.com/opensearch-project/OpenSearch/pull/8140)) +- Bump `mockito` from 5.2.0 to 5.4.0 ([#8181](https://github.com/opensearch-project/OpenSearch/pull/8181)) ### Changed - Replace jboss-annotations-api_1.2_spec with jakarta.annotation-api ([#7836](https://github.com/opensearch-project/OpenSearch/pull/7836)) diff --git a/buildSrc/version.properties b/buildSrc/version.properties index 33fad06328b80..84d3b2b814430 100644 --- a/buildSrc/version.properties +++ b/buildSrc/version.properties @@ -53,7 +53,7 @@ bouncycastle=1.70 randomizedrunner = 2.7.1 junit = 4.13.2 hamcrest = 2.1 -mockito = 5.2.0 +mockito = 5.4.0 objenesis = 3.2 bytebuddy = 1.14.3 From 2d9a88c35ddf5a110ff33ec980b12b665b6b0eb6 Mon Sep 17 00:00:00 2001 From: Kuanysh <90975457+Kuanysh-kst@users.noreply.github.com> Date: Thu, 22 Jun 2023 01:23:43 +0600 Subject: [PATCH 049/109] [CCI] [Extensions] Added the ExtensionActionUtil class (#6969) * add ExtensionActionUtill class Signed-off-by: Kuanysh * add ExtensionActionUtilTest class Signed-off-by: Kuanysh * refactoring Signed-off-by: Kuanysh * add method convertParamsToBytes Signed-off-by: Kuanysh * code refactoring Signed-off-by: Kuanysh * add ExtensionActionUtill class Signed-off-by: Kuanysh * add ExtensionActionUtilTest class Signed-off-by: Kuanysh * refactoring Signed-off-by: Kuanysh * add method convertParamsToBytes Signed-off-by: Kuanysh * code refactoring Signed-off-by: Kuanysh * refactoring Signed-off-by: Kuanysh * added new name to tests Signed-off-by: Kuanysh * code refactoring Signed-off-by: Kuanysh <90975457+Kuanysh-kst@users.noreply.github.com> * test code refactoring Signed-off-by: Kuanysh <90975457+Kuanysh-kst@users.noreply.github.com> * test class renaming Signed-off-by: Kuanysh <90975457+Kuanysh-kst@users.noreply.github.com> --------- Signed-off-by: Kuanysh Signed-off-by: Kuanysh <90975457+Kuanysh-kst@users.noreply.github.com> --- .../action/ExtensionActionUtil.java | 117 ++++++++++++++++++ .../action/ExtensionActionUtilTests.java | 106 ++++++++++++++++ 2 files changed, 223 insertions(+) create mode 100644 server/src/main/java/org/opensearch/extensions/action/ExtensionActionUtil.java create mode 100644 server/src/test/java/org/opensearch/extensions/action/ExtensionActionUtilTests.java diff --git a/server/src/main/java/org/opensearch/extensions/action/ExtensionActionUtil.java b/server/src/main/java/org/opensearch/extensions/action/ExtensionActionUtil.java new file mode 100644 index 0000000000000..8b898b3afb1e2 --- /dev/null +++ b/server/src/main/java/org/opensearch/extensions/action/ExtensionActionUtil.java @@ -0,0 +1,117 @@ +/* + * 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.extensions.action; + +import org.opensearch.action.ActionRequest; +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.Writeable; + +import java.io.IOException; +import java.lang.reflect.Constructor; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; + +/** + * ExtensionActionUtil - a class for creating and processing remote requests using byte arrays. + */ +public class ExtensionActionUtil { + + /** + * The Unicode UNIT SEPARATOR used to separate the Request class name and parameter bytes + */ + public static final byte UNIT_SEPARATOR = (byte) '\u001F'; + + /** + * @param request an instance of a request extending {@link ActionRequest}, containing information about the + * request being sent to the remote server. It is used to create a byte array containing the request data, + * which will be sent to the remote server. + * @return An Extension ActionRequest object that represents the deserialized data. + * If an error occurred during the deserialization process, the method will return {@code null}. + * @throws RuntimeException If a RuntimeException occurs while creating the proxy request bytes. + */ + public static byte[] createProxyRequestBytes(ActionRequest request) throws RuntimeException { + byte[] requestClassBytes = request.getClass().getName().getBytes(StandardCharsets.UTF_8); + byte[] requestBytes; + + try { + requestBytes = convertParamsToBytes(request); + assert requestBytes != null; + return ByteBuffer.allocate(requestClassBytes.length + 1 + requestBytes.length) + .put(requestClassBytes) + .put(UNIT_SEPARATOR) + .put(requestBytes) + .array(); + } catch (RuntimeException e) { + throw new RuntimeException("RuntimeException occurred while creating proxyRequestBytes"); + } + } + + /** + * @param requestBytes is a byte array containing the request data, used by the "createActionRequest" + * method to create an "ActionRequest" object, which represents the request model to be processed on the server. + * @return an "Action Request" object representing the request model for processing on the server, + * or {@code null} if the request data is invalid or null. + * @throws ReflectiveOperationException if an exception occurs during the reflective operation, such as when + * resolving the request class, accessing the constructor, or creating an instance using reflection + * @throws NullPointerException if a null pointer exception occurs during the creation of the ActionRequest object + */ + public static ActionRequest createActionRequest(byte[] requestBytes) throws ReflectiveOperationException { + int delimPos = delimPos(requestBytes); + String requestClassName = new String(Arrays.copyOfRange(requestBytes, 0, delimPos + 1), StandardCharsets.UTF_8).stripTrailing(); + try { + Class clazz = Class.forName(requestClassName); + Constructor constructor = clazz.getConstructor(StreamInput.class); + StreamInput requestByteStream = StreamInput.wrap(Arrays.copyOfRange(requestBytes, delimPos + 1, requestBytes.length)); + return (ActionRequest) constructor.newInstance(requestByteStream); + } catch (ReflectiveOperationException e) { + throw new ReflectiveOperationException( + "ReflectiveOperationException occurred while creating extensionAction request from bytes", + e + ); + } catch (NullPointerException e) { + throw new NullPointerException( + "NullPointerException occurred while creating extensionAction request from bytes" + e.getMessage() + ); + } + } + + /** + * Converts the given object of type T, which implements the {@link Writeable} interface, to a byte array. + * @param the type of the object to be converted to bytes, which must implement the {@link Writeable} interface. + * @param writeableObject the object of type T to be converted to bytes. + * @return a byte array containing the serialized bytes of the given object, or {@code null} if the input is invalid or null. + * @throws IllegalStateException if a failure occurs while writing the data + */ + public static byte[] convertParamsToBytes(T writeableObject) throws IllegalStateException { + try (BytesStreamOutput out = new BytesStreamOutput()) { + writeableObject.writeTo(out); + return BytesReference.toBytes(out.bytes()); + } catch (IOException ieo) { + throw new IllegalStateException("Failure writing bytes", ieo); + } + } + + /** + * Searches for the position of the unit separator byte in the given byte array. + * + * @param bytes the byte array to search for the unit separator byte. + * @return the index of the unit separator byte in the byte array, or -1 if it was not found. + */ + public static int delimPos(byte[] bytes) { + for (int offset = 0; offset < bytes.length; ++offset) { + if (bytes[offset] == ExtensionActionUtil.UNIT_SEPARATOR) { + return offset; + } + } + return -1; + } +} diff --git a/server/src/test/java/org/opensearch/extensions/action/ExtensionActionUtilTests.java b/server/src/test/java/org/opensearch/extensions/action/ExtensionActionUtilTests.java new file mode 100644 index 0000000000000..d2b889d33da9a --- /dev/null +++ b/server/src/test/java/org/opensearch/extensions/action/ExtensionActionUtilTests.java @@ -0,0 +1,106 @@ +/* + * 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.extensions.action; + +import org.junit.Before; +import org.mockito.Mockito; +import org.opensearch.action.ActionRequest; +import org.opensearch.action.ActionRequestValidationException; +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.io.stream.Writeable; +import org.opensearch.test.OpenSearchTestCase; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; + +import static org.opensearch.extensions.action.ExtensionActionUtil.UNIT_SEPARATOR; +import static org.opensearch.extensions.action.ExtensionActionUtil.createProxyRequestBytes; + +public class ExtensionActionUtilTests extends OpenSearchTestCase { + private byte[] myBytes; + private final String actionName = "org.opensearch.action.MyExampleRequest"; + private final byte[] actionNameBytes = MyExampleRequest.class.getName().getBytes(StandardCharsets.UTF_8); + + @Before + public void setup() throws IOException { + BytesStreamOutput out = new BytesStreamOutput(); + MyExampleRequest exampleRequest = new MyExampleRequest(actionName, actionNameBytes); + exampleRequest.writeTo(out); + + byte[] requestBytes = BytesReference.toBytes(out.bytes()); + byte[] requestClass = MyExampleRequest.class.getName().getBytes(StandardCharsets.UTF_8); + this.myBytes = ByteBuffer.allocate(requestClass.length + 1 + requestBytes.length) + .put(requestClass) + .put(UNIT_SEPARATOR) + .put(requestBytes) + .array(); + } + + public void testCreateProxyRequestBytes() throws IOException { + BytesStreamOutput out = new BytesStreamOutput(); + MyExampleRequest exampleRequest = new MyExampleRequest(actionName, actionNameBytes); + exampleRequest.writeTo(out); + + byte[] result = createProxyRequestBytes(exampleRequest); + assertArrayEquals(this.myBytes, result); + assertThrows(RuntimeException.class, () -> ExtensionActionUtil.createProxyRequestBytes(new MyExampleRequest(null, null))); + } + + public void testCreateActionRequest() throws ReflectiveOperationException { + ActionRequest actionRequest = ExtensionActionUtil.createActionRequest(myBytes); + assertThrows(NullPointerException.class, () -> ExtensionActionUtil.createActionRequest(null)); + assertThrows(ReflectiveOperationException.class, () -> ExtensionActionUtil.createActionRequest(actionNameBytes)); + assertNotNull(actionRequest); + assertFalse(actionRequest.getShouldStoreResult()); + } + + public void testConvertParamsToBytes() throws IOException { + Writeable mockWriteableObject = Mockito.mock(Writeable.class); + Mockito.doThrow(new IOException("Test IOException")).when(mockWriteableObject).writeTo(Mockito.any()); + assertThrows(IllegalStateException.class, () -> ExtensionActionUtil.convertParamsToBytes(mockWriteableObject)); + } + + public void testDelimPos() { + assertTrue(ExtensionActionUtil.delimPos(myBytes) > 0); + assertTrue(ExtensionActionUtil.delimPos(actionNameBytes) < 0); + assertEquals(-1, ExtensionActionUtil.delimPos(actionNameBytes)); + } + + private static class MyExampleRequest extends ActionRequest { + private final String param1; + private final byte[] param2; + + public MyExampleRequest(String param1, byte[] param2) { + this.param1 = param1; + this.param2 = param2; + } + + public MyExampleRequest(StreamInput in) throws IOException { + super(in); + param1 = in.readString(); + param2 = in.readByteArray(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(param1); + out.writeByteArray(param2); + } + + @Override + public ActionRequestValidationException validate() { + return null; + } + } +} From cc67469292bc8d5be1b476fb0fb11e07fe82f8a6 Mon Sep 17 00:00:00 2001 From: Kuanysh <90975457+Kuanysh-kst@users.noreply.github.com> Date: Thu, 22 Jun 2023 04:00:06 +0600 Subject: [PATCH 050/109] [CCI] [BUG] Fixing extension settings update consumers (#7456) * add equels for ClusterSettings Signed-off-by: Kuanysh * added junit Signed-off-by: Kuanysh * code refactoring Signed-off-by: Kuanysh * added changes to handleAddSettingsUpdateConsumer Signed-off-by: Kuanysh <90975457+Kuanysh-kst@users.noreply.github.com> * code refactoring Signed-off-by: Kuanysh Aimurzinov * code refactoring Signed-off-by: Kuanysh Aimurzinov * changed main method Signed-off-by: Kuanysh <90975457+Kuanysh-kst@users.noreply.github.com> --------- Signed-off-by: Kuanysh Signed-off-by: Kuanysh <90975457+Kuanysh-kst@users.noreply.github.com> Signed-off-by: Kuanysh Aimurzinov --- ...dSettingsUpdateConsumerRequestHandler.java | 60 ++++++++++++++----- .../extensions/ExtensionsManager.java | 3 +- .../common/settings/ScopedSettingsTests.java | 16 +++++ 3 files changed, 64 insertions(+), 15 deletions(-) diff --git a/server/src/main/java/org/opensearch/extensions/AddSettingsUpdateConsumerRequestHandler.java b/server/src/main/java/org/opensearch/extensions/AddSettingsUpdateConsumerRequestHandler.java index b1c8b655c4c6f..67c56b7f458ff 100644 --- a/server/src/main/java/org/opensearch/extensions/AddSettingsUpdateConsumerRequestHandler.java +++ b/server/src/main/java/org/opensearch/extensions/AddSettingsUpdateConsumerRequestHandler.java @@ -15,6 +15,7 @@ import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.SettingsException; +import org.opensearch.common.settings.SettingsModule; import org.opensearch.common.settings.WriteableSetting; import org.opensearch.transport.TransportResponse; import org.opensearch.transport.TransportService; @@ -31,6 +32,7 @@ public class AddSettingsUpdateConsumerRequestHandler { private final ClusterService clusterService; private final TransportService transportService; private final String updateSettingsRequestType; + private final SettingsModule settingsModule; /** * Instantiates a new Add Settings Update Consumer Request Handler with the {@link ClusterService} and {@link TransportService} @@ -42,11 +44,13 @@ public class AddSettingsUpdateConsumerRequestHandler { public AddSettingsUpdateConsumerRequestHandler( ClusterService clusterService, TransportService transportService, - String updateSettingsRequestType + String updateSettingsRequestType, + SettingsModule settingsModule ) { this.clusterService = clusterService; this.transportService = transportService; this.updateSettingsRequestType = updateSettingsRequestType; + this.settingsModule = settingsModule; } /** @@ -68,25 +72,53 @@ TransportResponse handleAddSettingsUpdateConsumerRequest(AddSettingsUpdateConsum // Extract setting and type from writeable setting Setting setting = extensionComponentSetting.getSetting(); + + // we need to get the actual setting from nodeSetting or indexsetting maps in SettingsModule + // use conditional based on setting properties + Setting settingForUpdateConsumer = null; + if (setting.hasIndexScope()) { + settingForUpdateConsumer = settingsModule.getIndexScopedSettings().get(setting.getKey()); + } else if (setting.hasNodeScope()) { + settingForUpdateConsumer = settingsModule.getClusterSettings().get(setting.getKey()); + } + // do a null check and throw IllegalArgument exception here if neither index or node scope + if (settingForUpdateConsumer == null) { + throw new IllegalArgumentException("Not index or node scope"); + } + WriteableSetting.SettingType settingType = extensionComponentSetting.getType(); - // Register setting update consumer with callback method to extension - clusterService.getClusterSettings().addSettingsUpdateConsumer(setting, (data) -> { - logger.debug("Sending extension request type: " + updateSettingsRequestType); - UpdateSettingsResponseHandler updateSettingsResponseHandler = new UpdateSettingsResponseHandler(); - transportService.sendRequest( - extensionNode, - updateSettingsRequestType, - new UpdateSettingsRequest(settingType, setting, data), - updateSettingsResponseHandler - ); - }); + if (setting.hasIndexScope()) { + // Register setting update consumer with callback method to extension + clusterService.getClusterSettings().addSettingsUpdateConsumer(settingForUpdateConsumer, (data) -> { + logger.debug("Sending extension request type: " + updateSettingsRequestType); + UpdateSettingsResponseHandler updateSettingsResponseHandler = new UpdateSettingsResponseHandler(); + transportService.sendRequest( + extensionNode, + updateSettingsRequestType, + new UpdateSettingsRequest(settingType, setting, data), + updateSettingsResponseHandler + ); + }); + } + if (setting.hasNodeScope()) { + // Register setting update consumer with callback method to extension + clusterService.getClusterSettings().addSettingsUpdateConsumer(settingForUpdateConsumer, (data) -> { + logger.debug("Sending extension request type: " + updateSettingsRequestType); + UpdateSettingsResponseHandler updateSettingsResponseHandler = new UpdateSettingsResponseHandler(); + transportService.sendRequest( + extensionNode, + updateSettingsRequestType, + new UpdateSettingsRequest(settingType, setting, data), + updateSettingsResponseHandler + ); + }); + } } - } catch (SettingsException e) { + } catch (SettingsException | IllegalArgumentException e) { logger.error(e.toString()); status = false; } - return new AcknowledgedResponse(status); } } diff --git a/server/src/main/java/org/opensearch/extensions/ExtensionsManager.java b/server/src/main/java/org/opensearch/extensions/ExtensionsManager.java index 9d74e8f22d2b1..5c6b9f3141aa0 100644 --- a/server/src/main/java/org/opensearch/extensions/ExtensionsManager.java +++ b/server/src/main/java/org/opensearch/extensions/ExtensionsManager.java @@ -172,7 +172,8 @@ public void initializeServicesAndRestHandler( this.addSettingsUpdateConsumerRequestHandler = new AddSettingsUpdateConsumerRequestHandler( clusterService, transportService, - REQUEST_EXTENSION_UPDATE_SETTINGS + REQUEST_EXTENSION_UPDATE_SETTINGS, + settingsModule ); this.client = client; this.extensionTransportActionsHandler = new ExtensionTransportActionsHandler( diff --git a/server/src/test/java/org/opensearch/common/settings/ScopedSettingsTests.java b/server/src/test/java/org/opensearch/common/settings/ScopedSettingsTests.java index 1dcb8ee00ebb2..0c5cece4249ef 100644 --- a/server/src/test/java/org/opensearch/common/settings/ScopedSettingsTests.java +++ b/server/src/test/java/org/opensearch/common/settings/ScopedSettingsTests.java @@ -1462,4 +1462,20 @@ public List getListValue(final List value) { ); } + public void testAddSettingsUpdateConsumer() { + Setting testSetting = Setting.intSetting("foo.bar", 1, Property.Dynamic, Property.NodeScope); + Setting testSetting2 = Setting.intSetting("foo.bar.baz", 1, Property.Dynamic, Property.NodeScope); + AbstractScopedSettings service = new ClusterSettings(Settings.EMPTY, new HashSet<>(Arrays.asList(testSetting, testSetting2))); + AtomicInteger consumer2 = new AtomicInteger(); + service.addSettingsUpdateConsumer(testSetting2, consumer2::set, (s) -> assertTrue(s > 0)); + Setting wrongKeySetting = Setting.intSetting("foo.bar.wrong", 1, Property.Dynamic, Property.NodeScope); + + expectThrows(SettingsException.class, () -> service.addSettingsUpdateConsumer(wrongKeySetting, consumer2::set, (i) -> { + if (i == 42) throw new AssertionError("wrong key"); + })); + + expectThrows(NullPointerException.class, () -> service.addSettingsUpdateConsumer(null, consumer2::set, (i) -> { + if (i == 42) throw new AssertionError("empty key"); + })); + } } From 9c5c6eb2a6da295aaea383e3c80effe246082563 Mon Sep 17 00:00:00 2001 From: Andrew Ross Date: Wed, 21 Jun 2023 16:17:58 -0700 Subject: [PATCH 051/109] Add action to label and close stalled PRs (#8178) Signed-off-by: Andrew Ross --- .github/workflows/stalled.yml | 28 ++++++++++++++++++++++++++++ 1 file changed, 28 insertions(+) create mode 100644 .github/workflows/stalled.yml diff --git a/.github/workflows/stalled.yml b/.github/workflows/stalled.yml new file mode 100644 index 0000000000000..0d03049a2e23c --- /dev/null +++ b/.github/workflows/stalled.yml @@ -0,0 +1,28 @@ +name: Close Stalled PRs +on: + schedule: + - cron: '15 15 * * *' # Run every day at 15:15 UTC / 7:15 PST / 8:15 PDT +permissions: + pull-requests: write +jobs: + stale: + runs-on: ubuntu-latest + steps: + - name: GitHub App token + id: github_app_token + uses: tibdex/github-app-token@v1.5.0 + with: + app_id: ${{ secrets.APP_ID }} + private_key: ${{ secrets.APP_PRIVATE_KEY }} + installation_id: 22958780 + - name: Stale PRs + uses: actions/stale@v8 + with: + repo-token: ${{ steps.github_app_token.outputs.token }} + stale-pr-label: 'stalled' + stale-pr-message: 'This PR is stalled because it has been open for 30 days with no activity. Remove stalled label or comment or this will be closed in 7 days.' + close-pr-message: 'This PR was closed because it has been stalled for 7 days with no activity.' + days-before-pr-stale: 30 + days-before-pr-close: 7 + days-before-issue-stale: -1 + days-before-issue-close: -1 From f015e6a6cc205d1300c852fc731aac90bac3d930 Mon Sep 17 00:00:00 2001 From: Owais Kazi Date: Wed, 21 Jun 2023 16:52:31 -0700 Subject: [PATCH 052/109] [Extensions] REST API to initialize an extension and dynamically load it (#8029) * Implemented REST API for initializing extension Signed-off-by: Owais Kazi * Cleanup extensions.yml design Signed-off-by: Owais Kazi * Added tests for RestInitializeExtensionAction Signed-off-by: Owais Kazi * Pulled extensions REST request in extensions directory Signed-off-by: Owais Kazi * Removed forbidden APIs from rest action and modified tests Signed-off-by: Owais Kazi * Added entry in changelog Signed-off-by: Owais Kazi * Added test for parse Signed-off-by: Owais Kazi * Addressed PR comments Signed-off-by: Owais Kazi * Addressed PR comments Signed-off-by: Owais Kazi * Spotless Fixed Signed-off-by: Owais Kazi * Handled exceptions Signed-off-by: Owais Kazi * Handled test failure Signed-off-by: Owais Kazi --------- Signed-off-by: Owais Kazi --- CHANGELOG.md | 1 + .../org/opensearch/action/ActionModule.java | 14 +- .../org/opensearch/bootstrap/Security.java | 1 - .../java/org/opensearch/env/Environment.java | 4 - .../extensions/ExtensionDependency.java | 39 +++ .../extensions/ExtensionsManager.java | 234 ++++---------- .../extensions/NoopExtensionsManager.java | 8 +- .../rest/RestActionsRequestHandler.java | 1 - .../rest/RestInitializeExtensionAction.java | 141 +++++++++ .../rest}/RestSendToExtensionAction.java | 5 +- .../main/java/org/opensearch/node/Node.java | 6 +- .../rest/extensions/package-info.java | 10 - .../opensearch/action/ActionModuleTests.java | 9 +- .../action/DynamicActionRegistryTests.java | 2 +- .../extensions/ExtensionsManagerTests.java | 294 +++++++++--------- .../RestInitializeExtensionActionTests.java | 124 ++++++++ .../rest/RestSendToExtensionActionTests.java | 5 +- 17 files changed, 535 insertions(+), 363 deletions(-) create mode 100644 server/src/main/java/org/opensearch/extensions/rest/RestInitializeExtensionAction.java rename server/src/main/java/org/opensearch/{rest/extensions => extensions/rest}/RestSendToExtensionAction.java (98%) delete mode 100644 server/src/main/java/org/opensearch/rest/extensions/package-info.java create mode 100644 server/src/test/java/org/opensearch/extensions/rest/RestInitializeExtensionActionTests.java diff --git a/CHANGELOG.md b/CHANGELOG.md index cb67e56c7845f..75287db6b4f07 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -82,6 +82,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Add descending order search optimization through reverse segment read. ([#7967](https://github.com/opensearch-project/OpenSearch/pull/7967)) - Update components of segrep backpressure to support remote store. ([#8020](https://github.com/opensearch-project/OpenSearch/pull/8020)) - Make remote cluster connection setup in async ([#8038](https://github.com/opensearch-project/OpenSearch/pull/8038)) +- Add API to initialize extensions ([#8029]()https://github.com/opensearch-project/OpenSearch/pull/8029) ### Dependencies - Bump `com.azure:azure-storage-common` from 12.21.0 to 12.21.1 (#7566, #7814) diff --git a/server/src/main/java/org/opensearch/action/ActionModule.java b/server/src/main/java/org/opensearch/action/ActionModule.java index 168fbae84fdf4..902ae7cc54e3f 100644 --- a/server/src/main/java/org/opensearch/action/ActionModule.java +++ b/server/src/main/java/org/opensearch/action/ActionModule.java @@ -294,8 +294,10 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.settings.SettingsFilter; import org.opensearch.common.util.FeatureFlags; +import org.opensearch.extensions.ExtensionsManager; import org.opensearch.extensions.action.ExtensionProxyAction; import org.opensearch.extensions.action.ExtensionProxyTransportAction; +import org.opensearch.extensions.rest.RestInitializeExtensionAction; import org.opensearch.index.seqno.RetentionLeaseActions; import org.opensearch.identity.IdentityService; import org.opensearch.indices.SystemIndices; @@ -453,7 +455,7 @@ import org.opensearch.rest.action.search.RestPutSearchPipelineAction; import org.opensearch.rest.action.search.RestSearchAction; import org.opensearch.rest.action.search.RestSearchScrollAction; -import org.opensearch.rest.extensions.RestSendToExtensionAction; +import org.opensearch.extensions.rest.RestSendToExtensionAction; import org.opensearch.tasks.Task; import org.opensearch.threadpool.ThreadPool; import org.opensearch.usage.UsageService; @@ -508,6 +510,7 @@ public class ActionModule extends AbstractModule { private final RequestValidators mappingRequestValidators; private final RequestValidators indicesAliasesRequestRequestValidators; private final ThreadPool threadPool; + private final ExtensionsManager extensionsManager; public ActionModule( Settings settings, @@ -521,7 +524,8 @@ public ActionModule( CircuitBreakerService circuitBreakerService, UsageService usageService, SystemIndices systemIndices, - IdentityService identityService + IdentityService identityService, + ExtensionsManager extensionsManager ) { this.settings = settings; this.indexNameExpressionResolver = indexNameExpressionResolver; @@ -530,6 +534,7 @@ public ActionModule( this.settingsFilter = settingsFilter; this.actionPlugins = actionPlugins; this.threadPool = threadPool; + this.extensionsManager = extensionsManager; actions = setupActions(actionPlugins); actionFilters = setupActionFilters(actionPlugins); dynamicActionRegistry = new DynamicActionRegistry(); @@ -947,6 +952,11 @@ public void initRestHandlers(Supplier nodesInCluster) { registerHandler.accept(new RestDeleteSearchPipelineAction()); } + // Extensions API + if (FeatureFlags.isEnabled(FeatureFlags.EXTENSIONS)) { + registerHandler.accept(new RestInitializeExtensionAction(extensionsManager)); + } + for (ActionPlugin plugin : actionPlugins) { for (RestHandler handler : plugin.getRestHandlers( settings, diff --git a/server/src/main/java/org/opensearch/bootstrap/Security.java b/server/src/main/java/org/opensearch/bootstrap/Security.java index 37c7fd5c0a96c..749c146de4f16 100644 --- a/server/src/main/java/org/opensearch/bootstrap/Security.java +++ b/server/src/main/java/org/opensearch/bootstrap/Security.java @@ -316,7 +316,6 @@ static void addFilePermissions(Permissions policy, Environment environment) thro addDirectoryPath(policy, Environment.PATH_HOME_SETTING.getKey(), environment.libDir(), "read,readlink", false); addDirectoryPath(policy, Environment.PATH_HOME_SETTING.getKey(), environment.modulesDir(), "read,readlink", false); addDirectoryPath(policy, Environment.PATH_HOME_SETTING.getKey(), environment.pluginsDir(), "read,readlink", false); - addDirectoryPath(policy, Environment.PATH_HOME_SETTING.getKey(), environment.extensionDir(), "read,readlink", false); addDirectoryPath(policy, "path.conf'", environment.configDir(), "read,readlink", false); // read-write dirs addDirectoryPath(policy, "java.io.tmpdir", environment.tmpDir(), "read,readlink,write,delete", false); diff --git a/server/src/main/java/org/opensearch/env/Environment.java b/server/src/main/java/org/opensearch/env/Environment.java index 938bca58c7081..a1e467ad1ba48 100644 --- a/server/src/main/java/org/opensearch/env/Environment.java +++ b/server/src/main/java/org/opensearch/env/Environment.java @@ -311,10 +311,6 @@ public Path pluginsDir() { return pluginsDir; } - public Path extensionDir() { - return extensionsDir; - } - public Path binDir() { return binDir; } diff --git a/server/src/main/java/org/opensearch/extensions/ExtensionDependency.java b/server/src/main/java/org/opensearch/extensions/ExtensionDependency.java index d2106cf8d399c..1423a30bbe307 100644 --- a/server/src/main/java/org/opensearch/extensions/ExtensionDependency.java +++ b/server/src/main/java/org/opensearch/extensions/ExtensionDependency.java @@ -16,6 +16,10 @@ 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.XContentParser; + +import static org.opensearch.common.xcontent.XContentParserUtils.ensureExpectedToken; /** * This class handles the dependent extensions information @@ -25,6 +29,8 @@ public class ExtensionDependency implements Writeable { private String uniqueId; private Version version; + private static final String UNIQUE_ID = "uniqueId"; + private static final String VERSION = "version"; public ExtensionDependency(String uniqueId, Version version) { this.uniqueId = uniqueId; @@ -54,6 +60,39 @@ public void writeTo(StreamOutput out) throws IOException { out.writeVersion(version); } + public static ExtensionDependency parse(XContentParser parser) throws IOException { + String uniqueId = null; + Version version = null; + ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.currentToken(), parser); + while (parser.nextToken() != XContentParser.Token.END_OBJECT) { + String fieldName = parser.currentName(); + parser.nextToken(); + + switch (fieldName) { + case UNIQUE_ID: + uniqueId = parser.text(); + break; + case VERSION: + try { + version = Version.fromString(parser.text()); + } catch (IllegalArgumentException e) { + throw e; + } + break; + default: + parser.skipChildren(); + break; + } + } + if (Strings.isNullOrEmpty(uniqueId)) { + throw new IOException("Required field [uniqueId] is missing in the request for the dependent extension"); + } else if (version == null) { + throw new IOException("Required field [version] is missing in the request for the dependent extension"); + } + return new ExtensionDependency(uniqueId, version); + + } + /** * The uniqueId of the dependency extension * diff --git a/server/src/main/java/org/opensearch/extensions/ExtensionsManager.java b/server/src/main/java/org/opensearch/extensions/ExtensionsManager.java index 5c6b9f3141aa0..9987497b5fac0 100644 --- a/server/src/main/java/org/opensearch/extensions/ExtensionsManager.java +++ b/server/src/main/java/org/opensearch/extensions/ExtensionsManager.java @@ -9,29 +9,18 @@ package org.opensearch.extensions; import java.io.IOException; -import java.io.InputStream; import java.net.InetAddress; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.ArrayList; -import java.util.Collection; import java.util.HashMap; import java.util.HashSet; -import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CompletionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; -import java.util.Arrays; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; -import org.opensearch.OpenSearchException; import org.opensearch.Version; import org.opensearch.action.ActionModule; import org.opensearch.action.ActionModule.DynamicActionRegistry; @@ -40,7 +29,8 @@ import org.opensearch.cluster.ClusterSettingsResponse; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.settings.Setting; -import org.opensearch.core.util.FileSystemUtils; +import org.opensearch.core.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; @@ -65,7 +55,6 @@ import org.opensearch.transport.TransportResponse; import org.opensearch.transport.TransportResponseHandler; import org.opensearch.transport.TransportService; -import org.yaml.snakeyaml.Yaml; import org.opensearch.env.EnvironmentSettingsResponse; /** @@ -100,7 +89,6 @@ public static enum OpenSearchRequestType { REQUEST_OPENSEARCH_NAMED_WRITEABLE_REGISTRY } - private final Path extensionsPath; private ExtensionTransportActionsHandler extensionTransportActionsHandler; private Map extensionSettingsMap; private Map initializedExtensions; @@ -117,13 +105,11 @@ public static enum OpenSearchRequestType { /** * Instantiate a new ExtensionsManager object to handle requests and responses from extensions. This is called during Node bootstrap. * - * @param extensionsPath Path to a directory containing extensions. * @param additionalSettings Additional settings to read in from extensions.yml * @throws IOException If the extensions discovery file is not properly retrieved. */ - public ExtensionsManager(Path extensionsPath, Set> additionalSettings) throws IOException { + public ExtensionsManager(Set> additionalSettings) throws IOException { logger.info("ExtensionsManager initialized"); - this.extensionsPath = extensionsPath; this.initializedExtensions = new HashMap(); this.extensionIdMap = new HashMap(); this.extensionSettingsMap = new HashMap(); @@ -137,12 +123,6 @@ public ExtensionsManager(Path extensionsPath, Set> additionalSettings } this.client = null; this.extensionTransportActionsHandler = null; - - /* - * Now Discover extensions - */ - discover(); - } /** @@ -307,60 +287,42 @@ private void registerRequestHandler(DynamicActionRegistry dynamicActionRegistry) ); } - /* - * Load and populate all extensions + /** + * Loads a single extension + * @param extension The extension to be loaded */ - protected void discover() throws IOException { - logger.info("Loading extensions : {}", extensionsPath); - if (!FileSystemUtils.isAccessibleDirectory(extensionsPath, logger)) { - return; - } + public void loadExtension(Extension extension) throws IOException { + validateExtension(extension); + DiscoveryExtensionNode discoveryExtensionNode = new DiscoveryExtensionNode( + extension.getName(), + extension.getUniqueId(), + new TransportAddress(InetAddress.getByName(extension.getHostAddress()), Integer.parseInt(extension.getPort())), + new HashMap(), + Version.fromString(extension.getOpensearchVersion()), + Version.fromString(extension.getMinimumCompatibleVersion()), + extension.getDependencies() + ); + extensionIdMap.put(extension.getUniqueId(), discoveryExtensionNode); + extensionSettingsMap.put(extension.getUniqueId(), extension); + logger.info("Loaded extension with uniqueId " + extension.getUniqueId() + ": " + extension); + } - List extensions = new ArrayList(); - if (Files.exists(extensionsPath.resolve("extensions.yml"))) { - try { - extensions = readFromExtensionsYml(extensionsPath.resolve("extensions.yml")).getExtensions(); - } catch (IOException e) { - throw new IOException("Could not read from extensions.yml", e); - } - for (Extension extension : extensions) { - loadExtension(extension); - } - if (!extensionIdMap.isEmpty()) { - logger.info("Loaded all extensions"); - } - } else { - logger.warn("Extensions.yml file is not present. No extensions will be loaded."); + private void validateField(String fieldName, String value) throws IOException { + if (Strings.isNullOrEmpty(value)) { + throw new IOException("Required field [" + fieldName + "] is missing in the request"); } } - /** - * Loads a single extension - * @param extension The extension to be loaded - */ - private void loadExtension(Extension extension) throws IOException { + private void validateExtension(Extension extension) throws IOException { + validateField("extension name", extension.getName()); + validateField("extension uniqueId", extension.getUniqueId()); + validateField("extension host address", extension.getHostAddress()); + validateField("extension port", extension.getPort()); + validateField("extension version", extension.getVersion()); + validateField("opensearch version", extension.getOpensearchVersion()); + validateField("minimum opensearch version", extension.getMinimumCompatibleVersion()); if (extensionIdMap.containsKey(extension.getUniqueId())) { - logger.info("Duplicate uniqueId " + extension.getUniqueId() + ". Did not load extension: " + extension); - } else { - try { - DiscoveryExtensionNode discoveryExtensionNode = new DiscoveryExtensionNode( - extension.getName(), - extension.getUniqueId(), - new TransportAddress(InetAddress.getByName(extension.getHostAddress()), Integer.parseInt(extension.getPort())), - new HashMap(), - Version.fromString(extension.getOpensearchVersion()), - Version.fromString(extension.getMinimumCompatibleVersion()), - extension.getDependencies() - ); - - extensionIdMap.put(extension.getUniqueId(), discoveryExtensionNode); - extensionSettingsMap.put(extension.getUniqueId(), extension); - logger.info("Loaded extension with uniqueId " + extension.getUniqueId() + ": " + extension); - } catch (OpenSearchException e) { - logger.error("Could not load extension with uniqueId " + extension.getUniqueId() + " due to " + e); - } catch (IllegalArgumentException e) { - throw e; - } + throw new IOException("Duplicate uniqueId [" + extension.getUniqueId() + "]. Did not load extension: " + extension); } } @@ -408,27 +370,35 @@ public String executor() { return ThreadPool.Names.GENERIC; } }; - try { - logger.info("Sending extension request type: " + REQUEST_EXTENSION_ACTION_NAME); - transportService.connectToExtensionNode(extension); - transportService.sendRequest( - extension, - REQUEST_EXTENSION_ACTION_NAME, - new InitializeExtensionRequest(transportService.getLocalNode(), extension), - initializeExtensionResponseHandler - ); - inProgressFuture.orTimeout(EXTENSION_REQUEST_WAIT_TIMEOUT, TimeUnit.SECONDS).join(); - } catch (CompletionException | ConnectTransportException e) { - if (e.getCause() instanceof TimeoutException || e instanceof ConnectTransportException) { - logger.info("No response from extension to request.", e); - } else if (e.getCause() instanceof RuntimeException) { - throw (RuntimeException) e.getCause(); - } else if (e.getCause() instanceof Error) { - throw (Error) e.getCause(); - } else { - throw new RuntimeException(e.getCause()); + + logger.info("Sending extension request type: " + REQUEST_EXTENSION_ACTION_NAME); + transportService.getThreadPool().generic().execute(new AbstractRunnable() { + @Override + public void onFailure(Exception e) { + extensionIdMap.remove(extension.getId()); + if (e.getCause() instanceof ConnectTransportException) { + logger.info("No response from extension to request.", e); + throw (ConnectTransportException) e.getCause(); + } else if (e.getCause() instanceof RuntimeException) { + throw (RuntimeException) e.getCause(); + } else if (e.getCause() instanceof Error) { + throw (Error) e.getCause(); + } else { + throw new RuntimeException(e.getCause()); + } } - } + + @Override + protected void doRun() throws Exception { + transportService.connectToExtensionNode(extension); + transportService.sendRequest( + extension, + REQUEST_EXTENSION_ACTION_NAME, + new InitializeExtensionRequest(transportService.getLocalNode(), extension), + initializeExtensionResponseHandler + ); + } + }); } /** @@ -467,84 +437,6 @@ TransportResponse handleExtensionRequest(ExtensionRequest extensionRequest) thro } } - private ExtensionsSettings readFromExtensionsYml(Path filePath) throws IOException { - Yaml yaml = new Yaml(); - try (InputStream inputStream = Files.newInputStream(filePath)) { - Map obj = yaml.load(inputStream); - if (obj == null) { - inputStream.close(); - throw new IOException("extensions.yml is empty"); - } - List> unreadExtensions = new ArrayList<>((Collection>) obj.get("extensions")); - List readExtensions = new ArrayList(); - Set additionalSettingsKeys = additionalSettings.stream().map(s -> s.getKey()).collect(Collectors.toSet()); - for (HashMap extensionMap : unreadExtensions) { - try { - // checking to see whether any required fields are missing from extension.yml file or not - String[] requiredFields = { - "name", - "uniqueId", - "hostAddress", - "port", - "version", - "opensearchVersion", - "minimumCompatibleVersion" }; - List missingFields = Arrays.stream(requiredFields) - .filter(field -> !extensionMap.containsKey(field)) - .collect(Collectors.toList()); - if (!missingFields.isEmpty()) { - throw new IOException("Extension is missing these required fields : " + missingFields); - } - - // Parse extension dependencies - List extensionDependencyList = new ArrayList(); - if (extensionMap.get("dependencies") != null) { - List> extensionDependencies = new ArrayList<>( - (Collection>) extensionMap.get("dependencies") - ); - for (HashMap dependency : extensionDependencies) { - extensionDependencyList.add( - new ExtensionDependency( - dependency.get("uniqueId").toString(), - Version.fromString(dependency.get("version").toString()) - ) - ); - } - } - - ExtensionScopedSettings extAdditionalSettings = new ExtensionScopedSettings(additionalSettings); - Map additionalSettingsMap = extensionMap.entrySet() - .stream() - .filter(kv -> additionalSettingsKeys.contains(kv.getKey())) - .collect(Collectors.toMap(map -> map.getKey(), map -> map.getValue())); - - Settings.Builder output = Settings.builder(); - output.loadFromMap(additionalSettingsMap); - extAdditionalSettings.applySettings(output.build()); - - // Create extension read from yml config - readExtensions.add( - new Extension( - extensionMap.get("name").toString(), - extensionMap.get("uniqueId").toString(), - extensionMap.get("hostAddress").toString(), - extensionMap.get("port").toString(), - extensionMap.get("version").toString(), - extensionMap.get("opensearchVersion").toString(), - extensionMap.get("minimumCompatibleVersion").toString(), - extensionDependencyList, - extAdditionalSettings - ) - ); - } catch (IOException e) { - logger.warn("loading extension has been failed because of exception : " + e.getMessage()); - } - } - inputStream.close(); - return new ExtensionsSettings(readExtensions); - } - } - static String getRequestExtensionActionName() { return REQUEST_EXTENSION_ACTION_NAME; } @@ -561,10 +453,6 @@ static Logger getLogger() { return logger; } - Path getExtensionsPath() { - return extensionsPath; - } - TransportService getTransportService() { return transportService; } diff --git a/server/src/main/java/org/opensearch/extensions/NoopExtensionsManager.java b/server/src/main/java/org/opensearch/extensions/NoopExtensionsManager.java index fb7160bc1bc67..d434074279041 100644 --- a/server/src/main/java/org/opensearch/extensions/NoopExtensionsManager.java +++ b/server/src/main/java/org/opensearch/extensions/NoopExtensionsManager.java @@ -9,7 +9,6 @@ package org.opensearch.extensions; import java.io.IOException; -import java.nio.file.Path; import java.util.Optional; import java.util.Set; @@ -32,7 +31,7 @@ public class NoopExtensionsManager extends ExtensionsManager { public NoopExtensionsManager() throws IOException { - super(Path.of(""), Set.of()); + super(Set.of()); } @Override @@ -59,11 +58,6 @@ public ExtensionActionResponse handleTransportRequest(ExtensionActionRequest req return new ExtensionActionResponse(new byte[0]); } - @Override - protected void discover() throws IOException { - // no-op - } - @Override public void initialize() { // no-op diff --git a/server/src/main/java/org/opensearch/extensions/rest/RestActionsRequestHandler.java b/server/src/main/java/org/opensearch/extensions/rest/RestActionsRequestHandler.java index 37638f2a333d5..d890c1b85bb81 100644 --- a/server/src/main/java/org/opensearch/extensions/rest/RestActionsRequestHandler.java +++ b/server/src/main/java/org/opensearch/extensions/rest/RestActionsRequestHandler.java @@ -13,7 +13,6 @@ import org.opensearch.extensions.DiscoveryExtensionNode; import org.opensearch.rest.RestController; import org.opensearch.rest.RestHandler; -import org.opensearch.rest.extensions.RestSendToExtensionAction; import org.opensearch.transport.TransportResponse; import org.opensearch.transport.TransportService; diff --git a/server/src/main/java/org/opensearch/extensions/rest/RestInitializeExtensionAction.java b/server/src/main/java/org/opensearch/extensions/rest/RestInitializeExtensionAction.java new file mode 100644 index 0000000000000..e0806f8172278 --- /dev/null +++ b/server/src/main/java/org/opensearch/extensions/rest/RestInitializeExtensionAction.java @@ -0,0 +1,141 @@ +/* + * 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.extensions.rest; + +import org.opensearch.client.node.NodeClient; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParser; +import org.opensearch.extensions.ExtensionDependency; +import org.opensearch.extensions.ExtensionScopedSettings; +import org.opensearch.extensions.ExtensionsManager; +import org.opensearch.extensions.ExtensionsSettings.Extension; +import org.opensearch.rest.BaseRestHandler; +import org.opensearch.rest.BytesRestResponse; +import org.opensearch.rest.RestRequest; +import org.opensearch.rest.RestStatus; +import org.opensearch.transport.ConnectTransportException; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CompletionException; +import java.util.concurrent.TimeoutException; + +import static org.opensearch.common.xcontent.XContentParserUtils.ensureExpectedToken; +import static org.opensearch.rest.RestRequest.Method.POST; + +/** + * An action that initializes an extension + */ +public class RestInitializeExtensionAction extends BaseRestHandler { + + private final ExtensionsManager extensionsManager; + + @Override + public String getName() { + return ExtensionsManager.REQUEST_EXTENSION_ACTION_NAME; + } + + @Override + public List routes() { + return List.of(new Route(POST, "/_extensions/initialize")); + } + + public RestInitializeExtensionAction(ExtensionsManager extensionsManager) { + this.extensionsManager = extensionsManager; + } + + @Override + public RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) throws IOException { + String name = null; + String uniqueId = null; + String hostAddress = null; + String port = null; + String version = null; + String openSearchVersion = null; + String minimumCompatibleVersion = null; + List dependencies = new ArrayList<>(); + + try (XContentParser parser = request.contentParser()) { + parser.nextToken(); + ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.currentToken(), parser); + while (parser.nextToken() != XContentParser.Token.END_OBJECT) { + String currentFieldName = parser.currentName(); + parser.nextToken(); + if ("name".equals(currentFieldName)) { + name = parser.text(); + } else if ("uniqueId".equals(currentFieldName)) { + uniqueId = parser.text(); + } else if ("hostAddress".equals(currentFieldName)) { + hostAddress = parser.text(); + } else if ("port".equals(currentFieldName)) { + port = parser.text(); + } else if ("version".equals(currentFieldName)) { + version = parser.text(); + } else if ("opensearchVersion".equals(currentFieldName)) { + openSearchVersion = parser.text(); + } else if ("minimumCompatibleVersion".equals(currentFieldName)) { + minimumCompatibleVersion = parser.text(); + } else if ("dependencies".equals(currentFieldName)) { + ensureExpectedToken(XContentParser.Token.START_ARRAY, parser.currentToken(), parser); + while (parser.nextToken() != XContentParser.Token.END_ARRAY) { + dependencies.add(ExtensionDependency.parse(parser)); + } + } + } + } catch (IOException e) { + throw new IOException("Missing attribute", e); + } + + Extension extension = new Extension( + name, + uniqueId, + hostAddress, + port, + version, + openSearchVersion, + minimumCompatibleVersion, + dependencies, + // TODO add this to the API (https://github.com/opensearch-project/OpenSearch/issues/8032) + new ExtensionScopedSettings(Collections.emptySet()) + ); + try { + extensionsManager.loadExtension(extension); + extensionsManager.initialize(); + } catch (CompletionException e) { + Throwable cause = e.getCause(); + if (cause instanceof TimeoutException) { + return channel -> channel.sendResponse( + new BytesRestResponse(RestStatus.REQUEST_TIMEOUT, "No response from extension to request.") + ); + } else if (cause instanceof ConnectTransportException || cause instanceof RuntimeException) { + return channel -> channel.sendResponse( + new BytesRestResponse(RestStatus.REQUEST_TIMEOUT, "Connection failed with the extension.") + ); + } + if (e.getCause() instanceof Error) { + throw (Error) e.getCause(); + } + } catch (Exception e) { + return channel -> channel.sendResponse(new BytesRestResponse(RestStatus.INTERNAL_SERVER_ERROR, e.getMessage())); + + } + + return channel -> { + try (XContentBuilder builder = channel.newBuilder()) { + builder.startObject(); + builder.field("success", "A request to initialize an extension has been sent."); + builder.endObject(); + channel.sendResponse(new BytesRestResponse(RestStatus.ACCEPTED, builder)); + } + }; + + } +} diff --git a/server/src/main/java/org/opensearch/rest/extensions/RestSendToExtensionAction.java b/server/src/main/java/org/opensearch/extensions/rest/RestSendToExtensionAction.java similarity index 98% rename from server/src/main/java/org/opensearch/rest/extensions/RestSendToExtensionAction.java rename to server/src/main/java/org/opensearch/extensions/rest/RestSendToExtensionAction.java index 51ff74b1869a0..073b3f3f45818 100644 --- a/server/src/main/java/org/opensearch/rest/extensions/RestSendToExtensionAction.java +++ b/server/src/main/java/org/opensearch/extensions/rest/RestSendToExtensionAction.java @@ -6,7 +6,7 @@ * compatible open source license. */ -package org.opensearch.rest.extensions; +package org.opensearch.extensions.rest; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -17,9 +17,6 @@ import org.opensearch.common.xcontent.XContentType; import org.opensearch.extensions.DiscoveryExtensionNode; import org.opensearch.extensions.ExtensionsManager; -import org.opensearch.extensions.rest.ExtensionRestRequest; -import org.opensearch.extensions.rest.RegisterRestActionsRequest; -import org.opensearch.extensions.rest.RestExecuteOnExtensionResponse; import org.opensearch.rest.BaseRestHandler; import org.opensearch.rest.BytesRestResponse; import org.opensearch.rest.NamedRoute; diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index 808c054de4969..688f2d05b203b 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -475,7 +475,7 @@ protected Node( for (ExtensionAwarePlugin extAwarePlugin : extensionAwarePlugins) { additionalSettings.addAll(extAwarePlugin.getExtensionSettings()); } - this.extensionsManager = new ExtensionsManager(initialEnvironment.extensionDir(), additionalSettings); + this.extensionsManager = new ExtensionsManager(additionalSettings); } else { this.extensionsManager = new NoopExtensionsManager(); } @@ -803,7 +803,8 @@ protected Node( circuitBreakerService, usageService, systemIndices, - identityService + identityService, + extensionsManager ); modules.add(actionModule); @@ -1306,7 +1307,6 @@ public Node start() throws NodeValidationException { assert clusterService.localNode().equals(localNodeFactory.getNode()) : "clusterService has a different local node than the factory provided"; transportService.acceptIncomingRequests(); - extensionsManager.initialize(); discovery.startInitialJoin(); final TimeValue initialStateTimeout = DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.get(settings()); configureNodeAndClusterIdStateListener(clusterService); diff --git a/server/src/main/java/org/opensearch/rest/extensions/package-info.java b/server/src/main/java/org/opensearch/rest/extensions/package-info.java deleted file mode 100644 index 64b92e8b5c149..0000000000000 --- a/server/src/main/java/org/opensearch/rest/extensions/package-info.java +++ /dev/null @@ -1,10 +0,0 @@ -/* - * 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. - */ - -/** REST classes for the extensions package. OpenSearch extensions provide extensibility to OpenSearch.*/ -package org.opensearch.rest.extensions; diff --git a/server/src/test/java/org/opensearch/action/ActionModuleTests.java b/server/src/test/java/org/opensearch/action/ActionModuleTests.java index 66af9ebfd814f..109c60aa1e4f1 100644 --- a/server/src/test/java/org/opensearch/action/ActionModuleTests.java +++ b/server/src/test/java/org/opensearch/action/ActionModuleTests.java @@ -46,6 +46,7 @@ import org.opensearch.common.settings.SettingsFilter; import org.opensearch.common.settings.SettingsModule; import org.opensearch.common.util.concurrent.ThreadContext; +import org.opensearch.extensions.ExtensionsManager; import org.opensearch.identity.IdentityService; import org.opensearch.plugins.ActionPlugin; import org.opensearch.plugins.ActionPlugin.ActionHandler; @@ -65,6 +66,7 @@ import java.io.IOException; import java.util.List; +import java.util.Set; import java.util.function.Supplier; import static java.util.Collections.emptyList; @@ -124,7 +126,7 @@ protected FakeAction() { ); } - public void testSetupRestHandlerContainsKnownBuiltin() { + public void testSetupRestHandlerContainsKnownBuiltin() throws IOException { SettingsModule settings = new SettingsModule(Settings.EMPTY); UsageService usageService = new UsageService(); ActionModule actionModule = new ActionModule( @@ -139,7 +141,8 @@ public void testSetupRestHandlerContainsKnownBuiltin() { null, usageService, null, - new IdentityService(Settings.EMPTY, new ArrayList<>()) + new IdentityService(Settings.EMPTY, new ArrayList<>()), + new ExtensionsManager(Set.of()) ); actionModule.initRestHandlers(null); // At this point the easiest way to confirm that a handler is loaded is to try to register another one on top of it and to fail @@ -196,6 +199,7 @@ public String getName() { null, usageService, null, + null, null ); Exception e = expectThrows(IllegalArgumentException.class, () -> actionModule.initRestHandlers(null)); @@ -246,6 +250,7 @@ public List getRestHandlers( null, usageService, null, + null, null ); actionModule.initRestHandlers(null); diff --git a/server/src/test/java/org/opensearch/action/DynamicActionRegistryTests.java b/server/src/test/java/org/opensearch/action/DynamicActionRegistryTests.java index a5b4f91ff1ed5..963d47df3baff 100644 --- a/server/src/test/java/org/opensearch/action/DynamicActionRegistryTests.java +++ b/server/src/test/java/org/opensearch/action/DynamicActionRegistryTests.java @@ -18,7 +18,7 @@ import org.opensearch.extensions.action.ExtensionTransportAction; import org.opensearch.rest.NamedRoute; import org.opensearch.rest.RestRequest; -import org.opensearch.rest.extensions.RestSendToExtensionAction; +import org.opensearch.extensions.rest.RestSendToExtensionAction; import org.opensearch.tasks.Task; import org.opensearch.tasks.TaskManager; import org.opensearch.test.OpenSearchTestCase; diff --git a/server/src/test/java/org/opensearch/extensions/ExtensionsManagerTests.java b/server/src/test/java/org/opensearch/extensions/ExtensionsManagerTests.java index 75a1d9ec62c82..f8ec138d8eff2 100644 --- a/server/src/test/java/org/opensearch/extensions/ExtensionsManagerTests.java +++ b/server/src/test/java/org/opensearch/extensions/ExtensionsManagerTests.java @@ -23,10 +23,7 @@ import java.io.IOException; import java.net.InetAddress; -import java.nio.charset.StandardCharsets; -import java.nio.file.Files; import java.nio.file.Path; -import java.security.AccessControlException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -40,6 +37,7 @@ import org.apache.logging.log4j.LogManager; import org.junit.After; import org.junit.Before; +import org.opensearch.OpenSearchException; import org.opensearch.Version; import org.opensearch.action.ActionModule; import org.opensearch.action.ActionModule.DynamicActionRegistry; @@ -47,12 +45,13 @@ import org.opensearch.client.node.NodeClient; import org.opensearch.cluster.ClusterSettingsResponse; import org.opensearch.common.util.FeatureFlags; +import org.opensearch.common.xcontent.json.JsonXContent; +import org.opensearch.core.xcontent.XContentParser; import org.opensearch.env.EnvironmentSettingsResponse; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.bytes.BytesReference; -import org.opensearch.common.io.PathUtils; import org.opensearch.common.io.stream.BytesStreamInput; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.io.stream.NamedWriteableRegistry; @@ -69,6 +68,7 @@ import org.opensearch.extensions.proto.ExtensionRequestProto; import org.opensearch.extensions.rest.RegisterRestActionsRequest; import org.opensearch.extensions.settings.RegisterCustomSettingsRequest; +import org.opensearch.extensions.ExtensionsSettings.Extension; import org.opensearch.identity.IdentityService; import org.opensearch.indices.breaker.NoneCircuitBreakerService; import org.opensearch.plugins.ExtensionAwarePlugin; @@ -203,10 +203,37 @@ public void tearDown() throws Exception { ThreadPool.terminate(threadPool, 30, TimeUnit.SECONDS); } - public void testDiscover() throws Exception { - Files.write(extensionDir.resolve("extensions.yml"), extensionsYmlLines, StandardCharsets.UTF_8); + public void testLoadExtensions() throws Exception { - ExtensionsManager extensionsManager = new ExtensionsManager(extensionDir, Set.of()); + Set> additionalSettings = extAwarePlugin.getExtensionSettings().stream().collect(Collectors.toSet()); + ExtensionScopedSettings extensionScopedSettings = new ExtensionScopedSettings(additionalSettings); + ExtensionsManager extensionsManager = new ExtensionsManager(additionalSettings); + ExtensionDependency dependentExtension = new ExtensionDependency("uniqueid0", Version.fromString("2.0.0")); + + Extension firstExtension = new Extension( + "firstExtension", + "uniqueid1", + "127.0.0.1", + "9300", + "0.0.7", + "3.0.0", + "3.0.0", + Collections.emptyList(), + extensionScopedSettings + ); + Extension secondExtension = new Extension( + "secondExtension", + "uniqueid2", + "127.0.0.1", + "9301", + "0.0.7", + "2.0.0", + "2.0.0", + List.of(dependentExtension), + extensionScopedSettings + ); + extensionsManager.loadExtension(firstExtension); + extensionsManager.loadExtension(secondExtension); List expectedExtensions = new ArrayList(); @@ -218,7 +245,7 @@ public void testDiscover() throws Exception { new DiscoveryExtensionNode( "firstExtension", "uniqueid1", - new TransportAddress(InetAddress.getByName("127.0.0.0"), 9300), + new TransportAddress(InetAddress.getByName("127.0.0.1"), 9300), new HashMap(), Version.fromString("3.0.0"), Version.fromString("3.0.0"), @@ -252,14 +279,37 @@ public void testDiscover() throws Exception { } } - public void testNonUniqueExtensionsDiscovery() throws Exception { - Path emptyExtensionDir = createTempDir(); - List nonUniqueYmlLines = extensionsYmlLines.stream() - .map(s -> s.replace("uniqueid2", "uniqueid1")) - .collect(Collectors.toList()); - Files.write(emptyExtensionDir.resolve("extensions.yml"), nonUniqueYmlLines, StandardCharsets.UTF_8); + public void testNonUniqueLoadedExtensions() throws Exception { - ExtensionsManager extensionsManager = new ExtensionsManager(emptyExtensionDir, Set.of()); + Extension firstExtension = new Extension( + "firstExtension", + "uniqueid1", + "127.0.0.0", + "9300", + "0.0.7", + "3.0.0", + "3.0.0", + Collections.emptyList(), + null + ); + Extension secondExtension = new Extension( + "secondExtension", + "uniqueid1", + "127.0.0.0", + "9300", + "0.0.7", + "3.0.0", + "3.0.0", + null, + null + ); + ExtensionsManager extensionsManager = new ExtensionsManager(Set.of()); + extensionsManager.loadExtension(firstExtension); + IOException exception = expectThrows(IOException.class, () -> extensionsManager.loadExtension(secondExtension)); + assertEquals( + "Duplicate uniqueId [uniqueid1]. Did not load extension: Extension [name=secondExtension, uniqueId=uniqueid1, hostAddress=127.0.0.0, port=9300, version=0.0.7, opensearchVersion=3.0.0, minimumCompatibleVersion=3.0.0]", + exception.getMessage() + ); List expectedExtensions = new ArrayList(); @@ -289,56 +339,15 @@ public void testNonUniqueExtensionsDiscovery() throws Exception { assertTrue(expectedExtensions.containsAll(emptyList())); } - public void testMissingRequiredFieldsInExtensionDiscovery() throws Exception { - Path emptyExtensionDir = createTempDir(); - ExtensionsManager extensionsManager; - List requiredFieldMissingYmlLines = extensionsYmlLines.stream() - .map(s -> s.replace(" minimumCompatibleVersion: '2.0.0'", "")) - .collect(Collectors.toList()); - Files.write(emptyExtensionDir.resolve("extensions.yml"), requiredFieldMissingYmlLines, StandardCharsets.UTF_8); + public void testMissingRequiredFieldsWhileLoadingExtension() throws Exception { - try (MockLogAppender mockLogAppender = MockLogAppender.createForLoggers(LogManager.getLogger(ExtensionsManager.class))) { + Extension firstExtension = new Extension("firstExtension", "uniqueid1", "127.0.0.0", "9300", "0.0.7", "3.0.0", "", null, null); + ExtensionsManager extensionsManager = new ExtensionsManager(Set.of()); - mockLogAppender.addExpectation( - new MockLogAppender.SeenEventExpectation( - "Required field is missing in extensions.yml", - "org.opensearch.extensions.ExtensionsManager", - Level.WARN, - "loading extension has been failed because of exception : Extension is missing these required fields : [minimumCompatibleVersion]" - ) - ); - - extensionsManager = new ExtensionsManager(emptyExtensionDir, Set.of()); + IOException exception = expectThrows(IOException.class, () -> extensionsManager.loadExtension(firstExtension)); + assertEquals("Required field [minimum opensearch version] is missing in the request", exception.getMessage()); - mockLogAppender.assertAllExpectationsMatched(); - } - - List expectedExtensions = new ArrayList(); - - expectedExtensions.add( - new DiscoveryExtensionNode( - "firstExtension", - "uniqueid1", - new TransportAddress(InetAddress.getByName("127.0.0.0"), 9300), - new HashMap(), - Version.fromString("3.0.0"), - Version.fromString("3.0.0"), - Collections.emptyList() - ) - ); - assertEquals(expectedExtensions.size(), extensionsManager.getExtensionIdMap().values().size()); - for (DiscoveryExtensionNode extension : expectedExtensions) { - DiscoveryExtensionNode initializedExtension = extensionsManager.getExtensionIdMap().get(extension.getId()); - assertEquals(extension.getName(), initializedExtension.getName()); - assertEquals(extension.getId(), initializedExtension.getId()); - assertEquals(extension.getAddress(), initializedExtension.getAddress()); - assertEquals(extension.getAttributes(), initializedExtension.getAttributes()); - assertEquals(extension.getVersion(), initializedExtension.getVersion()); - assertEquals(extension.getMinimumCompatibleVersion(), initializedExtension.getMinimumCompatibleVersion()); - assertEquals(extension.getDependencies(), initializedExtension.getDependencies()); - } - assertTrue(expectedExtensions.containsAll(emptyList())); - assertTrue(expectedExtensions.containsAll(emptyList())); + assertEquals(0, extensionsManager.getExtensionIdMap().values().size()); } public void testDiscoveryExtension() throws Exception { @@ -389,49 +398,18 @@ public void testExtensionDependency() throws Exception { } } - public void testNonAccessibleDirectory() throws Exception { - AccessControlException e = expectThrows( + public void testParseExtensionDependency() throws Exception { + XContentParser parser = createParser(JsonXContent.jsonXContent, "{\"uniqueId\": \"test1\", \"version\": \"2.0.0\"}"); - AccessControlException.class, - () -> new ExtensionsManager(PathUtils.get(""), Set.of()) - ); - assertEquals("access denied (\"java.io.FilePermission\" \"\" \"read\")", e.getMessage()); - } - - public void testNoExtensionsFile() throws Exception { - Settings settings = Settings.builder().build(); - - try (MockLogAppender mockLogAppender = MockLogAppender.createForLoggers(LogManager.getLogger(ExtensionsManager.class))) { - - mockLogAppender.addExpectation( - new MockLogAppender.SeenEventExpectation( - "No Extensions File Present", - "org.opensearch.extensions.ExtensionsManager", - Level.WARN, - "Extensions.yml file is not present. No extensions will be loaded." - ) - ); + assertEquals(XContentParser.Token.START_OBJECT, parser.nextToken()); + ExtensionDependency dependency = ExtensionDependency.parse(parser); - new ExtensionsManager(extensionDir, Set.of()); - - mockLogAppender.assertAllExpectationsMatched(); - } - } - - public void testEmptyExtensionsFile() throws Exception { - Path emptyExtensionDir = createTempDir(); - - List emptyExtensionsYmlLines = Arrays.asList(); - Files.write(emptyExtensionDir.resolve("extensions.yml"), emptyExtensionsYmlLines, StandardCharsets.UTF_8); - - Settings settings = Settings.builder().build(); - - expectThrows(IOException.class, () -> new ExtensionsManager(emptyExtensionDir, Set.of())); + assertEquals("test1", dependency.getUniqueId()); + assertEquals(Version.fromString("2.0.0"), dependency.getVersion()); } public void testInitialize() throws Exception { - Files.write(extensionDir.resolve("extensions.yml"), extensionsYmlLines, StandardCharsets.UTF_8); - ExtensionsManager extensionsManager = new ExtensionsManager(extensionDir, Set.of()); + ExtensionsManager extensionsManager = new ExtensionsManager(Set.of()); initialize(extensionsManager); @@ -472,9 +450,8 @@ public void testInitialize() throws Exception { } public void testHandleRegisterRestActionsRequest() throws Exception { - Files.write(extensionDir.resolve("extensions.yml"), extensionsYmlLines, StandardCharsets.UTF_8); - ExtensionsManager extensionsManager = new ExtensionsManager(extensionDir, Set.of()); + ExtensionsManager extensionsManager = new ExtensionsManager(Set.of()); initialize(extensionsManager); String uniqueIdStr = "uniqueid1"; @@ -488,8 +465,7 @@ public void testHandleRegisterRestActionsRequest() throws Exception { } public void testHandleRegisterSettingsRequest() throws Exception { - Files.write(extensionDir.resolve("extensions.yml"), extensionsYmlLines, StandardCharsets.UTF_8); - ExtensionsManager extensionsManager = new ExtensionsManager(extensionDir, Set.of()); + ExtensionsManager extensionsManager = new ExtensionsManager(Set.of()); initialize(extensionsManager); String uniqueIdStr = "uniqueid1"; @@ -505,7 +481,7 @@ public void testHandleRegisterSettingsRequest() throws Exception { } public void testHandleRegisterRestActionsRequestWithInvalidMethod() throws Exception { - ExtensionsManager extensionsManager = new ExtensionsManager(extensionDir, Set.of()); + ExtensionsManager extensionsManager = new ExtensionsManager(Set.of()); initialize(extensionsManager); String uniqueIdStr = "uniqueid1"; @@ -520,7 +496,7 @@ public void testHandleRegisterRestActionsRequestWithInvalidMethod() throws Excep } public void testHandleRegisterRestActionsRequestWithInvalidDeprecatedMethod() throws Exception { - ExtensionsManager extensionsManager = new ExtensionsManager(extensionDir, Set.of()); + ExtensionsManager extensionsManager = new ExtensionsManager(Set.of()); initialize(extensionsManager); String uniqueIdStr = "uniqueid1"; @@ -535,7 +511,7 @@ public void testHandleRegisterRestActionsRequestWithInvalidDeprecatedMethod() th } public void testHandleRegisterRestActionsRequestWithInvalidUri() throws Exception { - ExtensionsManager extensionsManager = new ExtensionsManager(extensionDir, Set.of()); + ExtensionsManager extensionsManager = new ExtensionsManager(Set.of()); initialize(extensionsManager); String uniqueIdStr = "uniqueid1"; List actionsList = List.of("GET", "PUT /bar", "POST /baz"); @@ -549,7 +525,7 @@ public void testHandleRegisterRestActionsRequestWithInvalidUri() throws Exceptio } public void testHandleRegisterRestActionsRequestWithInvalidDeprecatedUri() throws Exception { - ExtensionsManager extensionsManager = new ExtensionsManager(extensionDir, Set.of()); + ExtensionsManager extensionsManager = new ExtensionsManager(Set.of()); initialize(extensionsManager); String uniqueIdStr = "uniqueid1"; List actionsList = List.of("GET /foo", "PUT /bar", "POST /baz"); @@ -563,7 +539,7 @@ public void testHandleRegisterRestActionsRequestWithInvalidDeprecatedUri() throw } public void testHandleExtensionRequest() throws Exception { - ExtensionsManager extensionsManager = new ExtensionsManager(extensionDir, Set.of()); + ExtensionsManager extensionsManager = new ExtensionsManager(Set.of()); initialize(extensionsManager); ExtensionRequest clusterStateRequest = new ExtensionRequest(ExtensionRequestProto.RequestType.REQUEST_EXTENSION_CLUSTER_STATE); @@ -717,9 +693,7 @@ public void testEnvironmentSettingsDefaultValue() throws Exception { } public void testAddSettingsUpdateConsumerRequest() throws Exception { - Path extensionDir = createTempDir(); - Files.write(extensionDir.resolve("extensions.yml"), extensionsYmlLines, StandardCharsets.UTF_8); - ExtensionsManager extensionsManager = new ExtensionsManager(extensionDir, Set.of()); + ExtensionsManager extensionsManager = new ExtensionsManager(Set.of()); initialize(extensionsManager); List> componentSettings = List.of( @@ -763,10 +737,7 @@ public void testAddSettingsUpdateConsumerRequest() throws Exception { } public void testHandleAddSettingsUpdateConsumerRequest() throws Exception { - - Path extensionDir = createTempDir(); - Files.write(extensionDir.resolve("extensions.yml"), extensionsYmlLines, StandardCharsets.UTF_8); - ExtensionsManager extensionsManager = new ExtensionsManager(extensionDir, Set.of()); + ExtensionsManager extensionsManager = new ExtensionsManager(Set.of()); initialize(extensionsManager); List> componentSettings = List.of( @@ -786,9 +757,7 @@ public void testHandleAddSettingsUpdateConsumerRequest() throws Exception { } public void testUpdateSettingsRequest() throws Exception { - Path extensionDir = createTempDir(); - Files.write(extensionDir.resolve("extensions.yml"), extensionsYmlLines, StandardCharsets.UTF_8); - ExtensionsManager extensionsManager = new ExtensionsManager(extensionDir, Set.of()); + ExtensionsManager extensionsManager = new ExtensionsManager(Set.of()); initialize(extensionsManager); Setting componentSetting = Setting.boolSetting("falseSetting", false, Property.Dynamic); @@ -817,7 +786,7 @@ public void testUpdateSettingsRequest() throws Exception { public void testRegisterHandler() throws Exception { - ExtensionsManager extensionsManager = new ExtensionsManager(extensionDir, Set.of()); + ExtensionsManager extensionsManager = new ExtensionsManager(Set.of()); TransportService mockTransportService = spy( new TransportService( @@ -842,43 +811,50 @@ public void testRegisterHandler() throws Exception { } - public void testIncompatibleExtensionRegistration() throws IOException, IllegalAccessException { - - try (MockLogAppender mockLogAppender = MockLogAppender.createForLoggers(LogManager.getLogger(ExtensionsManager.class))) { - - mockLogAppender.addExpectation( - new MockLogAppender.SeenEventExpectation( - "Could not load extension with uniqueId", - "org.opensearch.extensions.ExtensionsManager", - Level.ERROR, - "Could not load extension with uniqueId uniqueid1 due to OpenSearchException[Extension minimumCompatibleVersion: 3.99.0 is greater than current" - ) - ); - - List incompatibleExtension = Arrays.asList( - "extensions:", - " - name: firstExtension", - " uniqueId: uniqueid1", - " hostAddress: '127.0.0.0'", - " port: '9300'", - " version: '0.0.7'", - " opensearchVersion: '3.0.0'", - " minimumCompatibleVersion: '3.99.0'" - ); - - Files.write(extensionDir.resolve("extensions.yml"), incompatibleExtension, StandardCharsets.UTF_8); - ExtensionsManager extensionsManager = new ExtensionsManager(extensionDir, Set.of()); - assertEquals(0, extensionsManager.getExtensionIdMap().values().size()); - mockLogAppender.assertAllExpectationsMatched(); - } + public void testIncompatibleExtensionRegistration() throws IOException { + ExtensionsManager extensionsManager = new ExtensionsManager(Set.of()); + Extension firstExtension = new Extension( + "firstExtension", + "uniqueid1", + "127.0.0.0", + "9300", + "0.0.7", + "3.0.0", + "3.99.0", + List.of(), + null + ); + expectThrows(OpenSearchException.class, () -> extensionsManager.loadExtension(firstExtension)); + assertEquals(0, extensionsManager.getExtensionIdMap().values().size()); } public void testAdditionalExtensionSettingsForExtensionWithCustomSettingSet() throws Exception { - Files.write(extensionDir.resolve("extensions.yml"), extensionsYmlLines, StandardCharsets.UTF_8); + Setting customSetting = Setting.simpleString("custom_extension_setting", "custom_setting", Property.ExtensionScope); + ExtensionAwarePlugin extAwarePlugin = new ExtensionAwarePlugin() { + @Override + public List> getExtensionSettings() { + List> settings = new ArrayList>(); + settings.add(customSetting); + return settings; + } + }; Set> additionalSettings = extAwarePlugin.getExtensionSettings().stream().collect(Collectors.toSet()); + ExtensionScopedSettings extensionScopedSettings = new ExtensionScopedSettings(additionalSettings); + Extension firstExtension = new Extension( + "firstExtension", + "uniqueid1", + "127.0.0.0", + "9300", + "0.0.7", + "3.0.0", + "3.0.0", + List.of(), + extensionScopedSettings + ); - ExtensionsManager extensionsManager = new ExtensionsManager(extensionDir, additionalSettings); + ExtensionsManager extensionsManager = new ExtensionsManager(additionalSettings); + extensionsManager.loadExtension(firstExtension); DiscoveryExtensionNode extension = new DiscoveryExtensionNode( "firstExtension", @@ -900,11 +876,23 @@ public void testAdditionalExtensionSettingsForExtensionWithCustomSettingSet() th } public void testAdditionalExtensionSettingsForExtensionWithoutCustomSettingSet() throws Exception { - Files.write(extensionDir.resolve("extensions.yml"), extensionsYmlLines, StandardCharsets.UTF_8); Set> additionalSettings = extAwarePlugin.getExtensionSettings().stream().collect(Collectors.toSet()); + ExtensionScopedSettings extensionScopedSettings = new ExtensionScopedSettings(additionalSettings); + Extension firstExtension = new Extension( + "secondExtension", + "uniqueid2", + "127.0.0.0", + "9301", + "0.0.7", + "2.0.0", + "2.0.0", + List.of(), + extensionScopedSettings + ); - ExtensionsManager extensionsManager = new ExtensionsManager(extensionDir, additionalSettings); + ExtensionsManager extensionsManager = new ExtensionsManager(additionalSettings); + extensionsManager.loadExtension(firstExtension); DiscoveryExtensionNode extension = new DiscoveryExtensionNode( "secondExtension", diff --git a/server/src/test/java/org/opensearch/extensions/rest/RestInitializeExtensionActionTests.java b/server/src/test/java/org/opensearch/extensions/rest/RestInitializeExtensionActionTests.java new file mode 100644 index 0000000000000..8d027b7fca9c2 --- /dev/null +++ b/server/src/test/java/org/opensearch/extensions/rest/RestInitializeExtensionActionTests.java @@ -0,0 +1,124 @@ +/* + * 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.extensions.rest; + +import java.util.Collections; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +import static java.util.Collections.emptyMap; +import static java.util.Collections.emptySet; +import static org.mockito.Mockito.mock; + +import org.junit.After; +import org.junit.Before; +import org.opensearch.Version; +import org.opensearch.cluster.node.DiscoveryNode; +import org.opensearch.common.bytes.BytesArray; +import org.opensearch.common.io.stream.NamedWriteableRegistry; +import org.opensearch.common.network.NetworkService; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.PageCacheRecycler; +import org.opensearch.common.xcontent.XContentType; +import org.opensearch.extensions.ExtensionsManager; +import org.opensearch.indices.breaker.NoneCircuitBreakerService; +import org.opensearch.rest.RestRequest; +import org.opensearch.rest.RestStatus; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.test.rest.FakeRestChannel; +import org.opensearch.test.rest.FakeRestRequest; +import org.opensearch.test.transport.MockTransportService; +import org.opensearch.threadpool.TestThreadPool; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.TransportService; +import org.opensearch.transport.nio.MockNioTransport; + +public class RestInitializeExtensionActionTests extends OpenSearchTestCase { + + private TransportService transportService; + private MockNioTransport transport; + private final ThreadPool threadPool = new TestThreadPool(RestInitializeExtensionActionTests.class.getSimpleName()); + + @Before + public void setup() throws Exception { + Settings settings = Settings.builder().put("cluster.name", "test").build(); + transport = new MockNioTransport( + settings, + Version.CURRENT, + threadPool, + new NetworkService(Collections.emptyList()), + PageCacheRecycler.NON_RECYCLING_INSTANCE, + new NamedWriteableRegistry(Collections.emptyList()), + new NoneCircuitBreakerService() + ); + transportService = new MockTransportService( + settings, + transport, + threadPool, + TransportService.NOOP_TRANSPORT_INTERCEPTOR, + (boundAddress) -> new DiscoveryNode( + "test_node", + "test_node", + boundAddress.publishAddress(), + emptyMap(), + emptySet(), + Version.CURRENT + ), + null, + Collections.emptySet() + ); + + } + + @Override + @After + public void tearDown() throws Exception { + super.tearDown(); + transportService.close(); + ThreadPool.terminate(threadPool, 30, TimeUnit.SECONDS); + } + + public void testRestInitializeExtensionActionResponse() throws Exception { + ExtensionsManager extensionsManager = mock(ExtensionsManager.class); + RestInitializeExtensionAction restInitializeExtensionAction = new RestInitializeExtensionAction(extensionsManager); + final String content = "{\"name\":\"ad-extension\",\"uniqueId\":\"ad-extension\",\"hostAddress\":\"127.0.0.1\"," + + "\"port\":\"4532\",\"version\":\"1.0\",\"opensearchVersion\":\"3.0.0\"," + + "\"minimumCompatibleVersion\":\"3.0.0\"}"; + RestRequest request = new FakeRestRequest.Builder(xContentRegistry()).withContent(new BytesArray(content), XContentType.JSON) + .withMethod(RestRequest.Method.POST) + .build(); + + FakeRestChannel channel = new FakeRestChannel(request, false, 0); + restInitializeExtensionAction.handleRequest(request, channel, null); + + assertEquals(channel.capturedResponse().status(), RestStatus.ACCEPTED); + assertTrue(channel.capturedResponse().content().utf8ToString().contains("A request to initialize an extension has been sent.")); + } + + public void testRestInitializeExtensionActionFailure() throws Exception { + ExtensionsManager extensionsManager = new ExtensionsManager(Set.of()); + RestInitializeExtensionAction restInitializeExtensionAction = new RestInitializeExtensionAction(extensionsManager); + + final String content = "{\"name\":\"ad-extension\",\"uniqueId\":\"\",\"hostAddress\":\"127.0.0.1\"," + + "\"port\":\"4532\",\"version\":\"1.0\",\"opensearchVersion\":\"3.0.0\"," + + "\"minimumCompatibleVersion\":\"3.0.0\"}"; + RestRequest request = new FakeRestRequest.Builder(xContentRegistry()).withContent(new BytesArray(content), XContentType.JSON) + .withMethod(RestRequest.Method.POST) + .build(); + + FakeRestChannel channel = new FakeRestChannel(request, false, 0); + restInitializeExtensionAction.handleRequest(request, channel, null); + + assertEquals(1, channel.errors().get()); + assertTrue( + channel.capturedResponse().content().utf8ToString().contains("Required field [extension uniqueId] is missing in the request") + ); + } + +} diff --git a/server/src/test/java/org/opensearch/extensions/rest/RestSendToExtensionActionTests.java b/server/src/test/java/org/opensearch/extensions/rest/RestSendToExtensionActionTests.java index df047afb677d9..fe8792b36f048 100644 --- a/server/src/test/java/org/opensearch/extensions/rest/RestSendToExtensionActionTests.java +++ b/server/src/test/java/org/opensearch/extensions/rest/RestSendToExtensionActionTests.java @@ -42,6 +42,7 @@ import org.opensearch.common.util.PageCacheRecycler; import org.opensearch.common.util.concurrent.ThreadContext; import org.opensearch.extensions.DiscoveryExtensionNode; +import org.opensearch.extensions.ExtensionsManager; import org.opensearch.extensions.action.ExtensionAction; import org.opensearch.extensions.action.ExtensionTransportAction; import org.opensearch.identity.IdentityService; @@ -49,7 +50,6 @@ import org.opensearch.rest.NamedRoute; import org.opensearch.rest.RestHandler.Route; import org.opensearch.rest.RestRequest.Method; -import org.opensearch.rest.extensions.RestSendToExtensionAction; import org.opensearch.test.OpenSearchTestCase; import org.opensearch.test.transport.MockTransportService; import org.opensearch.threadpool.TestThreadPool; @@ -118,7 +118,8 @@ public void setup() throws Exception { null, usageService, null, - new IdentityService(Settings.EMPTY, new ArrayList<>()) + new IdentityService(Settings.EMPTY, new ArrayList<>()), + new ExtensionsManager(Set.of()) ); dynamicActionRegistry = actionModule.getDynamicActionRegistry(); } From 619aeb5e5d035e15ca18fc9642d1eff941f6704f Mon Sep 17 00:00:00 2001 From: Sayali Gaikawad <61760125+gaiksaya@users.noreply.github.com> Date: Wed, 21 Jun 2023 19:47:05 -0700 Subject: [PATCH 053/109] Add a gradle task to check compability with child components (#7998) Add compatibility check gradle task ./gradlew checkCompatibility Signed-off-by: Sayali Gaikawad --- build.gradle | 13 +++ buildSrc/build.gradle | 1 + .../gradle/CheckCompatibilityTask.groovy | 108 ++++++++++++++++++ 3 files changed, 122 insertions(+) create mode 100644 buildSrc/src/main/groovy/org/opensearch/gradle/CheckCompatibilityTask.groovy diff --git a/build.gradle b/build.gradle index 4b188718e5f30..eec01e840149e 100644 --- a/build.gradle +++ b/build.gradle @@ -46,6 +46,7 @@ import org.gradle.plugins.ide.eclipse.model.EclipseJdt import org.gradle.plugins.ide.eclipse.model.SourceFolder import org.gradle.api.Project; import org.gradle.process.ExecResult; +import org.opensearch.gradle.CheckCompatibilityTask import static org.opensearch.gradle.util.GradleUtils.maybeConfigure @@ -643,3 +644,15 @@ tasks.withType(TestTaskReports).configureEach { tasks.named(JavaBasePlugin.CHECK_TASK_NAME) { dependsOn tasks.named('testAggregateTestReport', TestReport) } + +tasks.register('checkCompatibility', CheckCompatibilityTask) { + description('Checks the compatibility with child components') +} + +allprojects { project -> + project.afterEvaluate { + if (project.tasks.findByName('publishToMavenLocal')) { + checkCompatibility.dependsOn(project.tasks.publishToMavenLocal) + } + } +} diff --git a/buildSrc/build.gradle b/buildSrc/build.gradle index 314d27602b4ad..abbdc6f6a570e 100644 --- a/buildSrc/build.gradle +++ b/buildSrc/build.gradle @@ -122,6 +122,7 @@ dependencies { api 'org.jruby.jcodings:jcodings:1.0.58' api 'org.jruby.joni:joni:2.1.48' api "com.fasterxml.jackson.core:jackson-databind:${props.getProperty('jackson_databind')}" + api "org.ajoberstar.grgit:grgit-core:5.2.0" testFixturesApi "junit:junit:${props.getProperty('junit')}" testFixturesApi "com.carrotsearch.randomizedtesting:randomizedtesting-runner:${props.getProperty('randomizedrunner')}" diff --git a/buildSrc/src/main/groovy/org/opensearch/gradle/CheckCompatibilityTask.groovy b/buildSrc/src/main/groovy/org/opensearch/gradle/CheckCompatibilityTask.groovy new file mode 100644 index 0000000000000..ee6446fec6d57 --- /dev/null +++ b/buildSrc/src/main/groovy/org/opensearch/gradle/CheckCompatibilityTask.groovy @@ -0,0 +1,108 @@ +/* + * 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.gradle + +import groovy.json.JsonSlurper +import org.ajoberstar.grgit.Grgit +import org.ajoberstar.grgit.operation.BranchListOp +import org.gradle.api.DefaultTask +import org.gradle.api.tasks.Input +import org.gradle.api.tasks.Internal +import org.gradle.api.tasks.TaskAction +import org.gradle.internal.os.OperatingSystem + +import java.nio.file.Paths + +class CheckCompatibilityTask extends DefaultTask { + + static final String REPO_URL = 'https://raw.githubusercontent.com/opensearch-project/opensearch-plugins/main/plugins/.meta' + + @Input + List repositoryUrls = project.hasProperty('repositoryUrls') ? project.property('repositoryUrls').split(',') : getRepoUrls() + + @Input + String ref = project.hasProperty('ref') ? project.property('ref') : 'main' + + @Internal + List failedComponents = [] + + @Internal + List gitFailedComponents = [] + + @Internal + List compatibleComponents = [] + + @TaskAction + void checkCompatibility() { + logger.info("Checking compatibility for: $repositoryUrls for $ref") + repositoryUrls.parallelStream().forEach { repositoryUrl -> + def tempDir = File.createTempDir() + try { + if (cloneAndCheckout(repositoryUrl, tempDir)) { + if (repositoryUrl.toString().endsWithAny('notifications', 'notifications.git')) { + tempDir = Paths.get(tempDir.getAbsolutePath(), 'notifications') + } + project.exec { + workingDir = tempDir + def stdout = new ByteArrayOutputStream() + executable = (OperatingSystem.current().isWindows()) ? 'gradlew.bat' : './gradlew' + args 'assemble' + standardOutput stdout + } + compatibleComponents.add(repositoryUrl) + } else { + logger.lifecycle("Skipping compatibility check for $repositoryUrl") + } + } catch (ex) { + failedComponents.add(repositoryUrl) + logger.info("Gradle assemble failed for $repositoryUrl", ex) + } finally { + tempDir.deleteDir() + } + } + if (!failedComponents.isEmpty()) { + logger.lifecycle("Incompatible components: $failedComponents") + logger.info("Compatible components: $compatibleComponents") + } + if (!gitFailedComponents.isEmpty()) { + logger.lifecycle("Components skipped due to git failures: $gitFailedComponents") + logger.info("Compatible components: $compatibleComponents") + } + if (!compatibleComponents.isEmpty()) { + logger.lifecycle("Compatible components: $compatibleComponents") + } + } + + protected static List getRepoUrls() { + def json = new JsonSlurper().parse(REPO_URL.toURL()) + def labels = json.projects.values() + return labels as List + } + + protected boolean cloneAndCheckout(repoUrl, directory) { + try { + def grgit = Grgit.clone(dir: directory, uri: repoUrl) + def remoteBranches = grgit.branch.list(mode: BranchListOp.Mode.REMOTE) + String targetBranch = 'origin/' + ref + if (remoteBranches.find { it.name == targetBranch } == null) { + gitFailedComponents.add(repoUrl) + logger.info("$ref does not exist for $repoUrl. Skipping the compatibility check!!") + return false + } else { + logger.info("Checking out $targetBranch") + grgit.checkout(branch: targetBranch) + return true + } + } catch (ex) { + logger.error('Exception occurred during GitHub operations', ex) + gitFailedComponents.add(repoUrl) + return false + } + } +} From 63dc6aa795f8ec78597c7f93e732b979b6963d8f Mon Sep 17 00:00:00 2001 From: Stephen Crawford <65832608+scrawfor99@users.noreply.github.com> Date: Thu, 22 Jun 2023 08:27:15 -0400 Subject: [PATCH 054/109] Fix #7972 SearchBackpressureIT flaky tests (#8063) * fix thread issue Signed-off-by: Stephen Crawford * fix thread issue Signed-off-by: Stephen Crawford * Fix thresholds Signed-off-by: Stephen Crawford * Swap to object based Signed-off-by: Stephen Crawford * Spotless Signed-off-by: Stephen Crawford * Swap to preserve nulls Signed-off-by: Stephen Crawford * Spotless Signed-off-by: Stephen Crawford * Resolve npe Signed-off-by: Stephen Crawford * remove final declerations Signed-off-by: Stephen Crawford * spotless Signed-off-by: Stephen Crawford * add annotations Signed-off-by: Stephen Crawford * push to rerun tests Signed-off-by: Stephen Crawford * Fix idea Signed-off-by: Stephen Crawford * Fix idea Signed-off-by: Stephen Crawford --------- Signed-off-by: Stephen Crawford --- .../backpressure/SearchBackpressureIT.java | 4 +- .../org/opensearch/tasks/CancellableTask.java | 85 +++++++++++-------- 2 files changed, 50 insertions(+), 39 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/search/backpressure/SearchBackpressureIT.java b/server/src/internalClusterTest/java/org/opensearch/search/backpressure/SearchBackpressureIT.java index 1decd69ead7e3..a63c3287ea124 100644 --- a/server/src/internalClusterTest/java/org/opensearch/search/backpressure/SearchBackpressureIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/search/backpressure/SearchBackpressureIT.java @@ -151,7 +151,7 @@ public void testSearchShardTaskCancellationWithHighElapsedTime() throws Interrup public void testSearchTaskCancellationWithHighCpu() throws InterruptedException { Settings request = Settings.builder() .put(SearchBackpressureSettings.SETTING_MODE.getKey(), "enforced") - .put(SearchTaskSettings.SETTING_CPU_TIME_MILLIS_THRESHOLD.getKey(), 50) + .put(SearchTaskSettings.SETTING_CPU_TIME_MILLIS_THRESHOLD.getKey(), 1000) .build(); assertAcked(client().admin().cluster().prepareUpdateSettings().setPersistentSettings(request).get()); @@ -182,7 +182,7 @@ public void testSearchTaskCancellationWithHighCpu() throws InterruptedException public void testSearchShardTaskCancellationWithHighCpu() throws InterruptedException { Settings request = Settings.builder() .put(SearchBackpressureSettings.SETTING_MODE.getKey(), "enforced") - .put(SearchShardTaskSettings.SETTING_CPU_TIME_MILLIS_THRESHOLD.getKey(), 50) + .put(SearchShardTaskSettings.SETTING_CPU_TIME_MILLIS_THRESHOLD.getKey(), 1000) .build(); assertAcked(client().admin().cluster().prepareUpdateSettings().setPersistentSettings(request).get()); diff --git a/server/src/main/java/org/opensearch/tasks/CancellableTask.java b/server/src/main/java/org/opensearch/tasks/CancellableTask.java index d32d04f006bbd..dc28c26700e6c 100644 --- a/server/src/main/java/org/opensearch/tasks/CancellableTask.java +++ b/server/src/main/java/org/opensearch/tasks/CancellableTask.java @@ -33,10 +33,10 @@ package org.opensearch.tasks; import org.opensearch.common.Nullable; +import org.opensearch.common.SetOnce; import org.opensearch.common.unit.TimeValue; import java.util.Map; -import java.util.concurrent.atomic.AtomicBoolean; import static org.opensearch.search.SearchService.NO_TIMEOUT; @@ -47,17 +47,26 @@ */ public abstract class CancellableTask extends Task { - private volatile String reason; - private final AtomicBoolean cancelled = new AtomicBoolean(false); + private static class CancelledInfo { + String reason; + /** + * The time this task was cancelled as a wall clock time since epoch ({@link System#currentTimeMillis()} style). + */ + Long cancellationStartTime; + /** + * The time this task was cancelled as a relative time ({@link System#nanoTime()} style). + */ + Long cancellationStartTimeNanos; + + public CancelledInfo(String reason) { + this.reason = reason; + this.cancellationStartTime = System.currentTimeMillis(); + this.cancellationStartTimeNanos = System.nanoTime(); + } + } + + private final SetOnce cancelledInfo = new SetOnce<>(); private final TimeValue cancelAfterTimeInterval; - /** - * The time this task was cancelled as a wall clock time since epoch ({@link System#currentTimeMillis()} style). - */ - private Long cancellationStartTime = null; - /** - * The time this task was cancelled as a relative time ({@link System#nanoTime()} style). - */ - private Long cancellationStartTimeNanos = null; public CancellableTask(long id, String type, String action, String description, TaskId parentTaskId, Map headers) { this(id, type, action, description, parentTaskId, headers, NO_TIMEOUT); @@ -81,14 +90,29 @@ public CancellableTask( */ public void cancel(String reason) { assert reason != null; - if (cancelled.compareAndSet(false, true)) { - this.cancellationStartTime = System.currentTimeMillis(); - this.cancellationStartTimeNanos = System.nanoTime(); - this.reason = reason; + if (cancelledInfo.trySet(new CancelledInfo(reason))) { onCancelled(); } } + public boolean isCancelled() { + return cancelledInfo.get() != null; + } + + /** + * Returns true if this task can potentially have children that need to be cancelled when it parent is cancelled. + */ + public abstract boolean shouldCancelChildrenOnCancellation(); + + public TimeValue getCancellationTimeout() { + return cancelAfterTimeInterval; + } + + /** + * Called after the task is cancelled so that it can take any actions that it has to take. + */ + protected void onCancelled() {} + /** * Returns true if this task should be automatically cancelled if the coordinating node that * requested this task left the cluster. @@ -97,37 +121,24 @@ public boolean cancelOnParentLeaving() { return true; } + @Nullable public Long getCancellationStartTime() { - return cancellationStartTime; + CancelledInfo info = cancelledInfo.get(); + return (info != null) ? info.cancellationStartTime : null; } + @Nullable public Long getCancellationStartTimeNanos() { - return cancellationStartTimeNanos; - } - - /** - * Returns true if this task can potentially have children that need to be cancelled when it parent is cancelled. - */ - public abstract boolean shouldCancelChildrenOnCancellation(); - - public boolean isCancelled() { - return cancelled.get(); - } - - public TimeValue getCancellationTimeout() { - return cancelAfterTimeInterval; + CancelledInfo info = cancelledInfo.get(); + return (info != null) ? info.cancellationStartTimeNanos : null; } /** * The reason the task was cancelled or null if it hasn't been cancelled. */ @Nullable - public final String getReasonCancelled() { - return reason; + public String getReasonCancelled() { + CancelledInfo info = cancelledInfo.get(); + return (info != null) ? info.reason : null; } - - /** - * Called after the task is cancelled so that it can take any actions that it has to take. - */ - protected void onCancelled() {} } From 049129c7b64eef8c45f9e2f4c9bf165c8724c5a7 Mon Sep 17 00:00:00 2001 From: Jay Deng Date: Thu, 22 Jun 2023 09:46:13 -0700 Subject: [PATCH 055/109] Return null for aggregations factory even if empty aggs is present in search source (#8206) * Return null for aggregations factory even if empty aggs is present in search source Signed-off-by: Jay Deng * Update rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/60_empty.yml Co-authored-by: Andriy Redko Signed-off-by: Jay Deng --------- Signed-off-by: Jay Deng Co-authored-by: Andriy Redko --- .../test/search.aggregation/60_empty.yml | 29 +++++++++++++++++++ .../aggregations/AggregatorFactories.java | 2 +- 2 files changed, 30 insertions(+), 1 deletion(-) create mode 100644 rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/60_empty.yml diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/60_empty.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/60_empty.yml new file mode 100644 index 0000000000000..7b374e3f6a409 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/60_empty.yml @@ -0,0 +1,29 @@ +--- +"Empty aggs Body": + - do: + index: + index: test + id: 1 + body: { "double" : 42 } + + - do: + index: + index: test + id: 2 + body: { "double" : 100 } + + - do: + index: + index: test + id: 3 + body: { "double" : 50 } + + - do: + indices.refresh: {} + + - do: + search: + rest_total_hits_as_int: true + body: { "size" : 0, "aggs" : { } } + + - match: { hits.total: 3 } diff --git a/server/src/main/java/org/opensearch/search/aggregations/AggregatorFactories.java b/server/src/main/java/org/opensearch/search/aggregations/AggregatorFactories.java index f760070a9b650..e57776fd78c49 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/AggregatorFactories.java +++ b/server/src/main/java/org/opensearch/search/aggregations/AggregatorFactories.java @@ -235,7 +235,7 @@ private static AggregatorFactories.Builder parseAggregators(XContentParser parse } } - return factories; + return factories.count() > 0 ? factories : null; } public static final AggregatorFactories EMPTY = new AggregatorFactories(new AggregatorFactory[0]); From 3305c771bd5b85c4122e4ff1aa76af71b5c8160a Mon Sep 17 00:00:00 2001 From: Ashish Date: Thu, 22 Jun 2023 22:20:12 +0530 Subject: [PATCH 056/109] Make remote translog upload buffer interval setting dynamic (#8175) Signed-off-by: Ashish Singh --- ...CreateRemoteIndexClusterDefaultDocRep.java | 13 +++- .../remotestore/CreateRemoteIndexIT.java | 56 ++++++++++++---- .../CreateRemoteIndexTranslogDisabledIT.java | 21 +++++- .../cluster/metadata/IndexMetadata.java | 42 ------------ .../common/settings/IndexScopedSettings.java | 6 +- .../concurrent/BufferedAsyncIOProcessor.java | 14 ++-- .../org/opensearch/index/IndexSettings.java | 25 +++++-- .../opensearch/index/shard/IndexShard.java | 6 +- .../MetadataCreateIndexServiceTests.java | 44 ++++++++++--- .../BufferedAsyncIOProcessorTests.java | 8 +-- .../opensearch/index/IndexServiceTests.java | 20 ++++++ .../opensearch/index/IndexSettingsTests.java | 65 +++++++++++++------ ...overyWithRemoteTranslogOnPrimaryTests.java | 3 +- ...teStorePeerRecoverySourceHandlerTests.java | 3 +- 14 files changed, 216 insertions(+), 110 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/CreateRemoteIndexClusterDefaultDocRep.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/CreateRemoteIndexClusterDefaultDocRep.java index a4f6aceedc586..7b43ffb597661 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/CreateRemoteIndexClusterDefaultDocRep.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/CreateRemoteIndexClusterDefaultDocRep.java @@ -12,6 +12,7 @@ import org.opensearch.action.admin.indices.get.GetIndexResponse; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.settings.Settings; +import org.opensearch.index.IndexSettings; import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.test.OpenSearchIntegTestCase; @@ -46,7 +47,15 @@ public void testRemoteStoreTranslogDisabledByUser() throws Exception { .getIndex(new GetIndexRequest().indices("test-idx-1").includeDefaults(true)) .get(); Settings indexSettings = getIndexResponse.settings().get("test-idx-1"); - verifyRemoteStoreIndexSettings(indexSettings, "true", "my-segment-repo-1", "false", null, ReplicationType.SEGMENT.toString(), null); + verifyRemoteStoreIndexSettings( + indexSettings, + "true", + "my-segment-repo-1", + "false", + null, + ReplicationType.SEGMENT.toString(), + IndexSettings.DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL + ); } @Override @@ -84,7 +93,7 @@ public void testDefaultRemoteStoreNoUserOverrideExceptReplicationTypeSegment() t "true", "my-translog-repo-1", ReplicationType.SEGMENT.toString(), - null + IndexSettings.DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL ); } } diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/CreateRemoteIndexIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/CreateRemoteIndexIT.java index a3235fd2a91d4..fda344acad166 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/CreateRemoteIndexIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/CreateRemoteIndexIT.java @@ -14,7 +14,9 @@ import org.opensearch.action.admin.indices.get.GetIndexResponse; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.FeatureFlags; +import org.opensearch.index.IndexSettings; import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.test.FeatureFlagSetter; import org.opensearch.test.OpenSearchIntegTestCase; @@ -22,7 +24,7 @@ import static org.hamcrest.Matchers.containsString; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_STORE_ENABLED; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_STORE_REPOSITORY; -import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_TRANSLOG_BUFFER_INTERVAL; +import static org.opensearch.index.IndexSettings.INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_ENABLED; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REPLICATION_TYPE; @@ -101,7 +103,7 @@ public void testDefaultRemoteStoreNoUserOverride() throws Exception { "true", "my-translog-repo-1", ReplicationType.SEGMENT.toString(), - null + IndexSettings.DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL ); } @@ -124,7 +126,7 @@ public void testRemoteStoreDisabledByUser() throws Exception { null, null, client().settings().get(CLUSTER_SETTING_REPLICATION_TYPE), - null + IndexSettings.DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL ); } @@ -175,7 +177,15 @@ public void testReplicationTypeDocumentByUser() throws Exception { .getIndex(new GetIndexRequest().indices("test-idx-1").includeDefaults(true)) .get(); Settings indexSettings = getIndexResponse.settings().get("test-idx-1"); - verifyRemoteStoreIndexSettings(indexSettings, null, null, null, null, ReplicationType.DOCUMENT.toString(), null); + verifyRemoteStoreIndexSettings( + indexSettings, + null, + null, + null, + null, + ReplicationType.DOCUMENT.toString(), + IndexSettings.DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL + ); } public void testRemoteStoreSegmentRepoWithoutRemoteEnabledAndSegmentReplicationIllegalArgumentException() throws Exception { @@ -216,7 +226,7 @@ public void testRemoteStoreEnabledByUserWithRemoteRepo() throws Exception { "true", "my-translog-repo-1", ReplicationType.SEGMENT.toString(), - null + IndexSettings.DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL ); } @@ -232,7 +242,15 @@ public void testRemoteStoreTranslogDisabledByUser() throws Exception { .getIndex(new GetIndexRequest().indices("test-idx-1").includeDefaults(true)) .get(); Settings indexSettings = getIndexResponse.settings().get("test-idx-1"); - verifyRemoteStoreIndexSettings(indexSettings, "true", "my-segment-repo-1", "false", null, ReplicationType.SEGMENT.toString(), null); + verifyRemoteStoreIndexSettings( + indexSettings, + "true", + "my-segment-repo-1", + "false", + null, + ReplicationType.SEGMENT.toString(), + IndexSettings.DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL + ); } public void testRemoteStoreOverrideOnlyTranslogRepoIllegalArgumentException() throws Exception { @@ -307,7 +325,15 @@ public void testRemoteStoreOverrideTranslogDisabledCorrectly() throws Exception .getIndex(new GetIndexRequest().indices("test-idx-1").includeDefaults(true)) .get(); Settings indexSettings = getIndexResponse.settings().get("test-idx-1"); - verifyRemoteStoreIndexSettings(indexSettings, "true", "my-custom-repo", "false", null, ReplicationType.SEGMENT.toString(), null); + verifyRemoteStoreIndexSettings( + indexSettings, + "true", + "my-custom-repo", + "false", + null, + ReplicationType.SEGMENT.toString(), + IndexSettings.DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL + ); } public void testRemoteStoreOverrideTranslogDisabledWithTranslogRepoIllegalArgumentException() throws Exception { @@ -376,7 +402,7 @@ public void testRemoteStoreOverrideTranslogRepoCorrectly() throws Exception { "true", "my-custom-repo", ReplicationType.SEGMENT.toString(), - null + IndexSettings.DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL ); } @@ -392,7 +418,15 @@ public void testRemoteStoreOverrideReplicationTypeIndexSettings() throws Excepti .getIndex(new GetIndexRequest().indices("test-idx-1").includeDefaults(true)) .get(); Settings indexSettings = getIndexResponse.settings().get("test-idx-1"); - verifyRemoteStoreIndexSettings(indexSettings, null, null, null, null, ReplicationType.DOCUMENT.toString(), null); + verifyRemoteStoreIndexSettings( + indexSettings, + null, + null, + null, + null, + ReplicationType.DOCUMENT.toString(), + IndexSettings.DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL + ); } protected void verifyRemoteStoreIndexSettings( @@ -402,14 +436,14 @@ protected void verifyRemoteStoreIndexSettings( String isRemoteTranslogEnabled, String remoteTranslogRepo, String replicationType, - String translogBufferInterval + TimeValue translogBufferInterval ) { assertEquals(replicationType, indexSettings.get(SETTING_REPLICATION_TYPE)); assertEquals(isRemoteSegmentEnabled, indexSettings.get(SETTING_REMOTE_STORE_ENABLED)); assertEquals(remoteSegmentRepo, indexSettings.get(SETTING_REMOTE_STORE_REPOSITORY)); assertEquals(isRemoteTranslogEnabled, indexSettings.get(SETTING_REMOTE_TRANSLOG_STORE_ENABLED)); assertEquals(remoteTranslogRepo, indexSettings.get(SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY)); - assertEquals(translogBufferInterval, indexSettings.get(SETTING_REMOTE_TRANSLOG_BUFFER_INTERVAL)); + assertEquals(translogBufferInterval, INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.get(indexSettings)); } } diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/CreateRemoteIndexTranslogDisabledIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/CreateRemoteIndexTranslogDisabledIT.java index 977b5f9c1b90e..01c463392d7b3 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/CreateRemoteIndexTranslogDisabledIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/CreateRemoteIndexTranslogDisabledIT.java @@ -12,6 +12,7 @@ import org.opensearch.action.admin.indices.get.GetIndexResponse; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.settings.Settings; +import org.opensearch.index.IndexSettings; import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.test.OpenSearchIntegTestCase; @@ -49,7 +50,15 @@ public void testRemoteStoreEnabledByUserWithRemoteRepo() throws Exception { .getIndex(new GetIndexRequest().indices("test-idx-1").includeDefaults(true)) .get(); Settings indexSettings = getIndexResponse.settings().get("test-idx-1"); - verifyRemoteStoreIndexSettings(indexSettings, "true", "my-custom-repo", null, null, ReplicationType.SEGMENT.toString(), null); + verifyRemoteStoreIndexSettings( + indexSettings, + "true", + "my-custom-repo", + null, + null, + ReplicationType.SEGMENT.toString(), + IndexSettings.DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL + ); } public void testDefaultRemoteStoreNoUserOverride() throws Exception { @@ -63,7 +72,15 @@ public void testDefaultRemoteStoreNoUserOverride() throws Exception { .getIndex(new GetIndexRequest().indices("test-idx-1").includeDefaults(true)) .get(); Settings indexSettings = getIndexResponse.settings().get("test-idx-1"); - verifyRemoteStoreIndexSettings(indexSettings, "true", "my-segment-repo-1", null, null, ReplicationType.SEGMENT.toString(), null); + verifyRemoteStoreIndexSettings( + indexSettings, + "true", + "my-segment-repo-1", + null, + null, + ReplicationType.SEGMENT.toString(), + IndexSettings.DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL + ); } } 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 d701d53ba29a1..93a942fcbf7eb 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java @@ -54,7 +54,6 @@ 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.xcontent.XContentFactory; import org.opensearch.common.xcontent.XContentHelper; import org.opensearch.core.common.Strings; @@ -299,8 +298,6 @@ public Iterator> settings() { public static final String SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY = "index.remote_store.translog.repository"; - public static final String SETTING_REMOTE_TRANSLOG_BUFFER_INTERVAL = "index.remote_store.translog.buffer_interval"; - /** * Used to specify if the index data should be persisted in the remote store. */ @@ -449,45 +446,6 @@ public Iterator> settings() { Property.Final ); - public static final Setting INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING = Setting.timeSetting( - SETTING_REMOTE_TRANSLOG_BUFFER_INTERVAL, - TimeValue.timeValueMillis(100), - TimeValue.timeValueMillis(50), - new Setting.Validator<>() { - - @Override - public void validate(final TimeValue value) {} - - @Override - public void validate(final TimeValue value, final Map, Object> settings) { - if (value == null) { - throw new IllegalArgumentException( - "Setting " + SETTING_REMOTE_TRANSLOG_BUFFER_INTERVAL + " should be provided with a valid time value" - ); - } else { - final Boolean isRemoteTranslogStoreEnabled = (Boolean) settings.get(INDEX_REMOTE_TRANSLOG_STORE_ENABLED_SETTING); - if (isRemoteTranslogStoreEnabled == null || isRemoteTranslogStoreEnabled == false) { - throw new IllegalArgumentException( - "Setting " - + SETTING_REMOTE_TRANSLOG_BUFFER_INTERVAL - + " can only be set when " - + SETTING_REMOTE_TRANSLOG_STORE_ENABLED - + " is set to true" - ); - } - } - } - - @Override - public Iterator> settings() { - final List> settings = Collections.singletonList(INDEX_REMOTE_TRANSLOG_STORE_ENABLED_SETTING); - return settings.iterator(); - } - }, - Property.IndexScope, - Property.Final - ); - public static final String SETTING_AUTO_EXPAND_REPLICAS = "index.auto_expand_replicas"; public static final Setting INDEX_AUTO_EXPAND_REPLICAS_SETTING = AutoExpandReplicas.SETTING; diff --git a/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java b/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java index 95c0f3b55222f..08d8199afface 100644 --- a/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java @@ -205,6 +205,9 @@ public final class IndexScopedSettings extends AbstractScopedSettings { IndexSettings.SEARCHABLE_SNAPSHOT_ID_NAME, IndexSettings.SEARCHABLE_SNAPSHOT_ID_UUID, + // Settings for remote translog + IndexSettings.INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING, + // validate that built-in similarities don't get redefined Setting.groupSetting("index.similarity.", (s) -> { Map groups = s.getAsGroups(); @@ -233,8 +236,7 @@ public final class IndexScopedSettings extends AbstractScopedSettings { IndexMetadata.INDEX_REMOTE_STORE_ENABLED_SETTING, IndexMetadata.INDEX_REMOTE_STORE_REPOSITORY_SETTING, IndexMetadata.INDEX_REMOTE_TRANSLOG_STORE_ENABLED_SETTING, - IndexMetadata.INDEX_REMOTE_TRANSLOG_REPOSITORY_SETTING, - IndexMetadata.INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING + IndexMetadata.INDEX_REMOTE_TRANSLOG_REPOSITORY_SETTING ), FeatureFlags.CONCURRENT_SEGMENT_SEARCH, List.of(IndexSettings.INDEX_CONCURRENT_SEGMENT_SEARCH_SETTING) diff --git a/server/src/main/java/org/opensearch/common/util/concurrent/BufferedAsyncIOProcessor.java b/server/src/main/java/org/opensearch/common/util/concurrent/BufferedAsyncIOProcessor.java index f3d909e1c92bc..7079aa705d126 100644 --- a/server/src/main/java/org/opensearch/common/util/concurrent/BufferedAsyncIOProcessor.java +++ b/server/src/main/java/org/opensearch/common/util/concurrent/BufferedAsyncIOProcessor.java @@ -17,10 +17,11 @@ import java.util.List; import java.util.Objects; import java.util.function.Consumer; +import java.util.function.Supplier; /** * A variant of {@link AsyncIOProcessor} that allows to batch and buffer processing items at every - * {@link BufferedAsyncIOProcessor#bufferInterval} in a separate threadpool. + * {@link BufferedAsyncIOProcessor#getBufferInterval()} in a separate threadpool. *

* Requests are buffered till processor thread calls @{link drainAndProcessAndRelease} after bufferInterval. * If more requests are enqueued between invocations of drainAndProcessAndRelease, another processor thread @@ -32,18 +33,18 @@ public abstract class BufferedAsyncIOProcessor extends AsyncIOProcessor { private final ThreadPool threadpool; - private final TimeValue bufferInterval; + private final Supplier bufferIntervalSupplier; protected BufferedAsyncIOProcessor( Logger logger, int queueSize, ThreadContext threadContext, ThreadPool threadpool, - TimeValue bufferInterval + Supplier bufferIntervalSupplier ) { super(logger, queueSize, threadContext); this.threadpool = threadpool; - this.bufferInterval = bufferInterval; + this.bufferIntervalSupplier = bufferIntervalSupplier; } @Override @@ -81,11 +82,12 @@ private void process() { } private TimeValue getBufferInterval() { + long bufferInterval = bufferIntervalSupplier.get().getNanos(); long timeSinceLastRunStartInNS = System.nanoTime() - getLastRunStartTimeInNs(); - if (timeSinceLastRunStartInNS >= bufferInterval.getNanos()) { + if (timeSinceLastRunStartInNS >= bufferInterval) { return TimeValue.ZERO; } - return TimeValue.timeValueNanos(bufferInterval.getNanos() - timeSinceLastRunStartInNS); + return TimeValue.timeValueNanos(bufferInterval - timeSinceLastRunStartInNS); } protected abstract String getBufferProcessThreadPoolName(); diff --git a/server/src/main/java/org/opensearch/index/IndexSettings.java b/server/src/main/java/org/opensearch/index/IndexSettings.java index 9c6613495ba80..dae2d2369fc4c 100644 --- a/server/src/main/java/org/opensearch/index/IndexSettings.java +++ b/server/src/main/java/org/opensearch/index/IndexSettings.java @@ -595,6 +595,16 @@ public final class IndexSettings { Property.Dynamic ); + public static final TimeValue DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL = new TimeValue(650, TimeUnit.MILLISECONDS); + public static final TimeValue MINIMUM_REMOTE_TRANSLOG_BUFFER_INTERVAL = TimeValue.ZERO; + public static final Setting INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING = Setting.timeSetting( + "index.remote_store.translog.buffer_interval", + DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL, + MINIMUM_REMOTE_TRANSLOG_BUFFER_INTERVAL, + Property.Dynamic, + Property.IndexScope + ); + private final Index index; private final Version version; private final Logger logger; @@ -604,7 +614,7 @@ public final class IndexSettings { private final ReplicationType replicationType; private final boolean isRemoteStoreEnabled; private final boolean isRemoteTranslogStoreEnabled; - private final TimeValue remoteTranslogUploadBufferInterval; + private volatile TimeValue remoteTranslogUploadBufferInterval; private final String remoteStoreTranslogRepository; private final String remoteStoreRepository; private final boolean isRemoteSnapshot; @@ -775,10 +785,7 @@ public IndexSettings(final IndexMetadata indexMetadata, final Settings nodeSetti isRemoteStoreEnabled = settings.getAsBoolean(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, false); isRemoteTranslogStoreEnabled = settings.getAsBoolean(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_ENABLED, false); remoteStoreTranslogRepository = settings.get(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY); - remoteTranslogUploadBufferInterval = settings.getAsTime( - IndexMetadata.SETTING_REMOTE_TRANSLOG_BUFFER_INTERVAL, - TimeValue.timeValueMillis(100) - ); + remoteTranslogUploadBufferInterval = INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.get(settings); remoteStoreRepository = settings.get(IndexMetadata.SETTING_REMOTE_STORE_REPOSITORY); isRemoteSnapshot = IndexModule.Type.REMOTE_SNAPSHOT.match(this.settings); @@ -911,6 +918,10 @@ public IndexSettings(final IndexMetadata indexMetadata, final Settings nodeSetti scopedSettings.addSettingsUpdateConsumer(INDEX_MERGE_ON_FLUSH_ENABLED, this::setMergeOnFlushEnabled); scopedSettings.addSettingsUpdateConsumer(INDEX_MERGE_ON_FLUSH_POLICY, this::setMergeOnFlushPolicy); scopedSettings.addSettingsUpdateConsumer(DEFAULT_SEARCH_PIPELINE, this::setDefaultSearchPipeline); + scopedSettings.addSettingsUpdateConsumer( + INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING, + this::setRemoteTranslogUploadBufferInterval + ); } private void setSearchSegmentOrderReversed(boolean reversed) { @@ -1191,6 +1202,10 @@ public TimeValue getRemoteTranslogUploadBufferInterval() { return remoteTranslogUploadBufferInterval; } + public void setRemoteTranslogUploadBufferInterval(TimeValue remoteTranslogUploadBufferInterval) { + this.remoteTranslogUploadBufferInterval = remoteTranslogUploadBufferInterval; + } + /** * Returns this interval in which the shards of this index are asynchronously refreshed. {@code -1} means async refresh is disabled. */ diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index e368ee82b1084..93d4d4bbfec8b 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -382,7 +382,7 @@ public IndexShard( threadPool, this::getEngine, indexSettings.isRemoteTranslogStoreEnabled(), - indexSettings.getRemoteTranslogUploadBufferInterval() + indexSettings::getRemoteTranslogUploadBufferInterval ); this.mapperService = mapperService; this.indexCache = indexCache; @@ -4027,7 +4027,7 @@ private static AsyncIOProcessor createTranslogSyncProcessor( ThreadPool threadPool, Supplier engineSupplier, boolean bufferAsyncIoProcessor, - TimeValue bufferInterval + Supplier bufferIntervalSupplier ) { ThreadContext threadContext = threadPool.getThreadContext(); CheckedConsumer>>, IOException> writeConsumer = candidates -> { @@ -4042,7 +4042,7 @@ private static AsyncIOProcessor createTranslogSyncProcessor( } }; if (bufferAsyncIoProcessor) { - return new BufferedAsyncIOProcessor<>(logger, 102400, threadContext, threadPool, bufferInterval) { + return new BufferedAsyncIOProcessor<>(logger, 102400, threadContext, threadPool, bufferIntervalSupplier) { @Override protected void write(List>> candidates) throws IOException { writeConsumer.accept(candidates); diff --git a/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java b/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java index 2abb1e99facef..fbb86620bdc23 100644 --- a/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java +++ b/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java @@ -132,7 +132,6 @@ import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_READ_ONLY; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_STORE_ENABLED; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_STORE_REPOSITORY; -import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_TRANSLOG_BUFFER_INTERVAL; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_ENABLED; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REPLICATION_TYPE; @@ -143,6 +142,7 @@ import static org.opensearch.cluster.metadata.MetadataCreateIndexService.getIndexNumberOfRoutingShards; import static org.opensearch.cluster.metadata.MetadataCreateIndexService.parseV1Mappings; import static org.opensearch.cluster.metadata.MetadataCreateIndexService.resolveAndValidateAliases; +import static org.opensearch.index.IndexSettings.INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING; import static org.opensearch.index.IndexSettings.INDEX_SOFT_DELETES_SETTING; import static org.opensearch.indices.IndicesService.CLUSTER_REMOTE_STORE_REPOSITORY_SETTING; import static org.opensearch.indices.IndicesService.CLUSTER_REMOTE_TRANSLOG_REPOSITORY_SETTING; @@ -1262,7 +1262,7 @@ public void testRemoteStoreNoUserOverrideExceptReplicationTypeSegmentIndexSettin "true", "my-translog-repo-1", ReplicationType.SEGMENT.toString(), - null + IndexSettings.DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL ); } @@ -1295,7 +1295,7 @@ public void testRemoteStoreNoUserOverrideIndexSettings() { "true", "my-translog-repo-1", ReplicationType.SEGMENT.toString(), - null + IndexSettings.DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL ); } @@ -1323,7 +1323,15 @@ public void testRemoteStoreDisabledByUserIndexSettings() { Collections.emptySet(), false ); - verifyRemoteStoreIndexSettings(indexSettings, "false", null, null, null, ReplicationType.SEGMENT.toString(), null); + verifyRemoteStoreIndexSettings( + indexSettings, + "false", + null, + null, + null, + ReplicationType.SEGMENT.toString(), + IndexSettings.DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL + ); } public void testRemoteStoreTranslogDisabledByUserIndexSettings() { @@ -1350,7 +1358,15 @@ public void testRemoteStoreTranslogDisabledByUserIndexSettings() { Collections.emptySet(), false ); - verifyRemoteStoreIndexSettings(indexSettings, "true", "my-segment-repo-1", "false", null, ReplicationType.SEGMENT.toString(), null); + verifyRemoteStoreIndexSettings( + indexSettings, + "true", + "my-segment-repo-1", + "false", + null, + ReplicationType.SEGMENT.toString(), + IndexSettings.DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL + ); } public void testRemoteStoreOverrideSegmentRepoIndexSettings() { @@ -1387,7 +1403,7 @@ public void testRemoteStoreOverrideSegmentRepoIndexSettings() { "true", "my-translog-repo-1", ReplicationType.SEGMENT.toString(), - null + IndexSettings.DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL ); } @@ -1422,7 +1438,7 @@ public void testRemoteStoreOverrideTranslogRepoIndexSettings() { "true", "my-custom-repo", ReplicationType.SEGMENT.toString(), - null + IndexSettings.DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL ); } @@ -1450,7 +1466,15 @@ public void testRemoteStoreOverrideReplicationTypeIndexSettings() { Collections.emptySet(), false ); - verifyRemoteStoreIndexSettings(indexSettings, null, null, null, null, ReplicationType.DOCUMENT.toString(), null); + verifyRemoteStoreIndexSettings( + indexSettings, + null, + null, + null, + null, + ReplicationType.DOCUMENT.toString(), + IndexSettings.DEFAULT_REMOTE_TRANSLOG_BUFFER_INTERVAL + ); } public void testBuildIndexMetadata() { @@ -1743,14 +1767,14 @@ private void verifyRemoteStoreIndexSettings( String isRemoteTranslogEnabled, String remoteTranslogRepo, String replicationType, - String translogBufferInterval + TimeValue translogBufferInterval ) { assertEquals(replicationType, indexSettings.get(SETTING_REPLICATION_TYPE)); assertEquals(isRemoteSegmentEnabled, indexSettings.get(SETTING_REMOTE_STORE_ENABLED)); assertEquals(remoteSegmentRepo, indexSettings.get(SETTING_REMOTE_STORE_REPOSITORY)); assertEquals(isRemoteTranslogEnabled, indexSettings.get(SETTING_REMOTE_TRANSLOG_STORE_ENABLED)); assertEquals(remoteTranslogRepo, indexSettings.get(SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY)); - assertEquals(translogBufferInterval, indexSettings.get(SETTING_REMOTE_TRANSLOG_BUFFER_INTERVAL)); + assertEquals(translogBufferInterval, INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.get(indexSettings)); } } diff --git a/server/src/test/java/org/opensearch/common/util/concurrent/BufferedAsyncIOProcessorTests.java b/server/src/test/java/org/opensearch/common/util/concurrent/BufferedAsyncIOProcessorTests.java index f058189f904a0..c38956f965ac3 100644 --- a/server/src/test/java/org/opensearch/common/util/concurrent/BufferedAsyncIOProcessorTests.java +++ b/server/src/test/java/org/opensearch/common/util/concurrent/BufferedAsyncIOProcessorTests.java @@ -58,7 +58,7 @@ public void testConsumerCanThrowExceptions() { scaledRandomIntBetween(1, 2024), threadContext, threadpool, - TimeValue.timeValueMillis(50) + () -> TimeValue.timeValueMillis(50) ) { @Override protected void write(List>> candidates) throws IOException { @@ -100,7 +100,7 @@ public void testPreserveThreadContext() throws InterruptedException { scaledRandomIntBetween(1, 2024), threadContext, threadpool, - TimeValue.timeValueMillis(100) + () -> TimeValue.timeValueMillis(100) ) { @Override protected void write(List>> candidates) throws IOException { @@ -156,7 +156,7 @@ public void testSlowConsumer() { scaledRandomIntBetween(1, 2024), threadContext, threadpool, - TimeValue.timeValueMillis(100) + () -> TimeValue.timeValueMillis(100) ) { @Override protected void write(List>> candidates) throws IOException { @@ -220,7 +220,7 @@ public void testConsecutiveWritesAtLeastBufferIntervalAway() throws InterruptedE scaledRandomIntBetween(1, 2024), threadContext, threadpool, - TimeValue.timeValueMillis(bufferIntervalMs) + () -> TimeValue.timeValueMillis(bufferIntervalMs) ) { @Override protected void write(List>> candidates) throws IOException { diff --git a/server/src/test/java/org/opensearch/index/IndexServiceTests.java b/server/src/test/java/org/opensearch/index/IndexServiceTests.java index 8b133b31e97b7..209b7eb4c5696 100644 --- a/server/src/test/java/org/opensearch/index/IndexServiceTests.java +++ b/server/src/test/java/org/opensearch/index/IndexServiceTests.java @@ -502,4 +502,24 @@ public void testUpdateSyncIntervalDynamically() { indexMetadata = client().admin().cluster().prepareState().execute().actionGet().getState().metadata().index("test"); assertEquals("20s", indexMetadata.getSettings().get(IndexSettings.INDEX_TRANSLOG_SYNC_INTERVAL_SETTING.getKey())); } + + public void testUpdateRemoteTranslogBufferIntervalDynamically() { + Settings settings = Settings.builder().put(IndexSettings.INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.getKey(), "10s").build(); + IndexService indexService = createIndex("test", settings); + ensureGreen("test"); + + Settings.Builder builder = Settings.builder().put(IndexSettings.INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.getKey(), "5s"); + client().admin().indices().prepareUpdateSettings("test").setSettings(builder).get(); + IndexMetadata indexMetadata = client().admin().cluster().prepareState().execute().actionGet().getState().metadata().index("test"); + assertEquals("5s", indexMetadata.getSettings().get(IndexSettings.INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.getKey())); + + client().admin().indices().prepareClose("test").get(); + client().admin() + .indices() + .prepareUpdateSettings("test") + .setSettings(Settings.builder().put(IndexSettings.INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.getKey(), "20s")) + .get(); + indexMetadata = client().admin().cluster().prepareState().execute().actionGet().getState().metadata().index("test"); + assertEquals("20s", indexMetadata.getSettings().get(IndexSettings.INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.getKey())); + } } diff --git a/server/src/test/java/org/opensearch/index/IndexSettingsTests.java b/server/src/test/java/org/opensearch/index/IndexSettingsTests.java index bf3c6e15bde6e..490d54d778477 100644 --- a/server/src/test/java/org/opensearch/index/IndexSettingsTests.java +++ b/server/src/test/java/org/opensearch/index/IndexSettingsTests.java @@ -326,7 +326,7 @@ public void testIsWarmerEnabled() { } public void testRefreshInterval() { - String refreshInterval = getRandomTimeString(); + String refreshInterval = getRandomTimeString(false); IndexMetadata metadata = newIndexMeta( "index", Settings.builder() @@ -343,7 +343,7 @@ public void testRefreshInterval() { ), settings.getRefreshInterval() ); - String newRefreshInterval = getRandomTimeString(); + String newRefreshInterval = getRandomTimeString(false); settings.updateIndexMetadata( newIndexMeta("index", Settings.builder().put(IndexSettings.INDEX_REFRESH_INTERVAL_SETTING.getKey(), newRefreshInterval).build()) ); @@ -357,8 +357,12 @@ public void testRefreshInterval() { ); } - private String getRandomTimeString() { - int refreshIntervalInt = randomFrom(-1, Math.abs(randomInt())); + private String getRandomTimeString(boolean nonNegativeOnly) { + int start = -1; + if (nonNegativeOnly) { + start = 0; + } + int refreshIntervalInt = randomFrom(start, Math.abs(randomInt())); String refreshInterval = Integer.toString(refreshIntervalInt); if (refreshIntervalInt >= 0) { refreshInterval += randomFrom("s", "ms", "h"); @@ -965,7 +969,7 @@ public void testRemoteTranslogExplicitSetting() { .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_ENABLED, true) .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, "tlog-store") - .put(IndexMetadata.INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.getKey(), "200ms") + .put(IndexSettings.INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.getKey(), "200ms") .build() ); IndexSettings settings = new IndexSettings(metadata, Settings.EMPTY); @@ -1009,38 +1013,57 @@ public void testSetRemoteTranslogBufferIntervalDefaultSetting() { .put(IndexMetadata.SETTING_INDEX_VERSION_CREATED.getKey(), createdVersion) .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_ENABLED, true) .build(); - assertEquals(TimeValue.timeValueMillis(100), IndexMetadata.INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.get(settings)); + assertEquals(TimeValue.timeValueMillis(650), IndexSettings.INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.get(settings)); } - public void testSetRemoteTranslogBufferIntervalFailsWhenRemoteTranslogIsNotEnabled() { + public void testSetRemoteTranslogBufferIntervalFailsWhenEmpty() { Settings indexSettings = Settings.builder() .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_ENABLED, false) - .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_BUFFER_INTERVAL, "200ms") + .put(IndexSettings.INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.getKey(), "") .build(); IllegalArgumentException iae = expectThrows( IllegalArgumentException.class, - () -> IndexMetadata.INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.get(indexSettings) + () -> IndexSettings.INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.get(indexSettings) ); assertEquals( - "Setting index.remote_store.translog.buffer_interval can only be set when index.remote_store.translog.enabled is set to true", + "failed to parse setting [index.remote_store.translog.buffer_interval] with value [] as a time value: unit is missing or unrecognized", iae.getMessage() ); } - public void testSetRemoteTranslogBufferIntervalFailsWhenEmpty() { - Settings indexSettings = Settings.builder() - .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) - .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_ENABLED, false) - .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_BUFFER_INTERVAL, "") - .build(); - IllegalArgumentException iae = expectThrows( - IllegalArgumentException.class, - () -> IndexMetadata.INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.get(indexSettings) + public void testUpdateRemoteTranslogBufferInterval() { + String bufferInterval = getRandomTimeString(true); + IndexMetadata metadata = newIndexMeta( + "index", + Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexSettings.INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.getKey(), bufferInterval) + .build() ); + IndexSettings settings = new IndexSettings(metadata, Settings.EMPTY); assertEquals( - "failed to parse setting [index.remote_store.translog.buffer_interval] with value [] as a time value: unit is missing or unrecognized", - iae.getMessage() + TimeValue.parseTimeValue( + bufferInterval, + new TimeValue(1, TimeUnit.DAYS), + IndexSettings.INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.getKey() + ), + settings.getRemoteTranslogUploadBufferInterval() + ); + String newBufferInterval = getRandomTimeString(true); + settings.updateIndexMetadata( + newIndexMeta( + "index", + Settings.builder().put(IndexSettings.INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.getKey(), newBufferInterval).build() + ) + ); + assertEquals( + TimeValue.parseTimeValue( + newBufferInterval, + new TimeValue(1, TimeUnit.DAYS), + IndexSettings.INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.getKey() + ), + settings.getRemoteTranslogUploadBufferInterval() ); } diff --git a/server/src/test/java/org/opensearch/index/shard/ReplicaRecoveryWithRemoteTranslogOnPrimaryTests.java b/server/src/test/java/org/opensearch/index/shard/ReplicaRecoveryWithRemoteTranslogOnPrimaryTests.java index 7a2f67e206f74..aef90483b4bbd 100644 --- a/server/src/test/java/org/opensearch/index/shard/ReplicaRecoveryWithRemoteTranslogOnPrimaryTests.java +++ b/server/src/test/java/org/opensearch/index/shard/ReplicaRecoveryWithRemoteTranslogOnPrimaryTests.java @@ -14,6 +14,7 @@ import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.ShardRoutingState; import org.opensearch.common.settings.Settings; +import org.opensearch.index.IndexSettings; import org.opensearch.index.engine.DocIdSeqNoAndSource; import org.opensearch.index.engine.NRTReplicationEngine; import org.opensearch.index.engine.NRTReplicationEngineFactory; @@ -36,7 +37,7 @@ public class ReplicaRecoveryWithRemoteTranslogOnPrimaryTests extends OpenSearchI .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, "true") .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_ENABLED, "true") .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, "translog-repo") - .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_BUFFER_INTERVAL, "100ms") + .put(IndexSettings.INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.getKey(), "100ms") .build(); public void testStartSequenceForReplicaRecovery() throws Exception { diff --git a/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java b/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java index ff251f42ab21b..96022315743c2 100644 --- a/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java +++ b/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java @@ -10,6 +10,7 @@ import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.settings.Settings; +import org.opensearch.index.IndexSettings; import org.opensearch.index.engine.NRTReplicationEngineFactory; import org.opensearch.index.replication.OpenSearchIndexLevelReplicationTestCase; import org.opensearch.index.seqno.ReplicationTracker; @@ -23,7 +24,7 @@ public class RemoteStorePeerRecoverySourceHandlerTests extends OpenSearchIndexLe .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, "true") .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_ENABLED, "true") .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, "translog-repo") - .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_BUFFER_INTERVAL, "100ms") + .put(IndexSettings.INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.getKey(), "100ms") .build(); public void testReplicaShardRecoveryUptoLastFlushedCommit() throws Exception { From 1cf9c5c35dcc76a937c034355e30a5ebfe55cbe6 Mon Sep 17 00:00:00 2001 From: Andriy Redko Date: Thu, 22 Jun 2023 13:07:19 -0400 Subject: [PATCH 057/109] Bump 'netty' from 4.1.93.Final to 4.1.94.Final (#8191) Signed-off-by: Andriy Redko --- CHANGELOG.md | 1 + buildSrc/version.properties | 2 +- modules/transport-netty4/build.gradle | 1 - .../licenses/netty-buffer-4.1.93.Final.jar.sha1 | 1 - .../licenses/netty-buffer-4.1.94.Final.jar.sha1 | 1 + .../transport-netty4/licenses/netty-codec-4.1.93.Final.jar.sha1 | 1 - .../transport-netty4/licenses/netty-codec-4.1.94.Final.jar.sha1 | 1 + .../licenses/netty-codec-http-4.1.93.Final.jar.sha1 | 1 - .../licenses/netty-codec-http-4.1.94.Final.jar.sha1 | 1 + .../licenses/netty-codec-http2-4.1.93.Final.jar.sha1 | 1 - .../licenses/netty-codec-http2-4.1.94.Final.jar.sha1 | 1 + .../licenses/netty-common-4.1.93.Final.jar.sha1 | 1 - .../licenses/netty-common-4.1.94.Final.jar.sha1 | 1 + .../licenses/netty-handler-4.1.93.Final.jar.sha1 | 1 - .../licenses/netty-handler-4.1.94.Final.jar.sha1 | 1 + .../licenses/netty-resolver-4.1.93.Final.jar.sha1 | 1 - .../licenses/netty-resolver-4.1.94.Final.jar.sha1 | 1 + .../licenses/netty-transport-4.1.93.Final.jar.sha1 | 1 - .../licenses/netty-transport-4.1.94.Final.jar.sha1 | 1 + .../netty-transport-native-unix-common-4.1.93.Final.jar.sha1 | 1 - .../netty-transport-native-unix-common-4.1.94.Final.jar.sha1 | 1 + .../licenses/netty-codec-dns-4.1.93.Final.jar.sha1 | 1 - .../licenses/netty-codec-dns-4.1.94.Final.jar.sha1 | 1 + .../licenses/netty-codec-http2-4.1.93.Final.jar.sha1 | 1 - .../licenses/netty-codec-http2-4.1.94.Final.jar.sha1 | 1 + .../licenses/netty-codec-socks-4.1.93.Final.jar.sha1 | 1 - .../licenses/netty-codec-socks-4.1.94.Final.jar.sha1 | 1 + .../licenses/netty-handler-proxy-4.1.93.Final.jar.sha1 | 1 - .../licenses/netty-handler-proxy-4.1.94.Final.jar.sha1 | 1 + .../licenses/netty-resolver-dns-4.1.93.Final.jar.sha1 | 1 - .../licenses/netty-resolver-dns-4.1.94.Final.jar.sha1 | 1 + .../netty-transport-native-unix-common-4.1.93.Final.jar.sha1 | 1 - .../netty-transport-native-unix-common-4.1.94.Final.jar.sha1 | 1 + .../repository-hdfs/licenses/netty-all-4.1.93.Final.jar.sha1 | 1 - .../repository-hdfs/licenses/netty-all-4.1.94.Final.jar.sha1 | 1 + plugins/repository-s3/build.gradle | 1 - .../repository-s3/licenses/netty-buffer-4.1.93.Final.jar.sha1 | 1 - .../repository-s3/licenses/netty-buffer-4.1.94.Final.jar.sha1 | 1 + .../repository-s3/licenses/netty-codec-4.1.93.Final.jar.sha1 | 1 - .../repository-s3/licenses/netty-codec-4.1.94.Final.jar.sha1 | 1 + .../licenses/netty-codec-http-4.1.93.Final.jar.sha1 | 1 - .../licenses/netty-codec-http-4.1.94.Final.jar.sha1 | 1 + .../licenses/netty-codec-http2-4.1.93.Final.jar.sha1 | 1 - .../licenses/netty-codec-http2-4.1.94.Final.jar.sha1 | 1 + .../repository-s3/licenses/netty-common-4.1.93.Final.jar.sha1 | 1 - .../repository-s3/licenses/netty-common-4.1.94.Final.jar.sha1 | 1 + .../repository-s3/licenses/netty-handler-4.1.93.Final.jar.sha1 | 1 - .../repository-s3/licenses/netty-handler-4.1.94.Final.jar.sha1 | 1 + .../repository-s3/licenses/netty-resolver-4.1.93.Final.jar.sha1 | 1 - .../repository-s3/licenses/netty-resolver-4.1.94.Final.jar.sha1 | 1 + .../licenses/netty-transport-4.1.93.Final.jar.sha1 | 1 - .../licenses/netty-transport-4.1.94.Final.jar.sha1 | 1 + .../netty-transport-classes-epoll-4.1.93.Final.jar.sha1 | 1 - .../netty-transport-classes-epoll-4.1.94.Final.jar.sha1 | 1 + .../netty-transport-native-unix-common-4.1.93.Final.jar.sha1 | 1 - .../netty-transport-native-unix-common-4.1.94.Final.jar.sha1 | 1 + plugins/transport-nio/build.gradle | 1 - .../transport-nio/licenses/netty-buffer-4.1.93.Final.jar.sha1 | 1 - .../transport-nio/licenses/netty-buffer-4.1.94.Final.jar.sha1 | 1 + .../transport-nio/licenses/netty-codec-4.1.93.Final.jar.sha1 | 1 - .../transport-nio/licenses/netty-codec-4.1.94.Final.jar.sha1 | 1 + .../licenses/netty-codec-http-4.1.93.Final.jar.sha1 | 1 - .../licenses/netty-codec-http-4.1.94.Final.jar.sha1 | 1 + .../transport-nio/licenses/netty-common-4.1.93.Final.jar.sha1 | 1 - .../transport-nio/licenses/netty-common-4.1.94.Final.jar.sha1 | 1 + .../transport-nio/licenses/netty-handler-4.1.93.Final.jar.sha1 | 1 - .../transport-nio/licenses/netty-handler-4.1.94.Final.jar.sha1 | 1 + .../transport-nio/licenses/netty-resolver-4.1.93.Final.jar.sha1 | 1 - .../transport-nio/licenses/netty-resolver-4.1.94.Final.jar.sha1 | 1 + .../licenses/netty-transport-4.1.93.Final.jar.sha1 | 1 - .../licenses/netty-transport-4.1.94.Final.jar.sha1 | 1 + 71 files changed, 35 insertions(+), 37 deletions(-) delete mode 100644 modules/transport-netty4/licenses/netty-buffer-4.1.93.Final.jar.sha1 create mode 100644 modules/transport-netty4/licenses/netty-buffer-4.1.94.Final.jar.sha1 delete mode 100644 modules/transport-netty4/licenses/netty-codec-4.1.93.Final.jar.sha1 create mode 100644 modules/transport-netty4/licenses/netty-codec-4.1.94.Final.jar.sha1 delete mode 100644 modules/transport-netty4/licenses/netty-codec-http-4.1.93.Final.jar.sha1 create mode 100644 modules/transport-netty4/licenses/netty-codec-http-4.1.94.Final.jar.sha1 delete mode 100644 modules/transport-netty4/licenses/netty-codec-http2-4.1.93.Final.jar.sha1 create mode 100644 modules/transport-netty4/licenses/netty-codec-http2-4.1.94.Final.jar.sha1 delete mode 100644 modules/transport-netty4/licenses/netty-common-4.1.93.Final.jar.sha1 create mode 100644 modules/transport-netty4/licenses/netty-common-4.1.94.Final.jar.sha1 delete mode 100644 modules/transport-netty4/licenses/netty-handler-4.1.93.Final.jar.sha1 create mode 100644 modules/transport-netty4/licenses/netty-handler-4.1.94.Final.jar.sha1 delete mode 100644 modules/transport-netty4/licenses/netty-resolver-4.1.93.Final.jar.sha1 create mode 100644 modules/transport-netty4/licenses/netty-resolver-4.1.94.Final.jar.sha1 delete mode 100644 modules/transport-netty4/licenses/netty-transport-4.1.93.Final.jar.sha1 create mode 100644 modules/transport-netty4/licenses/netty-transport-4.1.94.Final.jar.sha1 delete mode 100644 modules/transport-netty4/licenses/netty-transport-native-unix-common-4.1.93.Final.jar.sha1 create mode 100644 modules/transport-netty4/licenses/netty-transport-native-unix-common-4.1.94.Final.jar.sha1 delete mode 100644 plugins/repository-azure/licenses/netty-codec-dns-4.1.93.Final.jar.sha1 create mode 100644 plugins/repository-azure/licenses/netty-codec-dns-4.1.94.Final.jar.sha1 delete mode 100644 plugins/repository-azure/licenses/netty-codec-http2-4.1.93.Final.jar.sha1 create mode 100644 plugins/repository-azure/licenses/netty-codec-http2-4.1.94.Final.jar.sha1 delete mode 100644 plugins/repository-azure/licenses/netty-codec-socks-4.1.93.Final.jar.sha1 create mode 100644 plugins/repository-azure/licenses/netty-codec-socks-4.1.94.Final.jar.sha1 delete mode 100644 plugins/repository-azure/licenses/netty-handler-proxy-4.1.93.Final.jar.sha1 create mode 100644 plugins/repository-azure/licenses/netty-handler-proxy-4.1.94.Final.jar.sha1 delete mode 100644 plugins/repository-azure/licenses/netty-resolver-dns-4.1.93.Final.jar.sha1 create mode 100644 plugins/repository-azure/licenses/netty-resolver-dns-4.1.94.Final.jar.sha1 delete mode 100644 plugins/repository-azure/licenses/netty-transport-native-unix-common-4.1.93.Final.jar.sha1 create mode 100644 plugins/repository-azure/licenses/netty-transport-native-unix-common-4.1.94.Final.jar.sha1 delete mode 100644 plugins/repository-hdfs/licenses/netty-all-4.1.93.Final.jar.sha1 create mode 100644 plugins/repository-hdfs/licenses/netty-all-4.1.94.Final.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/netty-buffer-4.1.93.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-buffer-4.1.94.Final.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/netty-codec-4.1.93.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-codec-4.1.94.Final.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/netty-codec-http-4.1.93.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-codec-http-4.1.94.Final.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/netty-codec-http2-4.1.93.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-codec-http2-4.1.94.Final.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/netty-common-4.1.93.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-common-4.1.94.Final.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/netty-handler-4.1.93.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-handler-4.1.94.Final.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/netty-resolver-4.1.93.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-resolver-4.1.94.Final.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/netty-transport-4.1.93.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-transport-4.1.94.Final.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/netty-transport-classes-epoll-4.1.93.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-transport-classes-epoll-4.1.94.Final.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/netty-transport-native-unix-common-4.1.93.Final.jar.sha1 create mode 100644 plugins/repository-s3/licenses/netty-transport-native-unix-common-4.1.94.Final.jar.sha1 delete mode 100644 plugins/transport-nio/licenses/netty-buffer-4.1.93.Final.jar.sha1 create mode 100644 plugins/transport-nio/licenses/netty-buffer-4.1.94.Final.jar.sha1 delete mode 100644 plugins/transport-nio/licenses/netty-codec-4.1.93.Final.jar.sha1 create mode 100644 plugins/transport-nio/licenses/netty-codec-4.1.94.Final.jar.sha1 delete mode 100644 plugins/transport-nio/licenses/netty-codec-http-4.1.93.Final.jar.sha1 create mode 100644 plugins/transport-nio/licenses/netty-codec-http-4.1.94.Final.jar.sha1 delete mode 100644 plugins/transport-nio/licenses/netty-common-4.1.93.Final.jar.sha1 create mode 100644 plugins/transport-nio/licenses/netty-common-4.1.94.Final.jar.sha1 delete mode 100644 plugins/transport-nio/licenses/netty-handler-4.1.93.Final.jar.sha1 create mode 100644 plugins/transport-nio/licenses/netty-handler-4.1.94.Final.jar.sha1 delete mode 100644 plugins/transport-nio/licenses/netty-resolver-4.1.93.Final.jar.sha1 create mode 100644 plugins/transport-nio/licenses/netty-resolver-4.1.94.Final.jar.sha1 delete mode 100644 plugins/transport-nio/licenses/netty-transport-4.1.93.Final.jar.sha1 create mode 100644 plugins/transport-nio/licenses/netty-transport-4.1.94.Final.jar.sha1 diff --git a/CHANGELOG.md b/CHANGELOG.md index 75287db6b4f07..16c794135cb23 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -109,6 +109,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `com.netflix.nebula:gradle-info-plugin` from 12.1.3 to 12.1.4 (#8139) - Bump `commons-io:commons-io` from 2.12.0 to 2.13.0 in /plugins/discovery-azure-classic ([#8140](https://github.com/opensearch-project/OpenSearch/pull/8140)) - Bump `mockito` from 5.2.0 to 5.4.0 ([#8181](https://github.com/opensearch-project/OpenSearch/pull/8181)) +- Bump `netty` from 4.1.93.Final to 4.1.94.Final ([#8191](https://github.com/opensearch-project/OpenSearch/pull/8191)) ### Changed - Replace jboss-annotations-api_1.2_spec with jakarta.annotation-api ([#7836](https://github.com/opensearch-project/OpenSearch/pull/7836)) diff --git a/buildSrc/version.properties b/buildSrc/version.properties index 84d3b2b814430..d3dbae38c2615 100644 --- a/buildSrc/version.properties +++ b/buildSrc/version.properties @@ -29,7 +29,7 @@ jakarta_annotation = 1.3.5 # when updating the JNA version, also update the version in buildSrc/build.gradle jna = 5.5.0 -netty = 4.1.93.Final +netty = 4.1.94.Final joda = 2.12.2 # client dependencies diff --git a/modules/transport-netty4/build.gradle b/modules/transport-netty4/build.gradle index e0a491ab63d7b..83c4db80b7798 100644 --- a/modules/transport-netty4/build.gradle +++ b/modules/transport-netty4/build.gradle @@ -127,7 +127,6 @@ thirdPartyAudit { 'com.aayushatharva.brotli4j.decoder.DecoderJNI$Status', 'com.aayushatharva.brotli4j.decoder.DecoderJNI$Wrapper', 'com.aayushatharva.brotli4j.encoder.BrotliEncoderChannel', - 'com.aayushatharva.brotli4j.encoder.Encoder', 'com.aayushatharva.brotli4j.encoder.Encoder$Mode', 'com.aayushatharva.brotli4j.encoder.Encoder$Parameters', // classes are missing diff --git a/modules/transport-netty4/licenses/netty-buffer-4.1.93.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-buffer-4.1.93.Final.jar.sha1 deleted file mode 100644 index 5c5a17a9466f1..0000000000000 --- a/modules/transport-netty4/licenses/netty-buffer-4.1.93.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -87fac21f4ef95157866b07b48e3c707a2f13c581 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-buffer-4.1.94.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-buffer-4.1.94.Final.jar.sha1 new file mode 100644 index 0000000000000..05b1c2a4d614e --- /dev/null +++ b/modules/transport-netty4/licenses/netty-buffer-4.1.94.Final.jar.sha1 @@ -0,0 +1 @@ +eec248b26f16e888688e5bb37b7eeda76b78d2f7 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-codec-4.1.93.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-codec-4.1.93.Final.jar.sha1 deleted file mode 100644 index 2b12a111335a2..0000000000000 --- a/modules/transport-netty4/licenses/netty-codec-4.1.93.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -503badb458b6586632be8d1f81aa4e5ab99a80fc \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-codec-4.1.94.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-codec-4.1.94.Final.jar.sha1 new file mode 100644 index 0000000000000..baa7e25f1ac49 --- /dev/null +++ b/modules/transport-netty4/licenses/netty-codec-4.1.94.Final.jar.sha1 @@ -0,0 +1 @@ +c70ef20ca338558147887df60f46341bc47f6900 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-codec-http-4.1.93.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-codec-http-4.1.93.Final.jar.sha1 deleted file mode 100644 index 6719e882e40fe..0000000000000 --- a/modules/transport-netty4/licenses/netty-codec-http-4.1.93.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -36acf0c94d03eb6ecef78a749a32cbb7dc0c57b4 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-codec-http-4.1.94.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-codec-http-4.1.94.Final.jar.sha1 new file mode 100644 index 0000000000000..8c018be2565e5 --- /dev/null +++ b/modules/transport-netty4/licenses/netty-codec-http-4.1.94.Final.jar.sha1 @@ -0,0 +1 @@ +9e5404764092c1f6305ad5719078f46ab228d587 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-codec-http2-4.1.93.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-codec-http2-4.1.93.Final.jar.sha1 deleted file mode 100644 index 02423842d6244..0000000000000 --- a/modules/transport-netty4/licenses/netty-codec-http2-4.1.93.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -0f1625b43bde13ec057da0d2fe381ded2547a70e \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-codec-http2-4.1.94.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-codec-http2-4.1.94.Final.jar.sha1 new file mode 100644 index 0000000000000..e73026b412972 --- /dev/null +++ b/modules/transport-netty4/licenses/netty-codec-http2-4.1.94.Final.jar.sha1 @@ -0,0 +1 @@ +f651595784d6cca4cbca6a8ad74c48fceed6cea8 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-common-4.1.93.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-common-4.1.93.Final.jar.sha1 deleted file mode 100644 index 2324a54dc5735..0000000000000 --- a/modules/transport-netty4/licenses/netty-common-4.1.93.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -1cfc49b91b0d3ddb30c9f7d8467e5d02ae8babdf \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-common-4.1.94.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-common-4.1.94.Final.jar.sha1 new file mode 100644 index 0000000000000..b787338551ede --- /dev/null +++ b/modules/transport-netty4/licenses/netty-common-4.1.94.Final.jar.sha1 @@ -0,0 +1 @@ +ad4ecf779ebc794cd351f57792f56ea01387b868 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-handler-4.1.93.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-handler-4.1.93.Final.jar.sha1 deleted file mode 100644 index 54e5b22047812..0000000000000 --- a/modules/transport-netty4/licenses/netty-handler-4.1.93.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -10f7ed9d8e1bfcba416074c70e5388be96116bfc \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-handler-4.1.94.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-handler-4.1.94.Final.jar.sha1 new file mode 100644 index 0000000000000..b08e85ba7adf8 --- /dev/null +++ b/modules/transport-netty4/licenses/netty-handler-4.1.94.Final.jar.sha1 @@ -0,0 +1 @@ +cd9121ce24d6d3f2898946d04b0ef3ec548b00b4 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-resolver-4.1.93.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-resolver-4.1.93.Final.jar.sha1 deleted file mode 100644 index c795762c2d7f1..0000000000000 --- a/modules/transport-netty4/licenses/netty-resolver-4.1.93.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -3860e99075f9e078364ed38f6d6fc8825b7a168a \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-resolver-4.1.94.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-resolver-4.1.94.Final.jar.sha1 new file mode 100644 index 0000000000000..4c9e4dda2b852 --- /dev/null +++ b/modules/transport-netty4/licenses/netty-resolver-4.1.94.Final.jar.sha1 @@ -0,0 +1 @@ +e96f649e8e9dcb29a1f8e95328b99c9eb6cf76c2 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-transport-4.1.93.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-transport-4.1.93.Final.jar.sha1 deleted file mode 100644 index 687cade3c7b3a..0000000000000 --- a/modules/transport-netty4/licenses/netty-transport-4.1.93.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -1a0894915c8027ce83b4d6a811c4e765955efd15 \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-transport-4.1.94.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-transport-4.1.94.Final.jar.sha1 new file mode 100644 index 0000000000000..ed7760b8e15d1 --- /dev/null +++ b/modules/transport-netty4/licenses/netty-transport-4.1.94.Final.jar.sha1 @@ -0,0 +1 @@ +ec783a737f96991a87b1d5794e2f9eb2024d708a \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-transport-native-unix-common-4.1.93.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-transport-native-unix-common-4.1.93.Final.jar.sha1 deleted file mode 100644 index 0f0acb2eccddf..0000000000000 --- a/modules/transport-netty4/licenses/netty-transport-native-unix-common-4.1.93.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -f4fb8b4c8da539091f43abcbb9f0389e48807eea \ No newline at end of file diff --git a/modules/transport-netty4/licenses/netty-transport-native-unix-common-4.1.94.Final.jar.sha1 b/modules/transport-netty4/licenses/netty-transport-native-unix-common-4.1.94.Final.jar.sha1 new file mode 100644 index 0000000000000..43bc960a347a1 --- /dev/null +++ b/modules/transport-netty4/licenses/netty-transport-native-unix-common-4.1.94.Final.jar.sha1 @@ -0,0 +1 @@ +3fa5f9d04b6b782d869d6e0657d896eeadca5866 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-codec-dns-4.1.93.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-codec-dns-4.1.93.Final.jar.sha1 deleted file mode 100644 index 8ddec4af248f4..0000000000000 --- a/plugins/repository-azure/licenses/netty-codec-dns-4.1.93.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -a1433e4ae1cc56c0809b5b740ff8800d20b33c36 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-codec-dns-4.1.94.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-codec-dns-4.1.94.Final.jar.sha1 new file mode 100644 index 0000000000000..670bd4c98a044 --- /dev/null +++ b/plugins/repository-azure/licenses/netty-codec-dns-4.1.94.Final.jar.sha1 @@ -0,0 +1 @@ +9180660dc8479e1594b60b02fc27404af0ea43a6 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-codec-http2-4.1.93.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-codec-http2-4.1.93.Final.jar.sha1 deleted file mode 100644 index 02423842d6244..0000000000000 --- a/plugins/repository-azure/licenses/netty-codec-http2-4.1.93.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -0f1625b43bde13ec057da0d2fe381ded2547a70e \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-codec-http2-4.1.94.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-codec-http2-4.1.94.Final.jar.sha1 new file mode 100644 index 0000000000000..e73026b412972 --- /dev/null +++ b/plugins/repository-azure/licenses/netty-codec-http2-4.1.94.Final.jar.sha1 @@ -0,0 +1 @@ +f651595784d6cca4cbca6a8ad74c48fceed6cea8 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-codec-socks-4.1.93.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-codec-socks-4.1.93.Final.jar.sha1 deleted file mode 100644 index 3e3f699a3b9a9..0000000000000 --- a/plugins/repository-azure/licenses/netty-codec-socks-4.1.93.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -81ca78969afc60073e47c3b7b361cc3839392c73 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-codec-socks-4.1.94.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-codec-socks-4.1.94.Final.jar.sha1 new file mode 100644 index 0000000000000..de2c4d00aef09 --- /dev/null +++ b/plugins/repository-azure/licenses/netty-codec-socks-4.1.94.Final.jar.sha1 @@ -0,0 +1 @@ +b9192c7cda295d75f236a13a0b1f5a008f05d516 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-handler-proxy-4.1.93.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-handler-proxy-4.1.93.Final.jar.sha1 deleted file mode 100644 index 1ac94e6579c89..0000000000000 --- a/plugins/repository-azure/licenses/netty-handler-proxy-4.1.93.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -c165c1efe1b9c0cc22546a057b530611a088768b \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-handler-proxy-4.1.94.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-handler-proxy-4.1.94.Final.jar.sha1 new file mode 100644 index 0000000000000..a2db8bece8f6f --- /dev/null +++ b/plugins/repository-azure/licenses/netty-handler-proxy-4.1.94.Final.jar.sha1 @@ -0,0 +1 @@ +26ba9d30b8f7b095155b9ac63378d6d9386d85c3 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-resolver-dns-4.1.93.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-resolver-dns-4.1.93.Final.jar.sha1 deleted file mode 100644 index 67eb275207963..0000000000000 --- a/plugins/repository-azure/licenses/netty-resolver-dns-4.1.93.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -d5560348ab139dc9ea2dd9c0aa8ddffd0bf1b60f \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-resolver-dns-4.1.94.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-resolver-dns-4.1.94.Final.jar.sha1 new file mode 100644 index 0000000000000..2fa927b3b77ba --- /dev/null +++ b/plugins/repository-azure/licenses/netty-resolver-dns-4.1.94.Final.jar.sha1 @@ -0,0 +1 @@ +25bbe90e10685ce63c32bd0db56574cffffa28de \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-transport-native-unix-common-4.1.93.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-transport-native-unix-common-4.1.93.Final.jar.sha1 deleted file mode 100644 index 0f0acb2eccddf..0000000000000 --- a/plugins/repository-azure/licenses/netty-transport-native-unix-common-4.1.93.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -f4fb8b4c8da539091f43abcbb9f0389e48807eea \ No newline at end of file diff --git a/plugins/repository-azure/licenses/netty-transport-native-unix-common-4.1.94.Final.jar.sha1 b/plugins/repository-azure/licenses/netty-transport-native-unix-common-4.1.94.Final.jar.sha1 new file mode 100644 index 0000000000000..43bc960a347a1 --- /dev/null +++ b/plugins/repository-azure/licenses/netty-transport-native-unix-common-4.1.94.Final.jar.sha1 @@ -0,0 +1 @@ +3fa5f9d04b6b782d869d6e0657d896eeadca5866 \ No newline at end of file diff --git a/plugins/repository-hdfs/licenses/netty-all-4.1.93.Final.jar.sha1 b/plugins/repository-hdfs/licenses/netty-all-4.1.93.Final.jar.sha1 deleted file mode 100644 index a0f7bc4bf65ef..0000000000000 --- a/plugins/repository-hdfs/licenses/netty-all-4.1.93.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -408780d8b32ce4510f5408b06efabc2230aaf9b6 \ No newline at end of file diff --git a/plugins/repository-hdfs/licenses/netty-all-4.1.94.Final.jar.sha1 b/plugins/repository-hdfs/licenses/netty-all-4.1.94.Final.jar.sha1 new file mode 100644 index 0000000000000..6766770f61e78 --- /dev/null +++ b/plugins/repository-hdfs/licenses/netty-all-4.1.94.Final.jar.sha1 @@ -0,0 +1 @@ +2a7df0424eed81818157f22613f36b72487ceb34 \ No newline at end of file diff --git a/plugins/repository-s3/build.gradle b/plugins/repository-s3/build.gradle index 2250f2fc88f05..44fd45b265e82 100644 --- a/plugins/repository-s3/build.gradle +++ b/plugins/repository-s3/build.gradle @@ -439,7 +439,6 @@ thirdPartyAudit { 'com.aayushatharva.brotli4j.decoder.DecoderJNI$Status', 'com.aayushatharva.brotli4j.decoder.DecoderJNI$Wrapper', 'com.aayushatharva.brotli4j.encoder.BrotliEncoderChannel', - 'com.aayushatharva.brotli4j.encoder.Encoder', 'com.aayushatharva.brotli4j.encoder.Encoder$Mode', 'com.aayushatharva.brotli4j.encoder.Encoder$Parameters', diff --git a/plugins/repository-s3/licenses/netty-buffer-4.1.93.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-buffer-4.1.93.Final.jar.sha1 deleted file mode 100644 index 5c5a17a9466f1..0000000000000 --- a/plugins/repository-s3/licenses/netty-buffer-4.1.93.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -87fac21f4ef95157866b07b48e3c707a2f13c581 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-buffer-4.1.94.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-buffer-4.1.94.Final.jar.sha1 new file mode 100644 index 0000000000000..05b1c2a4d614e --- /dev/null +++ b/plugins/repository-s3/licenses/netty-buffer-4.1.94.Final.jar.sha1 @@ -0,0 +1 @@ +eec248b26f16e888688e5bb37b7eeda76b78d2f7 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-codec-4.1.93.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-codec-4.1.93.Final.jar.sha1 deleted file mode 100644 index 2b12a111335a2..0000000000000 --- a/plugins/repository-s3/licenses/netty-codec-4.1.93.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -503badb458b6586632be8d1f81aa4e5ab99a80fc \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-codec-4.1.94.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-codec-4.1.94.Final.jar.sha1 new file mode 100644 index 0000000000000..baa7e25f1ac49 --- /dev/null +++ b/plugins/repository-s3/licenses/netty-codec-4.1.94.Final.jar.sha1 @@ -0,0 +1 @@ +c70ef20ca338558147887df60f46341bc47f6900 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-codec-http-4.1.93.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-codec-http-4.1.93.Final.jar.sha1 deleted file mode 100644 index 6719e882e40fe..0000000000000 --- a/plugins/repository-s3/licenses/netty-codec-http-4.1.93.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -36acf0c94d03eb6ecef78a749a32cbb7dc0c57b4 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-codec-http-4.1.94.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-codec-http-4.1.94.Final.jar.sha1 new file mode 100644 index 0000000000000..8c018be2565e5 --- /dev/null +++ b/plugins/repository-s3/licenses/netty-codec-http-4.1.94.Final.jar.sha1 @@ -0,0 +1 @@ +9e5404764092c1f6305ad5719078f46ab228d587 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-codec-http2-4.1.93.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-codec-http2-4.1.93.Final.jar.sha1 deleted file mode 100644 index 02423842d6244..0000000000000 --- a/plugins/repository-s3/licenses/netty-codec-http2-4.1.93.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -0f1625b43bde13ec057da0d2fe381ded2547a70e \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-codec-http2-4.1.94.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-codec-http2-4.1.94.Final.jar.sha1 new file mode 100644 index 0000000000000..e73026b412972 --- /dev/null +++ b/plugins/repository-s3/licenses/netty-codec-http2-4.1.94.Final.jar.sha1 @@ -0,0 +1 @@ +f651595784d6cca4cbca6a8ad74c48fceed6cea8 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-common-4.1.93.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-common-4.1.93.Final.jar.sha1 deleted file mode 100644 index 2324a54dc5735..0000000000000 --- a/plugins/repository-s3/licenses/netty-common-4.1.93.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -1cfc49b91b0d3ddb30c9f7d8467e5d02ae8babdf \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-common-4.1.94.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-common-4.1.94.Final.jar.sha1 new file mode 100644 index 0000000000000..b787338551ede --- /dev/null +++ b/plugins/repository-s3/licenses/netty-common-4.1.94.Final.jar.sha1 @@ -0,0 +1 @@ +ad4ecf779ebc794cd351f57792f56ea01387b868 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-handler-4.1.93.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-handler-4.1.93.Final.jar.sha1 deleted file mode 100644 index 54e5b22047812..0000000000000 --- a/plugins/repository-s3/licenses/netty-handler-4.1.93.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -10f7ed9d8e1bfcba416074c70e5388be96116bfc \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-handler-4.1.94.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-handler-4.1.94.Final.jar.sha1 new file mode 100644 index 0000000000000..b08e85ba7adf8 --- /dev/null +++ b/plugins/repository-s3/licenses/netty-handler-4.1.94.Final.jar.sha1 @@ -0,0 +1 @@ +cd9121ce24d6d3f2898946d04b0ef3ec548b00b4 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-resolver-4.1.93.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-resolver-4.1.93.Final.jar.sha1 deleted file mode 100644 index c795762c2d7f1..0000000000000 --- a/plugins/repository-s3/licenses/netty-resolver-4.1.93.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -3860e99075f9e078364ed38f6d6fc8825b7a168a \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-resolver-4.1.94.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-resolver-4.1.94.Final.jar.sha1 new file mode 100644 index 0000000000000..4c9e4dda2b852 --- /dev/null +++ b/plugins/repository-s3/licenses/netty-resolver-4.1.94.Final.jar.sha1 @@ -0,0 +1 @@ +e96f649e8e9dcb29a1f8e95328b99c9eb6cf76c2 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-transport-4.1.93.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-transport-4.1.93.Final.jar.sha1 deleted file mode 100644 index 687cade3c7b3a..0000000000000 --- a/plugins/repository-s3/licenses/netty-transport-4.1.93.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -1a0894915c8027ce83b4d6a811c4e765955efd15 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-transport-4.1.94.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-transport-4.1.94.Final.jar.sha1 new file mode 100644 index 0000000000000..ed7760b8e15d1 --- /dev/null +++ b/plugins/repository-s3/licenses/netty-transport-4.1.94.Final.jar.sha1 @@ -0,0 +1 @@ +ec783a737f96991a87b1d5794e2f9eb2024d708a \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-transport-classes-epoll-4.1.93.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-transport-classes-epoll-4.1.93.Final.jar.sha1 deleted file mode 100644 index 81a180b455cdd..0000000000000 --- a/plugins/repository-s3/licenses/netty-transport-classes-epoll-4.1.93.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -79c2d95406bc2ef38c08c34efd16039919db80be \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-transport-classes-epoll-4.1.94.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-transport-classes-epoll-4.1.94.Final.jar.sha1 new file mode 100644 index 0000000000000..72a392ea2917d --- /dev/null +++ b/plugins/repository-s3/licenses/netty-transport-classes-epoll-4.1.94.Final.jar.sha1 @@ -0,0 +1 @@ +240e36cd5c2ffaf655913f8857f2d58b26394679 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-transport-native-unix-common-4.1.93.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-transport-native-unix-common-4.1.93.Final.jar.sha1 deleted file mode 100644 index 0f0acb2eccddf..0000000000000 --- a/plugins/repository-s3/licenses/netty-transport-native-unix-common-4.1.93.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -f4fb8b4c8da539091f43abcbb9f0389e48807eea \ No newline at end of file diff --git a/plugins/repository-s3/licenses/netty-transport-native-unix-common-4.1.94.Final.jar.sha1 b/plugins/repository-s3/licenses/netty-transport-native-unix-common-4.1.94.Final.jar.sha1 new file mode 100644 index 0000000000000..43bc960a347a1 --- /dev/null +++ b/plugins/repository-s3/licenses/netty-transport-native-unix-common-4.1.94.Final.jar.sha1 @@ -0,0 +1 @@ +3fa5f9d04b6b782d869d6e0657d896eeadca5866 \ No newline at end of file diff --git a/plugins/transport-nio/build.gradle b/plugins/transport-nio/build.gradle index 42d0d40a1d449..8c0ee8ba718ac 100644 --- a/plugins/transport-nio/build.gradle +++ b/plugins/transport-nio/build.gradle @@ -62,7 +62,6 @@ thirdPartyAudit { 'com.aayushatharva.brotli4j.decoder.DecoderJNI$Status', 'com.aayushatharva.brotli4j.decoder.DecoderJNI$Wrapper', 'com.aayushatharva.brotli4j.encoder.BrotliEncoderChannel', - 'com.aayushatharva.brotli4j.encoder.Encoder', 'com.aayushatharva.brotli4j.encoder.Encoder$Mode', 'com.aayushatharva.brotli4j.encoder.Encoder$Parameters', diff --git a/plugins/transport-nio/licenses/netty-buffer-4.1.93.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-buffer-4.1.93.Final.jar.sha1 deleted file mode 100644 index 5c5a17a9466f1..0000000000000 --- a/plugins/transport-nio/licenses/netty-buffer-4.1.93.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -87fac21f4ef95157866b07b48e3c707a2f13c581 \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-buffer-4.1.94.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-buffer-4.1.94.Final.jar.sha1 new file mode 100644 index 0000000000000..05b1c2a4d614e --- /dev/null +++ b/plugins/transport-nio/licenses/netty-buffer-4.1.94.Final.jar.sha1 @@ -0,0 +1 @@ +eec248b26f16e888688e5bb37b7eeda76b78d2f7 \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-codec-4.1.93.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-codec-4.1.93.Final.jar.sha1 deleted file mode 100644 index 2b12a111335a2..0000000000000 --- a/plugins/transport-nio/licenses/netty-codec-4.1.93.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -503badb458b6586632be8d1f81aa4e5ab99a80fc \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-codec-4.1.94.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-codec-4.1.94.Final.jar.sha1 new file mode 100644 index 0000000000000..baa7e25f1ac49 --- /dev/null +++ b/plugins/transport-nio/licenses/netty-codec-4.1.94.Final.jar.sha1 @@ -0,0 +1 @@ +c70ef20ca338558147887df60f46341bc47f6900 \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-codec-http-4.1.93.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-codec-http-4.1.93.Final.jar.sha1 deleted file mode 100644 index 6719e882e40fe..0000000000000 --- a/plugins/transport-nio/licenses/netty-codec-http-4.1.93.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -36acf0c94d03eb6ecef78a749a32cbb7dc0c57b4 \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-codec-http-4.1.94.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-codec-http-4.1.94.Final.jar.sha1 new file mode 100644 index 0000000000000..8c018be2565e5 --- /dev/null +++ b/plugins/transport-nio/licenses/netty-codec-http-4.1.94.Final.jar.sha1 @@ -0,0 +1 @@ +9e5404764092c1f6305ad5719078f46ab228d587 \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-common-4.1.93.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-common-4.1.93.Final.jar.sha1 deleted file mode 100644 index 2324a54dc5735..0000000000000 --- a/plugins/transport-nio/licenses/netty-common-4.1.93.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -1cfc49b91b0d3ddb30c9f7d8467e5d02ae8babdf \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-common-4.1.94.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-common-4.1.94.Final.jar.sha1 new file mode 100644 index 0000000000000..b787338551ede --- /dev/null +++ b/plugins/transport-nio/licenses/netty-common-4.1.94.Final.jar.sha1 @@ -0,0 +1 @@ +ad4ecf779ebc794cd351f57792f56ea01387b868 \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-handler-4.1.93.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-handler-4.1.93.Final.jar.sha1 deleted file mode 100644 index 54e5b22047812..0000000000000 --- a/plugins/transport-nio/licenses/netty-handler-4.1.93.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -10f7ed9d8e1bfcba416074c70e5388be96116bfc \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-handler-4.1.94.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-handler-4.1.94.Final.jar.sha1 new file mode 100644 index 0000000000000..b08e85ba7adf8 --- /dev/null +++ b/plugins/transport-nio/licenses/netty-handler-4.1.94.Final.jar.sha1 @@ -0,0 +1 @@ +cd9121ce24d6d3f2898946d04b0ef3ec548b00b4 \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-resolver-4.1.93.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-resolver-4.1.93.Final.jar.sha1 deleted file mode 100644 index c795762c2d7f1..0000000000000 --- a/plugins/transport-nio/licenses/netty-resolver-4.1.93.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -3860e99075f9e078364ed38f6d6fc8825b7a168a \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-resolver-4.1.94.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-resolver-4.1.94.Final.jar.sha1 new file mode 100644 index 0000000000000..4c9e4dda2b852 --- /dev/null +++ b/plugins/transport-nio/licenses/netty-resolver-4.1.94.Final.jar.sha1 @@ -0,0 +1 @@ +e96f649e8e9dcb29a1f8e95328b99c9eb6cf76c2 \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-transport-4.1.93.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-transport-4.1.93.Final.jar.sha1 deleted file mode 100644 index 687cade3c7b3a..0000000000000 --- a/plugins/transport-nio/licenses/netty-transport-4.1.93.Final.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -1a0894915c8027ce83b4d6a811c4e765955efd15 \ No newline at end of file diff --git a/plugins/transport-nio/licenses/netty-transport-4.1.94.Final.jar.sha1 b/plugins/transport-nio/licenses/netty-transport-4.1.94.Final.jar.sha1 new file mode 100644 index 0000000000000..ed7760b8e15d1 --- /dev/null +++ b/plugins/transport-nio/licenses/netty-transport-4.1.94.Final.jar.sha1 @@ -0,0 +1 @@ +ec783a737f96991a87b1d5794e2f9eb2024d708a \ No newline at end of file From 2652d532a6d30fd6e2d001969bbbc3b72a2a4a8e Mon Sep 17 00:00:00 2001 From: Rishikesh Pasham <62345295+Rishikesh1159@users.noreply.github.com> Date: Thu, 22 Jun 2023 13:44:24 -0700 Subject: [PATCH 058/109] [Segment Replication + Remote Storage] Remove implicit replication strategy change when configuring remote storage with cluster setting (#8186) * Removing overriding of replication type setting in updateRemoteStoreSettings() Signed-off-by: Rishikesh1159 * fixing failing test testRemoteStoreNoUserOverrideConflictingReplicationTypeIndexSettings() by modifying exception message. Signed-off-by: Rishikesh1159 * Fix failing tests. Signed-off-by: Rishikesh1159 * remove unnecessary system index integ test. Signed-off-by: Rishikesh1159 * revert previous change of removing system index integ test. Signed-off-by: Rishikesh1159 * Address comments on PR. Signed-off-by: Rishikesh1159 * Address comments. Signed-off-by: Rishikesh1159 * Address comments on PR. Signed-off-by: Rishikesh1159 --------- Signed-off-by: Rishikesh1159 Signed-off-by: Rishikesh Pasham <62345295+Rishikesh1159@users.noreply.github.com> --- ...CreateRemoteIndexClusterDefaultDocRep.java | 2 +- .../remotestore/CreateRemoteIndexIT.java | 41 +++++++++++++++++++ .../cluster/metadata/IndexMetadata.java | 8 +--- .../metadata/MetadataCreateIndexService.java | 15 ++++--- .../MetadataCreateIndexServiceTests.java | 2 +- 5 files changed, 55 insertions(+), 13 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/CreateRemoteIndexClusterDefaultDocRep.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/CreateRemoteIndexClusterDefaultDocRep.java index 7b43ffb597661..40b608388736a 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/CreateRemoteIndexClusterDefaultDocRep.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/CreateRemoteIndexClusterDefaultDocRep.java @@ -70,7 +70,7 @@ public void testDefaultRemoteStoreNoUserOverride() throws Exception { ); assertThat( exc.getMessage(), - containsString("Cannot enable [index.remote_store.enabled] when [cluster.indices.replication.strategy] is DOCUMENT") + containsString("Cannot enable [index.remote_store.enabled] when [index.replication.type] is DOCUMENT") ); } diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/CreateRemoteIndexIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/CreateRemoteIndexIT.java index fda344acad166..cf53fa7fac2cd 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/CreateRemoteIndexIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/CreateRemoteIndexIT.java @@ -16,11 +16,18 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.FeatureFlags; +import org.opensearch.indices.SystemIndexDescriptor; import org.opensearch.index.IndexSettings; import org.opensearch.indices.replication.common.ReplicationType; +import org.opensearch.plugins.Plugin; +import org.opensearch.plugins.SystemIndexPlugin; import org.opensearch.test.FeatureFlagSetter; import org.opensearch.test.OpenSearchIntegTestCase; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; + import static org.hamcrest.Matchers.containsString; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_STORE_ENABLED; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_STORE_REPOSITORY; @@ -59,6 +66,20 @@ protected Settings nodeSettings(int nodeOriginal) { return builder.build(); } + public static class TestPlugin extends Plugin implements SystemIndexPlugin { + @Override + public Collection getSystemIndexDescriptors(Settings settings) { + return Collections.singletonList( + new SystemIndexDescriptor(SYSTEM_INDEX_NAME, "System index for [" + getTestClass().getName() + ']') + ); + } + } + + @Override + protected Collection> nodePlugins() { + return Arrays.asList(CreateRemoteIndexIT.TestPlugin.class); + } + @Override protected Settings featureFlagSettings() { return Settings.builder().put(super.featureFlagSettings()).put(FeatureFlags.REMOTE_STORE, "true").build(); @@ -107,6 +128,26 @@ public void testDefaultRemoteStoreNoUserOverride() throws Exception { ); } + private static final String SYSTEM_INDEX_NAME = ".test-system-index"; + + public void testSystemIndexWithRemoteStoreClusterSetting() throws Exception { + IllegalArgumentException illegalArgumentException = expectThrows( + IllegalArgumentException.class, + () -> createIndex(SYSTEM_INDEX_NAME) + ); + assertThat( + illegalArgumentException.getMessage(), + containsString( + "Cannot enable [" + + SETTING_REMOTE_STORE_ENABLED + + "] when [" + + SETTING_REPLICATION_TYPE + + "] is " + + ReplicationType.DOCUMENT + ) + ); + } + public void testRemoteStoreDisabledByUser() throws Exception { Settings settings = Settings.builder() .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) 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 93a942fcbf7eb..ddc603541e038 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/IndexMetadata.java @@ -91,7 +91,6 @@ import static org.opensearch.cluster.node.DiscoveryNodeFilters.OpType.OR; import static org.opensearch.common.settings.Settings.readSettingsFromStream; import static org.opensearch.common.settings.Settings.writeSettingsToStream; -import static org.opensearch.indices.IndicesService.CLUSTER_REPLICATION_TYPE_SETTING; /** * Index metadata information @@ -311,11 +310,8 @@ public void validate(final Boolean value) {} @Override public void validate(final Boolean value, final Map, Object> settings) { - final Object clusterSettingReplicationType = settings.get(CLUSTER_REPLICATION_TYPE_SETTING); final Object replicationType = settings.get(INDEX_REPLICATION_TYPE_SETTING); - if ((replicationType).equals(ReplicationType.SEGMENT) == false - && (clusterSettingReplicationType).equals(ReplicationType.SEGMENT) == false - && value == true) { + if (ReplicationType.SEGMENT.equals(replicationType) == false && value) { throw new IllegalArgumentException( "To enable " + INDEX_REMOTE_STORE_ENABLED_SETTING.getKey() @@ -329,7 +325,7 @@ public void validate(final Boolean value, final Map, Object> settings @Override public Iterator> settings() { - final List> settings = List.of(INDEX_REPLICATION_TYPE_SETTING, CLUSTER_REPLICATION_TYPE_SETTING); + final List> settings = List.of(INDEX_REPLICATION_TYPE_SETTING); return settings.iterator(); } }, 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 6e22de7c72d07..a2a11b6bc9271 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java @@ -951,7 +951,13 @@ private static void updateReplicationStrategy( } if (CLUSTER_REPLICATION_TYPE_SETTING.exists(clusterSettings) && INDEX_REPLICATION_TYPE_SETTING.exists(requestSettings) == false) { settingsBuilder.put(SETTING_REPLICATION_TYPE, CLUSTER_REPLICATION_TYPE_SETTING.get(clusterSettings)); + return; + } + if (INDEX_REPLICATION_TYPE_SETTING.exists(requestSettings) == true) { + settingsBuilder.put(SETTING_REPLICATION_TYPE, INDEX_REPLICATION_TYPE_SETTING.get(requestSettings)); + return; } + settingsBuilder.put(SETTING_REPLICATION_TYPE, CLUSTER_REPLICATION_TYPE_SETTING.getDefault(clusterSettings)); } /** @@ -967,20 +973,19 @@ private static void updateRemoteStoreSettings(Settings.Builder settingsBuilder, return; } - // Verify REPLICATION_TYPE cluster level setting is not conflicting with Remote Store - if (INDEX_REPLICATION_TYPE_SETTING.exists(requestSettings) == false - && CLUSTER_REPLICATION_TYPE_SETTING.get(clusterSettings).equals(ReplicationType.DOCUMENT)) { + // Verify index has replication type as SEGMENT + if (ReplicationType.DOCUMENT.equals(ReplicationType.parseString(settingsBuilder.get(SETTING_REPLICATION_TYPE)))) { throw new IllegalArgumentException( "Cannot enable [" + SETTING_REMOTE_STORE_ENABLED + "] when [" - + CLUSTER_REPLICATION_TYPE_SETTING.getKey() + + SETTING_REPLICATION_TYPE + "] is " + ReplicationType.DOCUMENT ); } - settingsBuilder.put(SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT).put(SETTING_REMOTE_STORE_ENABLED, true); + settingsBuilder.put(SETTING_REMOTE_STORE_ENABLED, true); String remoteStoreRepo; if (Objects.equals(requestSettings.get(INDEX_REMOTE_STORE_ENABLED_SETTING.getKey()), "true")) { diff --git a/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java b/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java index fbb86620bdc23..206c8914a61ee 100644 --- a/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java +++ b/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java @@ -1226,7 +1226,7 @@ public void testRemoteStoreNoUserOverrideConflictingReplicationTypeIndexSettings ); assertThat( exc.getMessage(), - containsString("Cannot enable [index.remote_store.enabled] when [cluster.indices.replication.strategy] is DOCUMENT") + containsString("Cannot enable [index.remote_store.enabled] when [index.replication.type] is DOCUMENT") ); } From 4be891829a39d72d0d3e66a4740dcff037cc0e10 Mon Sep 17 00:00:00 2001 From: Andrew Ross Date: Thu, 22 Jun 2023 14:08:17 -0700 Subject: [PATCH 059/109] Update compressed cluster state version guard (#8185) This change was backported to 2.x in #7945. Signed-off-by: Andrew Ross --- .../java/org/opensearch/cluster/coordination/JoinHelper.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/opensearch/cluster/coordination/JoinHelper.java b/server/src/main/java/org/opensearch/cluster/coordination/JoinHelper.java index 08cfea1abf270..114051311ab97 100644 --- a/server/src/main/java/org/opensearch/cluster/coordination/JoinHelper.java +++ b/server/src/main/java/org/opensearch/cluster/coordination/JoinHelper.java @@ -447,7 +447,7 @@ public String executor() { } public void sendValidateJoinRequest(DiscoveryNode node, ClusterState state, ActionListener listener) { - if (node.getVersion().before(Version.V_3_0_0)) { + if (node.getVersion().before(Version.V_2_9_0)) { transportService.sendRequest( node, VALIDATE_JOIN_ACTION_NAME, From 84dffac7e5211dc2b74da4a5605c9845816d7942 Mon Sep 17 00:00:00 2001 From: Subhobrata Dey Date: Thu, 22 Jun 2023 14:48:33 -0700 Subject: [PATCH 060/109] Add hnsw graph based storage & query layer to events correlation engine (#7350) Signed-off-by: Subhobrata Dey --- .../CorrelationVectorsEngineIT.java | 312 ++++++++++++++ .../correlation/EventsCorrelationPlugin.java | 39 +- .../core/index/CorrelationParamsContext.java | 148 +++++++ .../correlation/core/index/VectorField.java | 51 +++ .../BasePerFieldCorrelationVectorsFormat.java | 104 +++++ .../index/codec/CorrelationCodecService.java | 38 ++ .../index/codec/CorrelationCodecVersion.java | 103 +++++ .../correlation950/CorrelationCodec.java | 46 ++ .../PerFieldCorrelationVectorsFormat.java | 35 ++ .../codec/correlation950/package-info.java | 12 + .../core/index/codec/package-info.java | 12 + .../mapper/CorrelationVectorFieldMapper.java | 174 ++++++++ .../core/index/mapper/VectorFieldMapper.java | 399 ++++++++++++++++++ .../core/index/mapper/package-info.java | 12 + .../correlation/core/index/package-info.java | 12 + .../index/query/CorrelationQueryBuilder.java | 330 +++++++++++++++ .../index/query/CorrelationQueryFactory.java | 142 +++++++ .../core/index/query/package-info.java | 12 + .../services/org.apache.lucene.codecs.Codec | 1 + .../index/CorrelationParamsContextTests.java | 170 ++++++++ .../core/index/VectorFieldTests.java | 83 ++++ .../correlation950/CorrelationCodecTests.java | 120 ++++++ .../CorrelationVectorFieldMapperTests.java | 309 ++++++++++++++ .../query/CorrelationQueryBuilderTests.java | 268 ++++++++++++ .../EventsCorrelationSettingsTests.java | 58 +++ 25 files changed, 2989 insertions(+), 1 deletion(-) create mode 100644 plugins/events-correlation-engine/src/javaRestTest/java/org/opensearch/plugin/correlation/CorrelationVectorsEngineIT.java create mode 100644 plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/CorrelationParamsContext.java create mode 100644 plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/VectorField.java create mode 100644 plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/codec/BasePerFieldCorrelationVectorsFormat.java create mode 100644 plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/codec/CorrelationCodecService.java create mode 100644 plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/codec/CorrelationCodecVersion.java create mode 100644 plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/codec/correlation950/CorrelationCodec.java create mode 100644 plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/codec/correlation950/PerFieldCorrelationVectorsFormat.java create mode 100644 plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/codec/correlation950/package-info.java create mode 100644 plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/codec/package-info.java create mode 100644 plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/mapper/CorrelationVectorFieldMapper.java create mode 100644 plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/mapper/VectorFieldMapper.java create mode 100644 plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/mapper/package-info.java create mode 100644 plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/package-info.java create mode 100644 plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/query/CorrelationQueryBuilder.java create mode 100644 plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/query/CorrelationQueryFactory.java create mode 100644 plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/query/package-info.java create mode 100644 plugins/events-correlation-engine/src/main/resources/META-INF/services/org.apache.lucene.codecs.Codec create mode 100644 plugins/events-correlation-engine/src/test/java/org/opensearch/plugin/correlation/core/index/CorrelationParamsContextTests.java create mode 100644 plugins/events-correlation-engine/src/test/java/org/opensearch/plugin/correlation/core/index/VectorFieldTests.java create mode 100644 plugins/events-correlation-engine/src/test/java/org/opensearch/plugin/correlation/core/index/codec/correlation950/CorrelationCodecTests.java create mode 100644 plugins/events-correlation-engine/src/test/java/org/opensearch/plugin/correlation/core/index/mapper/CorrelationVectorFieldMapperTests.java create mode 100644 plugins/events-correlation-engine/src/test/java/org/opensearch/plugin/correlation/core/index/query/CorrelationQueryBuilderTests.java create mode 100644 plugins/events-correlation-engine/src/test/java/org/opensearch/plugin/correlation/settings/EventsCorrelationSettingsTests.java diff --git a/plugins/events-correlation-engine/src/javaRestTest/java/org/opensearch/plugin/correlation/CorrelationVectorsEngineIT.java b/plugins/events-correlation-engine/src/javaRestTest/java/org/opensearch/plugin/correlation/CorrelationVectorsEngineIT.java new file mode 100644 index 0000000000000..f8736f05dadfc --- /dev/null +++ b/plugins/events-correlation-engine/src/javaRestTest/java/org/opensearch/plugin/correlation/CorrelationVectorsEngineIT.java @@ -0,0 +1,312 @@ +/* + * 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.plugin.correlation; + +import org.apache.hc.core5.http.Header; +import org.apache.hc.core5.http.HttpEntity; +import org.apache.lucene.index.VectorSimilarityFunction; +import org.junit.Assert; +import org.opensearch.client.Request; +import org.opensearch.client.RequestOptions; +import org.opensearch.client.Response; +import org.opensearch.client.ResponseException; +import org.opensearch.client.RestClient; +import org.opensearch.client.WarningsHandler; +import org.opensearch.common.Strings; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.xcontent.XContentFactory; +import org.opensearch.common.xcontent.XContentType; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.index.IndexSettings; +import org.opensearch.rest.RestStatus; +import org.opensearch.test.rest.OpenSearchRestTestCase; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * Correlation Vectors Engine e2e tests + */ +public class CorrelationVectorsEngineIT extends OpenSearchRestTestCase { + + private static final int DIMENSION = 4; + private static final String PROPERTIES_FIELD_NAME = "properties"; + private static final String TYPE_FIELD_NAME = "type"; + private static final String CORRELATION_VECTOR_TYPE = "correlation_vector"; + private static final String DIMENSION_FIELD_NAME = "dimension"; + private static final int M = 16; + private static final int EF_CONSTRUCTION = 128; + private static final String INDEX_NAME = "test-index-1"; + private static final Float[][] TEST_VECTORS = new Float[][] { + { 1.0f, 1.0f, 1.0f, 1.0f }, + { 2.0f, 2.0f, 2.0f, 2.0f }, + { 3.0f, 3.0f, 3.0f, 3.0f } }; + private static final float[][] TEST_QUERY_VECTORS = new float[][] { + { 1.0f, 1.0f, 1.0f, 1.0f }, + { 2.0f, 2.0f, 2.0f, 2.0f }, + { 3.0f, 3.0f, 3.0f, 3.0f } }; + private static final Map> VECTOR_SIMILARITY_TO_SCORE = Map.of( + VectorSimilarityFunction.EUCLIDEAN, + (similarity) -> 1 / (1 + similarity), + VectorSimilarityFunction.DOT_PRODUCT, + (similarity) -> (1 + similarity) / 2, + VectorSimilarityFunction.COSINE, + (similarity) -> (1 + similarity) / 2 + ); + + /** + * test the e2e storage and query layer of events-correlation-engine + * @throws IOException IOException + */ + @SuppressWarnings("unchecked") + public void testQuery() throws IOException { + String textField = "text-field"; + String luceneField = "lucene-field"; + XContentBuilder builder = XContentFactory.jsonBuilder() + .startObject() + .startObject(PROPERTIES_FIELD_NAME) + .startObject(textField) + .field(TYPE_FIELD_NAME, "text") + .endObject() + .startObject(luceneField) + .field(TYPE_FIELD_NAME, CORRELATION_VECTOR_TYPE) + .field(DIMENSION_FIELD_NAME, DIMENSION) + .startObject("correlation_ctx") + .field("similarityFunction", VectorSimilarityFunction.EUCLIDEAN.name()) + .startObject("parameters") + .field("m", M) + .field("ef_construction", EF_CONSTRUCTION) + .endObject() + .endObject() + .endObject() + .endObject() + .endObject(); + + String mapping = Strings.toString(builder); + createTestIndexWithMappingJson(client(), INDEX_NAME, mapping, getCorrelationDefaultIndexSettings()); + + for (int idx = 0; idx < TEST_VECTORS.length; ++idx) { + addCorrelationDoc( + INDEX_NAME, + String.valueOf(idx + 1), + List.of(textField, luceneField), + List.of(java.util.UUID.randomUUID().toString(), TEST_VECTORS[idx]) + ); + } + refreshAllIndices(); + Assert.assertEquals(TEST_VECTORS.length, getDocCount(INDEX_NAME)); + + int k = 2; + for (float[] query : TEST_QUERY_VECTORS) { + + String correlationQuery = "{\n" + + " \"query\": {\n" + + " \"correlation\": {\n" + + " \"lucene-field\": {\n" + + " \"vector\": \n" + + Arrays.toString(query) + + " ,\n" + + " \"k\": 2,\n" + + " \"boost\": 1\n" + + " }\n" + + " }\n" + + " }\n" + + "}"; + + Response response = searchCorrelationIndex(INDEX_NAME, correlationQuery, k); + Map responseBody = entityAsMap(response); + Assert.assertEquals(2, ((List) ((Map) responseBody.get("hits")).get("hits")).size()); + @SuppressWarnings("unchecked") + double actualScore1 = Double.parseDouble( + ((List>) ((Map) responseBody.get("hits")).get("hits")).get(0).get("_score").toString() + ); + @SuppressWarnings("unchecked") + double actualScore2 = Double.parseDouble( + ((List>) ((Map) responseBody.get("hits")).get("hits")).get(1).get("_score").toString() + ); + @SuppressWarnings("unchecked") + List hit1 = ((Map>) ((List>) ((Map) responseBody.get("hits")) + .get("hits")).get(0).get("_source")).get(luceneField).stream().map(Double::floatValue).collect(Collectors.toList()); + float[] resultVector1 = new float[hit1.size()]; + for (int i = 0; i < hit1.size(); ++i) { + resultVector1[i] = hit1.get(i); + } + + @SuppressWarnings("unchecked") + List hit2 = ((Map>) ((List>) ((Map) responseBody.get("hits")) + .get("hits")).get(1).get("_source")).get(luceneField).stream().map(Double::floatValue).collect(Collectors.toList()); + float[] resultVector2 = new float[hit2.size()]; + for (int i = 0; i < hit2.size(); ++i) { + resultVector2[i] = hit2.get(i); + } + + double rawScore1 = VectorSimilarityFunction.EUCLIDEAN.compare(resultVector1, query); + Assert.assertEquals(rawScore1, actualScore1, 0.0001); + double rawScore2 = VectorSimilarityFunction.EUCLIDEAN.compare(resultVector2, query); + Assert.assertEquals(rawScore2, actualScore2, 0.0001); + } + } + + /** + * unhappy test for the e2e storage and query layer of events-correlation-engine with no index exist + */ + public void testQueryWithNoIndexExist() { + float[] query = new float[] { 1.0f, 1.0f, 1.0f, 1.0f }; + String correlationQuery = "{\n" + + " \"query\": {\n" + + " \"correlation\": {\n" + + " \"lucene-field\": {\n" + + " \"vector\": \n" + + Arrays.toString(query) + + " ,\n" + + " \"k\": 2,\n" + + " \"boost\": 1\n" + + " }\n" + + " }\n" + + " }\n" + + "}"; + Exception ex = assertThrows(ResponseException.class, () -> { searchCorrelationIndex(INDEX_NAME, correlationQuery, 2); }); + String expectedMessage = String.format(Locale.ROOT, "no such index [%s]", INDEX_NAME); + String actualMessage = ex.getMessage(); + Assert.assertTrue(actualMessage.contains(expectedMessage)); + } + + /** + * unhappy test for the e2e storage and query layer of events-correlation-engine with wrong mapping + */ + public void testQueryWithWrongMapping() throws IOException { + String textField = "text-field"; + String luceneField = "lucene-field"; + XContentBuilder builder = XContentFactory.jsonBuilder() + .startObject() + .startObject(PROPERTIES_FIELD_NAME) + .startObject(textField) + .field(TYPE_FIELD_NAME, "text") + .endObject() + .startObject(luceneField) + .field(TYPE_FIELD_NAME, CORRELATION_VECTOR_TYPE) + .field("test", DIMENSION) + .startObject("correlation_ctx") + .field("similarityFunction", VectorSimilarityFunction.EUCLIDEAN.name()) + .startObject("parameters") + .field("m", M) + .field("ef_construction", EF_CONSTRUCTION) + .endObject() + .endObject() + .endObject() + .endObject() + .endObject(); + + String mapping = Strings.toString(builder); + Exception ex = assertThrows(ResponseException.class, () -> { + createTestIndexWithMappingJson(client(), INDEX_NAME, mapping, getCorrelationDefaultIndexSettings()); + }); + + String expectedMessage = String.format( + Locale.ROOT, + "unknown parameter [test] on mapper [%s] of type [correlation_vector]", + luceneField + ); + String actualMessage = ex.getMessage(); + Assert.assertTrue(actualMessage.contains(expectedMessage)); + } + + private String createTestIndexWithMappingJson(RestClient client, String index, String mapping, Settings settings) throws IOException { + Request request = new Request("PUT", "/" + index); + String entity = "{\"settings\": " + Strings.toString(XContentType.JSON, settings); + if (mapping != null) { + entity = entity + ",\"mappings\" : " + mapping; + } + + entity = entity + "}"; + if (!settings.getAsBoolean(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true)) { + expectSoftDeletesWarning(request, index); + } + + request.setJsonEntity(entity); + client.performRequest(request); + return index; + } + + private Settings getCorrelationDefaultIndexSettings() { + return Settings.builder().put("number_of_shards", 1).put("number_of_replicas", 0).put("index.correlation", true).build(); + } + + private void addCorrelationDoc(String index, String docId, List fieldNames, List vectors) throws IOException { + Request request = new Request("POST", "/" + index + "/_doc/" + docId + "?refresh=true"); + + XContentBuilder builder = XContentFactory.jsonBuilder().startObject(); + for (int i = 0; i < fieldNames.size(); i++) { + builder.field(fieldNames.get(i), vectors.get(i)); + } + builder.endObject(); + + request.setJsonEntity(Strings.toString(builder)); + Response response = client().performRequest(request); + assertEquals(request.getEndpoint() + ": failed", RestStatus.CREATED, RestStatus.fromCode(response.getStatusLine().getStatusCode())); + } + + private Response searchCorrelationIndex(String index, String correlationQuery, int resultSize) throws IOException { + Request request = new Request("POST", "/" + index + "/_search"); + + request.addParameter("size", Integer.toString(resultSize)); + request.addParameter("explain", Boolean.toString(true)); + request.addParameter("search_type", "query_then_fetch"); + request.setJsonEntity(correlationQuery); + + Response response = client().performRequest(request); + Assert.assertEquals("Search failed", RestStatus.OK, restStatus(response)); + return response; + } + + private int getDocCount(String index) throws IOException { + Response response = makeRequest( + client(), + "GET", + String.format(Locale.getDefault(), "/%s/_count", index), + Collections.emptyMap(), + null + ); + Assert.assertEquals(RestStatus.OK, restStatus(response)); + return Integer.parseInt(entityAsMap(response).get("count").toString()); + } + + private Response makeRequest( + RestClient client, + String method, + String endpoint, + Map params, + HttpEntity entity, + Header... headers + ) throws IOException { + Request request = new Request(method, endpoint); + RequestOptions.Builder options = RequestOptions.DEFAULT.toBuilder(); + options.setWarningsHandler(WarningsHandler.PERMISSIVE); + + for (Header header : headers) { + options.addHeader(header.getName(), header.getValue()); + } + request.setOptions(options.build()); + request.addParameters(params); + if (entity != null) { + request.setEntity(entity); + } + return client.performRequest(request); + } + + private RestStatus restStatus(Response response) { + return RestStatus.fromCode(response.getStatusLine().getStatusCode()); + } +} diff --git a/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/EventsCorrelationPlugin.java b/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/EventsCorrelationPlugin.java index 443a794bd99df..6945f21a0fd7c 100644 --- a/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/EventsCorrelationPlugin.java +++ b/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/EventsCorrelationPlugin.java @@ -23,13 +23,23 @@ import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.env.Environment; import org.opensearch.env.NodeEnvironment; +import org.opensearch.index.IndexSettings; +import org.opensearch.index.codec.CodecServiceFactory; +import org.opensearch.index.mapper.Mapper; +import org.opensearch.plugin.correlation.core.index.codec.CorrelationCodecService; +import org.opensearch.plugin.correlation.core.index.mapper.CorrelationVectorFieldMapper; +import org.opensearch.plugin.correlation.core.index.mapper.VectorFieldMapper; +import org.opensearch.plugin.correlation.core.index.query.CorrelationQueryBuilder; import org.opensearch.plugin.correlation.rules.action.IndexCorrelationRuleAction; import org.opensearch.plugin.correlation.rules.resthandler.RestIndexCorrelationRuleAction; import org.opensearch.plugin.correlation.rules.transport.TransportIndexCorrelationRuleAction; import org.opensearch.plugin.correlation.settings.EventsCorrelationSettings; import org.opensearch.plugin.correlation.utils.CorrelationRuleIndices; import org.opensearch.plugins.ActionPlugin; +import org.opensearch.plugins.EnginePlugin; +import org.opensearch.plugins.MapperPlugin; import org.opensearch.plugins.Plugin; +import org.opensearch.plugins.SearchPlugin; import org.opensearch.repositories.RepositoriesService; import org.opensearch.rest.RestController; import org.opensearch.rest.RestHandler; @@ -38,13 +48,16 @@ import org.opensearch.watcher.ResourceWatcherService; import java.util.Collection; +import java.util.Collections; import java.util.List; +import java.util.Map; +import java.util.Optional; import java.util.function.Supplier; /** * Plugin class for events-correlation-engine */ -public class EventsCorrelationPlugin extends Plugin implements ActionPlugin { +public class EventsCorrelationPlugin extends Plugin implements ActionPlugin, MapperPlugin, SearchPlugin, EnginePlugin { /** * events-correlation-engine base uri @@ -93,6 +106,30 @@ public List getRestHandlers( return List.of(new RestIndexCorrelationRuleAction()); } + @Override + public Map getMappers() { + return Collections.singletonMap(CorrelationVectorFieldMapper.CONTENT_TYPE, new VectorFieldMapper.TypeParser()); + } + + @Override + public Optional getCustomCodecServiceFactory(IndexSettings indexSettings) { + if (indexSettings.getValue(EventsCorrelationSettings.IS_CORRELATION_INDEX_SETTING)) { + return Optional.of(CorrelationCodecService::new); + } + return Optional.empty(); + } + + @Override + public List> getQueries() { + return Collections.singletonList( + new QuerySpec<>( + CorrelationQueryBuilder.NAME_FIELD.getPreferredName(), + CorrelationQueryBuilder::new, + CorrelationQueryBuilder::parse + ) + ); + } + @Override public List> getActions() { return List.of(new ActionPlugin.ActionHandler<>(IndexCorrelationRuleAction.INSTANCE, TransportIndexCorrelationRuleAction.class)); diff --git a/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/CorrelationParamsContext.java b/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/CorrelationParamsContext.java new file mode 100644 index 0000000000000..c7a23dabfd312 --- /dev/null +++ b/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/CorrelationParamsContext.java @@ -0,0 +1,148 @@ +/* + * 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.plugin.correlation.core.index; + +import org.apache.lucene.index.VectorSimilarityFunction; +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.xcontent.ToXContentFragment; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.index.mapper.MapperParsingException; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Locale; +import java.util.Map; + +/** + * Defines vector similarity function, m and ef_construction hyper parameters field mappings for correlation_vector type. + * + * @opensearch.internal + */ +public class CorrelationParamsContext implements ToXContentFragment, Writeable { + + /** + * Vector Similarity Function field + */ + public static final String VECTOR_SIMILARITY_FUNCTION = "similarityFunction"; + /** + * Parameters field to define m and ef_construction + */ + public static final String PARAMETERS = "parameters"; + + private final VectorSimilarityFunction similarityFunction; + private final Map parameters; + + /** + * Parameterized ctor for CorrelationParamsContext + * @param similarityFunction Vector Similarity Function + * @param parameters Parameters to define m and ef_construction + */ + public CorrelationParamsContext(VectorSimilarityFunction similarityFunction, Map parameters) { + this.similarityFunction = similarityFunction; + this.parameters = parameters; + } + + /** + * Parameterized ctor for CorrelationParamsContext + * @param sin StreamInput + * @throws IOException IOException + */ + public CorrelationParamsContext(StreamInput sin) throws IOException { + this.similarityFunction = VectorSimilarityFunction.valueOf(sin.readString()); + if (sin.available() > 0) { + this.parameters = sin.readMap(); + } else { + this.parameters = null; + } + } + + /** + * Parse into CorrelationParamsContext + * @param in Object + * @return CorrelationParamsContext + */ + public static CorrelationParamsContext parse(Object in) { + if (!(in instanceof Map)) { + throw new MapperParsingException("Unable to parse CorrelationParamsContext"); + } + + @SuppressWarnings("unchecked") + Map contextMap = (Map) in; + VectorSimilarityFunction similarityFunction = VectorSimilarityFunction.EUCLIDEAN; + Map parameters = new HashMap<>(); + + if (contextMap.containsKey(VECTOR_SIMILARITY_FUNCTION)) { + Object value = contextMap.get(VECTOR_SIMILARITY_FUNCTION); + + if (value != null && !(value instanceof String)) { + throw new MapperParsingException(String.format(Locale.getDefault(), "%s must be a string", VECTOR_SIMILARITY_FUNCTION)); + } + + try { + similarityFunction = VectorSimilarityFunction.valueOf((String) value); + } catch (IllegalArgumentException ex) { + throw new MapperParsingException(String.format(Locale.getDefault(), "Invalid %s: %s", VECTOR_SIMILARITY_FUNCTION, value)); + } + } + if (contextMap.containsKey(PARAMETERS)) { + Object value = contextMap.get(PARAMETERS); + if (!(value instanceof Map)) { + throw new MapperParsingException("Unable to parse parameters for Correlation context"); + } + + @SuppressWarnings("unchecked") + Map valueMap = (Map) value; + parameters.putAll(valueMap); + } + return new CorrelationParamsContext(similarityFunction, parameters); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field(VECTOR_SIMILARITY_FUNCTION, similarityFunction.name()); + if (params == null) { + builder.field(PARAMETERS, (String) null); + } else { + builder.startObject(PARAMETERS); + for (Map.Entry parameter : parameters.entrySet()) { + builder.field(parameter.getKey(), parameter.getValue()); + } + builder.endObject(); + } + builder.endObject(); + return builder; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeString(similarityFunction.name()); + if (this.parameters != null) { + out.writeMap(parameters); + } + } + + /** + * get Vector Similarity Function + * @return Vector Similarity Function + */ + public VectorSimilarityFunction getSimilarityFunction() { + return similarityFunction; + } + + /** + * Get Parameters to define m and ef_construction + * @return Parameters to define m and ef_construction + */ + public Map getParameters() { + return parameters; + } +} diff --git a/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/VectorField.java b/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/VectorField.java new file mode 100644 index 0000000000000..61efd6b9a87ae --- /dev/null +++ b/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/VectorField.java @@ -0,0 +1,51 @@ +/* + * 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.plugin.correlation.core.index; + +import org.apache.lucene.document.Field; +import org.apache.lucene.index.IndexableFieldType; +import org.apache.lucene.util.BytesRef; +import org.opensearch.common.io.stream.BytesStreamOutput; + +import java.io.IOException; + +/** + * Generic Vector Field defining a correlation vector name, float array. + * + * @opensearch.internal + */ +public class VectorField extends Field { + + /** + * Parameterized ctor for VectorField + * @param name name of the field + * @param value float array value for the field + * @param type type of the field + */ + public VectorField(String name, float[] value, IndexableFieldType type) { + super(name, new BytesRef(), type); + try { + final byte[] floatToByte = floatToByteArray(value); + this.setBytesValue(floatToByte); + } catch (IOException ex) { + throw new RuntimeException(ex); + } + } + + /** + * converts float array based vector to byte array. + * @param input float array + * @return byte array + */ + protected static byte[] floatToByteArray(float[] input) throws IOException { + BytesStreamOutput objectStream = new BytesStreamOutput(); + objectStream.writeFloatArray(input); + return objectStream.bytes().toBytesRef().bytes; + } +} diff --git a/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/codec/BasePerFieldCorrelationVectorsFormat.java b/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/codec/BasePerFieldCorrelationVectorsFormat.java new file mode 100644 index 0000000000000..7763b1e42d63e --- /dev/null +++ b/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/codec/BasePerFieldCorrelationVectorsFormat.java @@ -0,0 +1,104 @@ +/* + * 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.plugin.correlation.core.index.codec; + +import org.apache.lucene.codecs.KnnVectorsFormat; +import org.apache.lucene.codecs.perfield.PerFieldKnnVectorsFormat; +import org.opensearch.index.mapper.MapperService; +import org.opensearch.plugin.correlation.core.index.mapper.CorrelationVectorFieldMapper; + +import java.util.Locale; +import java.util.Map; +import java.util.Optional; +import java.util.function.BiFunction; +import java.util.function.Supplier; + +/** + * Class to define the hyper-parameters m and ef_construction for insert and store of correlation vectors into HNSW graphs based lucene index. + * + * @opensearch.internal + */ +public abstract class BasePerFieldCorrelationVectorsFormat extends PerFieldKnnVectorsFormat { + /** + * the hyper-parameters for constructing HNSW graphs. + * https://lucene.apache.org/core/9_4_0/core/org/apache/lucene/util/hnsw/HnswGraph.html + */ + public static final String METHOD_PARAMETER_M = "m"; + /** + * the hyper-parameters for constructing HNSW graphs. + * https://lucene.apache.org/core/9_4_0/core/org/apache/lucene/util/hnsw/HnswGraph.html + */ + public static final String METHOD_PARAMETER_EF_CONSTRUCTION = "ef_construction"; + + private final Optional mapperService; + private final int defaultMaxConnections; + private final int defaultBeamWidth; + private final Supplier defaultFormatSupplier; + private final BiFunction formatSupplier; + + /** + * Parameterized ctor of BasePerFieldCorrelationVectorsFormat + * @param mapperService mapper service + * @param defaultMaxConnections default m + * @param defaultBeamWidth default ef_construction + * @param defaultFormatSupplier default format supplier + * @param formatSupplier format supplier + */ + public BasePerFieldCorrelationVectorsFormat( + Optional mapperService, + int defaultMaxConnections, + int defaultBeamWidth, + Supplier defaultFormatSupplier, + BiFunction formatSupplier + ) { + this.mapperService = mapperService; + this.defaultMaxConnections = defaultMaxConnections; + this.defaultBeamWidth = defaultBeamWidth; + this.defaultFormatSupplier = defaultFormatSupplier; + this.formatSupplier = formatSupplier; + } + + @Override + public KnnVectorsFormat getKnnVectorsFormatForField(String field) { + if (!isCorrelationVectorFieldType(field)) { + return defaultFormatSupplier.get(); + } + + var type = (CorrelationVectorFieldMapper.CorrelationVectorFieldType) mapperService.orElseThrow( + () -> new IllegalArgumentException( + String.format(Locale.getDefault(), "Cannot read field type for field [%s] because mapper service is not available", field) + ) + ).fieldType(field); + + var params = type.getCorrelationParams().getParameters(); + int maxConnections = getMaxConnections(params); + int beamWidth = getBeamWidth(params); + + return formatSupplier.apply(maxConnections, beamWidth); + } + + private boolean isCorrelationVectorFieldType(final String field) { + return mapperService.isPresent() + && mapperService.get().fieldType(field) instanceof CorrelationVectorFieldMapper.CorrelationVectorFieldType; + } + + private int getMaxConnections(final Map params) { + if (params != null && params.containsKey(METHOD_PARAMETER_M)) { + return (int) params.get(METHOD_PARAMETER_M); + } + return defaultMaxConnections; + } + + private int getBeamWidth(final Map params) { + if (params != null && params.containsKey(METHOD_PARAMETER_EF_CONSTRUCTION)) { + return (int) params.get(METHOD_PARAMETER_EF_CONSTRUCTION); + } + return defaultBeamWidth; + } +} diff --git a/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/codec/CorrelationCodecService.java b/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/codec/CorrelationCodecService.java new file mode 100644 index 0000000000000..0b70e7ed66f3d --- /dev/null +++ b/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/codec/CorrelationCodecService.java @@ -0,0 +1,38 @@ +/* + * 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.plugin.correlation.core.index.codec; + +import org.apache.lucene.codecs.Codec; +import org.opensearch.index.codec.CodecService; +import org.opensearch.index.codec.CodecServiceConfig; +import org.opensearch.index.mapper.MapperService; + +/** + * custom Correlation Codec Service + * + * @opensearch.internal + */ +public class CorrelationCodecService extends CodecService { + + private final MapperService mapperService; + + /** + * Parameterized ctor for CorrelationCodecService + * @param codecServiceConfig Generic codec service config + */ + public CorrelationCodecService(CodecServiceConfig codecServiceConfig) { + super(codecServiceConfig.getMapperService(), codecServiceConfig.getLogger()); + mapperService = codecServiceConfig.getMapperService(); + } + + @Override + public Codec codec(String name) { + return CorrelationCodecVersion.current().getCorrelationCodecSupplier().apply(super.codec(name), mapperService); + } +} diff --git a/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/codec/CorrelationCodecVersion.java b/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/codec/CorrelationCodecVersion.java new file mode 100644 index 0000000000000..5e2cb8bfbc03a --- /dev/null +++ b/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/codec/CorrelationCodecVersion.java @@ -0,0 +1,103 @@ +/* + * 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.plugin.correlation.core.index.codec; + +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.lucene95.Lucene95Codec; +import org.opensearch.index.mapper.MapperService; +import org.opensearch.plugin.correlation.core.index.codec.correlation950.CorrelationCodec; +import org.opensearch.plugin.correlation.core.index.codec.correlation950.PerFieldCorrelationVectorsFormat; + +import java.util.Optional; +import java.util.function.BiFunction; +import java.util.function.Supplier; + +/** + * CorrelationCodecVersion enum + * + * @opensearch.internal + */ +public enum CorrelationCodecVersion { + V_9_5_0( + "CorrelationCodec", + new Lucene95Codec(), + new PerFieldCorrelationVectorsFormat(Optional.empty()), + (userCodec, mapperService) -> new CorrelationCodec(userCodec, new PerFieldCorrelationVectorsFormat(Optional.of(mapperService))), + CorrelationCodec::new + ); + + private static final CorrelationCodecVersion CURRENT = V_9_5_0; + private final String codecName; + private final Codec defaultCodecDelegate; + private final PerFieldCorrelationVectorsFormat perFieldCorrelationVectorsFormat; + private final BiFunction correlationCodecSupplier; + private final Supplier defaultCorrelationCodecSupplier; + + CorrelationCodecVersion( + String codecName, + Codec defaultCodecDelegate, + PerFieldCorrelationVectorsFormat perFieldCorrelationVectorsFormat, + BiFunction correlationCodecSupplier, + Supplier defaultCorrelationCodecSupplier + ) { + this.codecName = codecName; + this.defaultCodecDelegate = defaultCodecDelegate; + this.perFieldCorrelationVectorsFormat = perFieldCorrelationVectorsFormat; + this.correlationCodecSupplier = correlationCodecSupplier; + this.defaultCorrelationCodecSupplier = defaultCorrelationCodecSupplier; + } + + /** + * get codec name + * @return codec name + */ + public String getCodecName() { + return codecName; + } + + /** + * get default codec delegate + * @return default codec delegate + */ + public Codec getDefaultCodecDelegate() { + return defaultCodecDelegate; + } + + /** + * get correlation vectors format + * @return correlation vectors format + */ + public PerFieldCorrelationVectorsFormat getPerFieldCorrelationVectorsFormat() { + return perFieldCorrelationVectorsFormat; + } + + /** + * get correlation codec supplier + * @return correlation codec supplier + */ + public BiFunction getCorrelationCodecSupplier() { + return correlationCodecSupplier; + } + + /** + * get default correlation codec supplier + * @return default correlation codec supplier + */ + public Supplier getDefaultCorrelationCodecSupplier() { + return defaultCorrelationCodecSupplier; + } + + /** + * static method to get correlation codec version + * @return correlation codec version + */ + public static final CorrelationCodecVersion current() { + return CURRENT; + } +} diff --git a/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/codec/correlation950/CorrelationCodec.java b/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/codec/correlation950/CorrelationCodec.java new file mode 100644 index 0000000000000..f91ba429fbea9 --- /dev/null +++ b/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/codec/correlation950/CorrelationCodec.java @@ -0,0 +1,46 @@ +/* + * 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.plugin.correlation.core.index.codec.correlation950; + +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.codecs.FilterCodec; +import org.apache.lucene.codecs.KnnVectorsFormat; +import org.opensearch.plugin.correlation.core.index.codec.CorrelationCodecVersion; + +/** + * Correlation Codec class + * + * @opensearch.internal + */ +public class CorrelationCodec extends FilterCodec { + private static final CorrelationCodecVersion VERSION = CorrelationCodecVersion.V_9_5_0; + private final PerFieldCorrelationVectorsFormat perFieldCorrelationVectorsFormat; + + /** + * ctor for CorrelationCodec + */ + public CorrelationCodec() { + this(VERSION.getDefaultCodecDelegate(), VERSION.getPerFieldCorrelationVectorsFormat()); + } + + /** + * Parameterized ctor for CorrelationCodec + * @param delegate codec delegate + * @param perFieldCorrelationVectorsFormat correlation vectors format + */ + public CorrelationCodec(Codec delegate, PerFieldCorrelationVectorsFormat perFieldCorrelationVectorsFormat) { + super(VERSION.getCodecName(), delegate); + this.perFieldCorrelationVectorsFormat = perFieldCorrelationVectorsFormat; + } + + @Override + public KnnVectorsFormat knnVectorsFormat() { + return perFieldCorrelationVectorsFormat; + } +} diff --git a/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/codec/correlation950/PerFieldCorrelationVectorsFormat.java b/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/codec/correlation950/PerFieldCorrelationVectorsFormat.java new file mode 100644 index 0000000000000..f6862ecc17736 --- /dev/null +++ b/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/codec/correlation950/PerFieldCorrelationVectorsFormat.java @@ -0,0 +1,35 @@ +/* + * 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.plugin.correlation.core.index.codec.correlation950; + +import org.apache.lucene.codecs.lucene95.Lucene95HnswVectorsFormat; +import org.opensearch.index.mapper.MapperService; +import org.opensearch.plugin.correlation.core.index.codec.BasePerFieldCorrelationVectorsFormat; + +import java.util.Optional; + +/** + * Class to define the hyper-parameters m and ef_construction for insert and store of correlation vectors into HNSW graphs based lucene index. + */ +public class PerFieldCorrelationVectorsFormat extends BasePerFieldCorrelationVectorsFormat { + + /** + * Parameterized ctor for PerFieldCorrelationVectorsFormat + * @param mapperService mapper service + */ + public PerFieldCorrelationVectorsFormat(final Optional mapperService) { + super( + mapperService, + Lucene95HnswVectorsFormat.DEFAULT_MAX_CONN, + Lucene95HnswVectorsFormat.DEFAULT_BEAM_WIDTH, + Lucene95HnswVectorsFormat::new, + Lucene95HnswVectorsFormat::new + ); + } +} diff --git a/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/codec/correlation950/package-info.java b/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/codec/correlation950/package-info.java new file mode 100644 index 0000000000000..b4dad34d2718e --- /dev/null +++ b/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/codec/correlation950/package-info.java @@ -0,0 +1,12 @@ +/* + * 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. + */ + +/** + * custom Lucene9.5 codec package for events-correlation-engine + */ +package org.opensearch.plugin.correlation.core.index.codec.correlation950; diff --git a/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/codec/package-info.java b/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/codec/package-info.java new file mode 100644 index 0000000000000..862b7cd253f04 --- /dev/null +++ b/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/codec/package-info.java @@ -0,0 +1,12 @@ +/* + * 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. + */ + +/** + * custom codec package for events-correlation-engine + */ +package org.opensearch.plugin.correlation.core.index.codec; diff --git a/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/mapper/CorrelationVectorFieldMapper.java b/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/mapper/CorrelationVectorFieldMapper.java new file mode 100644 index 0000000000000..a1918f3c954d0 --- /dev/null +++ b/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/mapper/CorrelationVectorFieldMapper.java @@ -0,0 +1,174 @@ +/* + * 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.plugin.correlation.core.index.mapper; + +import org.apache.lucene.document.FieldType; +import org.apache.lucene.document.KnnFloatVectorField; +import org.apache.lucene.document.StoredField; +import org.apache.lucene.index.DocValuesType; +import org.apache.lucene.index.VectorSimilarityFunction; +import org.opensearch.common.Explicit; +import org.opensearch.index.mapper.FieldMapper; +import org.opensearch.index.mapper.ParseContext; +import org.opensearch.plugin.correlation.core.index.CorrelationParamsContext; +import org.opensearch.plugin.correlation.core.index.VectorField; + +import java.io.IOException; +import java.util.Locale; +import java.util.Optional; + +import static org.apache.lucene.index.FloatVectorValues.MAX_DIMENSIONS; + +/** + * Field mapper for the correlation vector type + * + * @opensearch.internal + */ +public class CorrelationVectorFieldMapper extends VectorFieldMapper { + + private static final int LUCENE_MAX_DIMENSION = MAX_DIMENSIONS; + + private final FieldType vectorFieldType; + + /** + * Parameterized ctor for CorrelationVectorFieldMapper + * @param input Object containing name of the field, type and other details. + */ + public CorrelationVectorFieldMapper(final CreateLuceneFieldMapperInput input) { + super( + input.getName(), + input.getMappedFieldType(), + input.getMultiFields(), + input.getCopyTo(), + input.getIgnoreMalformed(), + input.isStored(), + input.isHasDocValues() + ); + + this.correlationParams = input.getCorrelationParams(); + final VectorSimilarityFunction vectorSimilarityFunction = this.correlationParams.getSimilarityFunction(); + + final int dimension = input.getMappedFieldType().getDimension(); + if (dimension > LUCENE_MAX_DIMENSION) { + throw new IllegalArgumentException( + String.format( + Locale.ROOT, + "Dimension value cannot be greater than [%s] but got [%s] for vector [%s]", + LUCENE_MAX_DIMENSION, + dimension, + input.getName() + ) + ); + } + + this.fieldType = KnnFloatVectorField.createFieldType(dimension, vectorSimilarityFunction); + + if (this.hasDocValues) { + this.vectorFieldType = buildDocValuesFieldType(); + } else { + this.vectorFieldType = null; + } + } + + private static FieldType buildDocValuesFieldType() { + FieldType field = new FieldType(); + field.setDocValuesType(DocValuesType.BINARY); + field.freeze(); + return field; + } + + @Override + protected void parseCreateField(ParseContext context, int dimension) throws IOException { + Optional arrayOptional = getFloatsFromContext(context, dimension); + + if (arrayOptional.isEmpty()) { + return; + } + final float[] array = arrayOptional.get(); + + KnnFloatVectorField point = new KnnFloatVectorField(name(), array, fieldType); + + context.doc().add(point); + if (fieldType.stored()) { + context.doc().add(new StoredField(name(), point.toString())); + } + if (hasDocValues && vectorFieldType != null) { + context.doc().add(new VectorField(name(), array, vectorFieldType)); + } + context.path().remove(); + } + + static class CreateLuceneFieldMapperInput { + String name; + + CorrelationVectorFieldType mappedFieldType; + + FieldMapper.MultiFields multiFields; + + FieldMapper.CopyTo copyTo; + + Explicit ignoreMalformed; + boolean stored; + boolean hasDocValues; + + CorrelationParamsContext correlationParams; + + public CreateLuceneFieldMapperInput( + String name, + CorrelationVectorFieldType mappedFieldType, + FieldMapper.MultiFields multiFields, + FieldMapper.CopyTo copyTo, + Explicit ignoreMalformed, + boolean stored, + boolean hasDocValues, + CorrelationParamsContext correlationParams + ) { + this.name = name; + this.mappedFieldType = mappedFieldType; + this.multiFields = multiFields; + this.copyTo = copyTo; + this.ignoreMalformed = ignoreMalformed; + this.stored = stored; + this.hasDocValues = hasDocValues; + this.correlationParams = correlationParams; + } + + public String getName() { + return name; + } + + public CorrelationVectorFieldType getMappedFieldType() { + return mappedFieldType; + } + + public FieldMapper.MultiFields getMultiFields() { + return multiFields; + } + + public FieldMapper.CopyTo getCopyTo() { + return copyTo; + } + + public Explicit getIgnoreMalformed() { + return ignoreMalformed; + } + + public boolean isStored() { + return stored; + } + + public boolean isHasDocValues() { + return hasDocValues; + } + + public CorrelationParamsContext getCorrelationParams() { + return correlationParams; + } + } +} diff --git a/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/mapper/VectorFieldMapper.java b/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/mapper/VectorFieldMapper.java new file mode 100644 index 0000000000000..5ac6d92792295 --- /dev/null +++ b/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/mapper/VectorFieldMapper.java @@ -0,0 +1,399 @@ +/* + * 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.plugin.correlation.core.index.mapper; + +import org.apache.lucene.search.FieldExistsQuery; +import org.apache.lucene.search.Query; +import org.opensearch.common.Explicit; +import org.opensearch.common.xcontent.support.XContentMapValues; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParser; +import org.opensearch.index.mapper.FieldMapper; +import org.opensearch.index.mapper.MappedFieldType; +import org.opensearch.index.mapper.Mapper; +import org.opensearch.index.mapper.MapperParsingException; +import org.opensearch.index.mapper.ParametrizedFieldMapper; +import org.opensearch.index.mapper.ParseContext; +import org.opensearch.index.mapper.TextSearchInfo; +import org.opensearch.index.mapper.ValueFetcher; +import org.opensearch.index.query.QueryShardContext; +import org.opensearch.index.query.QueryShardException; +import org.opensearch.plugin.correlation.core.index.CorrelationParamsContext; +import org.opensearch.search.lookup.SearchLookup; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Optional; + +/** + * Parameterized field mapper for Correlation Vector type + * + * @opensearch.internal + */ +public abstract class VectorFieldMapper extends ParametrizedFieldMapper { + + /** + * name of Correlation Vector type + */ + public static final String CONTENT_TYPE = "correlation_vector"; + /** + * dimension of the correlation vectors + */ + public static final String DIMENSION = "dimension"; + /** + * context e.g. parameters and vector similarity function of Correlation Vector type + */ + public static final String CORRELATION_CONTEXT = "correlation_ctx"; + + private static VectorFieldMapper toType(FieldMapper in) { + return (VectorFieldMapper) in; + } + + /** + * definition of VectorFieldMapper.Builder + */ + public static class Builder extends ParametrizedFieldMapper.Builder { + protected Boolean ignoreMalformed; + + protected final Parameter stored = Parameter.boolParam("store", false, m -> toType(m).stored, false); + protected final Parameter hasDocValues = Parameter.boolParam("doc_values", false, m -> toType(m).hasDocValues, true); + protected final Parameter dimension = new Parameter<>(DIMENSION, false, () -> -1, (n, c, o) -> { + if (o == null) { + throw new IllegalArgumentException("Dimension cannot be null"); + } + int value; + try { + value = XContentMapValues.nodeIntegerValue(o); + } catch (Exception ex) { + throw new IllegalArgumentException( + String.format(Locale.getDefault(), "Unable to parse [dimension] from provided value [%s] for vector [%s]", o, name) + ); + } + if (value <= 0) { + throw new IllegalArgumentException( + String.format(Locale.getDefault(), "Dimension value must be greater than 0 for vector: %s", name) + ); + } + return value; + }, m -> toType(m).dimension); + + protected final Parameter correlationParamsContext = new Parameter<>( + CORRELATION_CONTEXT, + false, + () -> null, + (n, c, o) -> CorrelationParamsContext.parse(o), + m -> toType(m).correlationParams + ); + + protected final Parameter> meta = Parameter.metaParam(); + + /** + * Parameterized ctor for VectorFieldMapper.Builder + * @param name name + */ + public Builder(String name) { + super(name); + } + + @Override + protected List> getParameters() { + return Arrays.asList(stored, hasDocValues, dimension, meta, correlationParamsContext); + } + + protected Explicit ignoreMalformed(BuilderContext context) { + if (ignoreMalformed != null) { + return new Explicit<>(ignoreMalformed, true); + } + if (context.indexSettings() != null) { + return new Explicit<>(IGNORE_MALFORMED_SETTING.get(context.indexSettings()), false); + } + return Defaults.IGNORE_MALFORMED; + } + + @Override + public ParametrizedFieldMapper build(BuilderContext context) { + final CorrelationParamsContext correlationParams = correlationParamsContext.getValue(); + final MultiFields multiFieldsBuilder = this.multiFieldsBuilder.build(this, context); + final CopyTo copyToBuilder = copyTo.build(); + final Explicit ignoreMalformed = ignoreMalformed(context); + final Map metaValue = meta.getValue(); + + final CorrelationVectorFieldType mappedFieldType = new CorrelationVectorFieldType( + buildFullName(context), + metaValue, + dimension.getValue(), + correlationParams + ); + + CorrelationVectorFieldMapper.CreateLuceneFieldMapperInput createLuceneFieldMapperInput = + new CorrelationVectorFieldMapper.CreateLuceneFieldMapperInput( + name, + mappedFieldType, + multiFieldsBuilder, + copyToBuilder, + ignoreMalformed, + stored.get(), + hasDocValues.get(), + correlationParams + ); + return new CorrelationVectorFieldMapper(createLuceneFieldMapperInput); + } + } + + /** + * deifintion of VectorFieldMapper.TypeParser + */ + public static class TypeParser implements Mapper.TypeParser { + + /** + * default constructor of VectorFieldMapper.TypeParser + */ + public TypeParser() {} + + @Override + public Mapper.Builder parse(String name, Map node, ParserContext context) throws MapperParsingException { + Builder builder = new VectorFieldMapper.Builder(name); + builder.parse(name, context, node); + + if (builder.dimension.getValue() == -1) { + throw new IllegalArgumentException(String.format(Locale.getDefault(), "Dimension value missing for vector: %s", name)); + } + return builder; + } + } + + /** + * deifintion of VectorFieldMapper.CorrelationVectorFieldType + */ + public static class CorrelationVectorFieldType extends MappedFieldType { + int dimension; + CorrelationParamsContext correlationParams; + + /** + * Parameterized ctor for VectorFieldMapper.CorrelationVectorFieldType + * @param name name of the field + * @param meta meta of the field + * @param dimension dimension of the field + */ + public CorrelationVectorFieldType(String name, Map meta, int dimension) { + this(name, meta, dimension, null); + } + + /** + * Parameterized ctor for VectorFieldMapper.CorrelationVectorFieldType + * @param name name of the field + * @param meta meta of the field + * @param dimension dimension of the field + * @param correlationParams correlation params for the field + */ + public CorrelationVectorFieldType( + String name, + Map meta, + int dimension, + CorrelationParamsContext correlationParams + ) { + super(name, false, false, true, TextSearchInfo.NONE, meta); + this.dimension = dimension; + this.correlationParams = correlationParams; + } + + @Override + public ValueFetcher valueFetcher(QueryShardContext context, SearchLookup searchLookup, String s) { + throw new UnsupportedOperationException("Correlation Vector do not support fields search"); + } + + @Override + public String typeName() { + return CONTENT_TYPE; + } + + @Override + public Query existsQuery(QueryShardContext context) { + return new FieldExistsQuery(name()); + } + + @Override + public Query termQuery(Object o, QueryShardContext context) { + throw new QueryShardException( + context, + String.format( + Locale.getDefault(), + "Correlation vector do not support exact searching, use Correlation queries instead: [%s]", + name() + ) + ); + } + + /** + * get dimension + * @return dimension + */ + public int getDimension() { + return dimension; + } + + /** + * get correlation params + * @return correlation params + */ + public CorrelationParamsContext getCorrelationParams() { + return correlationParams; + } + } + + protected Explicit ignoreMalformed; + protected boolean stored; + protected boolean hasDocValues; + protected Integer dimension; + protected CorrelationParamsContext correlationParams; + + /** + * Parameterized ctor for VectorFieldMapper + * @param simpleName name of field + * @param mappedFieldType field type of field + * @param multiFields multi fields + * @param copyTo copy to + * @param ignoreMalformed ignore malformed + * @param stored stored field + * @param hasDocValues has doc values + */ + public VectorFieldMapper( + String simpleName, + CorrelationVectorFieldType mappedFieldType, + FieldMapper.MultiFields multiFields, + FieldMapper.CopyTo copyTo, + Explicit ignoreMalformed, + boolean stored, + boolean hasDocValues + ) { + super(simpleName, mappedFieldType, multiFields, copyTo); + this.ignoreMalformed = ignoreMalformed; + this.stored = stored; + this.hasDocValues = hasDocValues; + this.dimension = mappedFieldType.getDimension(); + } + + @Override + protected VectorFieldMapper clone() { + return (VectorFieldMapper) super.clone(); + } + + @Override + protected String contentType() { + return CONTENT_TYPE; + } + + @Override + protected void parseCreateField(ParseContext parseContext) throws IOException { + parseCreateField(parseContext, fieldType().getDimension()); + } + + protected abstract void parseCreateField(ParseContext parseContext, int dimension) throws IOException; + + Optional getFloatsFromContext(ParseContext context, int dimension) throws IOException { + context.path().add(simpleName()); + + List vector = new ArrayList<>(); + XContentParser.Token token = context.parser().currentToken(); + float value; + if (token == XContentParser.Token.START_ARRAY) { + token = context.parser().nextToken(); + while (token != XContentParser.Token.END_ARRAY) { + value = context.parser().floatValue(); + + if (Float.isNaN(value)) { + throw new IllegalArgumentException("Correlation vector values cannot be NaN"); + } + + if (Float.isInfinite(value)) { + throw new IllegalArgumentException("Correlation vector values cannot be infinity"); + } + vector.add(value); + token = context.parser().nextToken(); + } + } else if (token == XContentParser.Token.VALUE_NUMBER) { + value = context.parser().floatValue(); + if (Float.isNaN(value)) { + throw new IllegalArgumentException("Correlation vector values cannot be NaN"); + } + + if (Float.isInfinite(value)) { + throw new IllegalArgumentException("Correlation vector values cannot be infinity"); + } + vector.add(value); + context.parser().nextToken(); + } else if (token == XContentParser.Token.VALUE_NULL) { + context.path().remove(); + return Optional.empty(); + } + + if (dimension != vector.size()) { + String errorMessage = String.format( + Locale.ROOT, + "Vector dimension mismatch. Expected: %d, Given: %d", + dimension, + vector.size() + ); + throw new IllegalArgumentException(errorMessage); + } + + float[] array = new float[vector.size()]; + int i = 0; + for (Float f : vector) { + array[i++] = f; + } + return Optional.of(array); + } + + @Override + protected boolean docValuesByDefault() { + return true; + } + + @Override + public ParametrizedFieldMapper.Builder getMergeBuilder() { + return new VectorFieldMapper.Builder(simpleName()).init(this); + } + + @Override + public boolean parsesArrayValue() { + return true; + } + + @Override + public CorrelationVectorFieldType fieldType() { + return (CorrelationVectorFieldType) super.fieldType(); + } + + @Override + protected void doXContentBody(XContentBuilder builder, boolean includeDefaults, Params params) throws IOException { + super.doXContentBody(builder, includeDefaults, params); + if (includeDefaults || ignoreMalformed.explicit()) { + builder.field(Names.IGNORE_MALFORMED, ignoreMalformed.value()); + } + } + + /** + * Class for constants used in parent class VectorFieldMapper + */ + public static class Names { + public static final String IGNORE_MALFORMED = "ignore_malformed"; + } + + /** + * Class for constants used in parent class VectorFieldMapper + */ + public static class Defaults { + public static final Explicit IGNORE_MALFORMED = new Explicit<>(false, false); + } +} diff --git a/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/mapper/package-info.java b/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/mapper/package-info.java new file mode 100644 index 0000000000000..4fdc622c3d886 --- /dev/null +++ b/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/mapper/package-info.java @@ -0,0 +1,12 @@ +/* + * 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. + */ + +/** + * correlation field mapper package + */ +package org.opensearch.plugin.correlation.core.index.mapper; diff --git a/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/package-info.java b/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/package-info.java new file mode 100644 index 0000000000000..cfc0ffdfa81f1 --- /dev/null +++ b/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/package-info.java @@ -0,0 +1,12 @@ +/* + * 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 to wrap Lucene KnnFloatVectorField and KnnFloatVectorQuery for Opensearch events-correlation-engine + */ +package org.opensearch.plugin.correlation.core.index; diff --git a/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/query/CorrelationQueryBuilder.java b/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/query/CorrelationQueryBuilder.java new file mode 100644 index 0000000000000..a71a7874f178e --- /dev/null +++ b/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/query/CorrelationQueryBuilder.java @@ -0,0 +1,330 @@ +/* + * 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.plugin.correlation.core.index.query; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.lucene.search.Query; +import org.opensearch.common.ParsingException; +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.XContentBuilder; +import org.opensearch.core.xcontent.XContentParser; +import org.opensearch.index.mapper.MappedFieldType; +import org.opensearch.index.mapper.NumberFieldMapper; +import org.opensearch.index.query.AbstractQueryBuilder; +import org.opensearch.index.query.QueryBuilder; +import org.opensearch.index.query.QueryShardContext; +import org.opensearch.plugin.correlation.core.index.mapper.VectorFieldMapper; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Locale; +import java.util.Objects; + +/** + * Constructs a query to get correlated events or documents for a particular event or document. + * + * @opensearch.internal + */ +public class CorrelationQueryBuilder extends AbstractQueryBuilder { + + private static final Logger log = LogManager.getLogger(CorrelationQueryBuilder.class); + protected static final ParseField VECTOR_FIELD = new ParseField("vector"); + protected static final ParseField K_FIELD = new ParseField("k"); + protected static final ParseField FILTER_FIELD = new ParseField("filter"); + /** + * max number of neighbors that can be retrieved. + */ + public static int K_MAX = 10000; + + /** + * name of the query + */ + public static final ParseField NAME_FIELD = new ParseField("correlation"); + + private String fieldName; + private float[] vector; + private int k = 0; + private double boost; + private QueryBuilder filter; + + private CorrelationQueryBuilder() {} + + /** + * parameterized ctor for CorrelationQueryBuilder + * @param fieldName field name for query + * @param vector query vector + * @param k number of nearby neighbors + */ + public CorrelationQueryBuilder(String fieldName, float[] vector, int k) { + this(fieldName, vector, k, null); + } + + /** + * parameterized ctor for CorrelationQueryBuilder + * @param fieldName field name for query + * @param vector query vector + * @param k number of nearby neighbors + * @param filter optional filter query + */ + public CorrelationQueryBuilder(String fieldName, float[] vector, int k, QueryBuilder filter) { + if (Strings.isNullOrEmpty(fieldName)) { + throw new IllegalArgumentException( + String.format(Locale.getDefault(), "[%s] requires fieldName", NAME_FIELD.getPreferredName()) + ); + } + if (vector == null) { + throw new IllegalArgumentException( + String.format(Locale.getDefault(), "[%s] requires query vector", NAME_FIELD.getPreferredName()) + ); + } + if (vector.length == 0) { + throw new IllegalArgumentException( + String.format(Locale.getDefault(), "[%s] query vector is empty", NAME_FIELD.getPreferredName()) + ); + } + if (k <= 0) { + throw new IllegalArgumentException(String.format(Locale.getDefault(), "[%s] requires k > 0", NAME_FIELD.getPreferredName())); + } + if (k > K_MAX) { + throw new IllegalArgumentException(String.format(Locale.getDefault(), "[%s] requires k <= ", K_MAX)); + } + + this.fieldName = fieldName; + this.vector = vector; + this.k = k; + this.filter = filter; + } + + /** + * parameterized ctor for CorrelationQueryBuilder + * @param sin StreamInput + * @throws IOException IOException + */ + public CorrelationQueryBuilder(StreamInput sin) throws IOException { + super(sin); + this.fieldName = sin.readString(); + this.vector = sin.readFloatArray(); + this.k = sin.readInt(); + this.filter = sin.readOptionalNamedWriteable(QueryBuilder.class); + } + + private static float[] objectsToFloats(List objs) { + float[] vector = new float[objs.size()]; + for (int i = 0; i < objs.size(); ++i) { + vector[i] = ((Number) objs.get(i)).floatValue(); + } + return vector; + } + + /** + * parse into CorrelationQueryBuilder + * @param xcp XContentParser + * @return CorrelationQueryBuilder + */ + public static CorrelationQueryBuilder parse(XContentParser xcp) throws IOException { + String fieldName = null; + List vector = null; + float boost = AbstractQueryBuilder.DEFAULT_BOOST; + + int k = 0; + QueryBuilder filter = null; + String queryName = null; + String currentFieldName = null; + XContentParser.Token token; + while ((token = xcp.nextToken()) != XContentParser.Token.END_OBJECT) { + if (token == XContentParser.Token.FIELD_NAME) { + currentFieldName = xcp.currentName(); + } else if (token == XContentParser.Token.START_OBJECT) { + throwParsingExceptionOnMultipleFields(NAME_FIELD.getPreferredName(), xcp.getTokenLocation(), fieldName, currentFieldName); + fieldName = currentFieldName; + while ((token = xcp.nextToken()) != XContentParser.Token.END_OBJECT) { + if (token == XContentParser.Token.FIELD_NAME) { + currentFieldName = xcp.currentName(); + } else if (token.isValue() || token == XContentParser.Token.START_ARRAY) { + if (VECTOR_FIELD.match(currentFieldName, xcp.getDeprecationHandler())) { + vector = xcp.list(); + } else if (AbstractQueryBuilder.BOOST_FIELD.match(currentFieldName, xcp.getDeprecationHandler())) { + boost = xcp.floatValue(); + } else if (K_FIELD.match(currentFieldName, xcp.getDeprecationHandler())) { + k = (Integer) NumberFieldMapper.NumberType.INTEGER.parse(xcp.objectBytes(), false); + } else if (AbstractQueryBuilder.NAME_FIELD.match(currentFieldName, xcp.getDeprecationHandler())) { + queryName = xcp.text(); + } else { + throw new ParsingException( + xcp.getTokenLocation(), + "[" + NAME_FIELD.getPreferredName() + "] query does not support [" + currentFieldName + "]" + ); + } + } else if (token == XContentParser.Token.START_OBJECT) { + String tokenName = xcp.currentName(); + if (FILTER_FIELD.getPreferredName().equals(tokenName)) { + filter = parseInnerQueryBuilder(xcp); + } else { + throw new ParsingException( + xcp.getTokenLocation(), + "[" + NAME_FIELD.getPreferredName() + "] unknown token [" + token + "]" + ); + } + } else { + throw new ParsingException( + xcp.getTokenLocation(), + "[" + NAME_FIELD.getPreferredName() + "] unknown token [" + token + "] after [" + currentFieldName + "]" + ); + } + } + } else { + throwParsingExceptionOnMultipleFields(NAME_FIELD.getPreferredName(), xcp.getTokenLocation(), fieldName, xcp.currentName()); + fieldName = xcp.currentName(); + vector = xcp.list(); + } + } + + assert vector != null; + CorrelationQueryBuilder correlationQueryBuilder = new CorrelationQueryBuilder(fieldName, objectsToFloats(vector), k, filter); + correlationQueryBuilder.queryName(queryName); + correlationQueryBuilder.boost(boost); + return correlationQueryBuilder; + } + + public void setFieldName(String fieldName) { + this.fieldName = fieldName; + } + + /** + * get field name + * @return field name + */ + public String fieldName() { + return fieldName; + } + + public void setVector(float[] vector) { + this.vector = vector; + } + + /** + * get query vector + * @return query vector + */ + public Object vector() { + return vector; + } + + public void setK(int k) { + this.k = k; + } + + /** + * get number of nearby neighbors + * @return number of nearby neighbors + */ + public int getK() { + return k; + } + + public void setBoost(double boost) { + this.boost = boost; + } + + /** + * get boost + * @return boost + */ + public double getBoost() { + return boost; + } + + public void setFilter(QueryBuilder filter) { + this.filter = filter; + } + + /** + * get optional filter + * @return optional filter + */ + public QueryBuilder getFilter() { + return filter; + } + + @Override + protected void doWriteTo(StreamOutput out) throws IOException { + out.writeString(fieldName); + out.writeFloatArray(vector); + out.writeInt(k); + out.writeOptionalNamedWriteable(filter); + } + + @Override + public void doXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(fieldName); + + builder.field(VECTOR_FIELD.getPreferredName(), vector); + builder.field(K_FIELD.getPreferredName(), k); + if (filter != null) { + builder.field(FILTER_FIELD.getPreferredName(), filter); + } + printBoostAndQueryName(builder); + builder.endObject(); + } + + @Override + protected Query doToQuery(QueryShardContext context) throws IOException { + MappedFieldType mappedFieldType = context.fieldMapper(fieldName); + + if (!(mappedFieldType instanceof VectorFieldMapper.CorrelationVectorFieldType)) { + throw new IllegalArgumentException(String.format(Locale.getDefault(), "Field '%s' is not knn_vector type.", this.fieldName)); + } + + VectorFieldMapper.CorrelationVectorFieldType correlationVectorFieldType = + (VectorFieldMapper.CorrelationVectorFieldType) mappedFieldType; + int fieldDimension = correlationVectorFieldType.getDimension(); + + if (fieldDimension != vector.length) { + throw new IllegalArgumentException( + String.format( + Locale.getDefault(), + "Query vector has invalid dimension: %d. Dimension should be: %d", + vector.length, + fieldDimension + ) + ); + } + + String indexName = context.index().getName(); + CorrelationQueryFactory.CreateQueryRequest createQueryRequest = new CorrelationQueryFactory.CreateQueryRequest( + indexName, + this.fieldName, + this.vector, + this.k, + this.filter, + context + ); + return CorrelationQueryFactory.create(createQueryRequest); + } + + @Override + protected boolean doEquals(CorrelationQueryBuilder other) { + return Objects.equals(fieldName, other.fieldName) && Arrays.equals(vector, other.vector) && Objects.equals(k, other.k); + } + + @Override + protected int doHashCode() { + return Objects.hash(fieldName, vector, k); + } + + @Override + public String getWriteableName() { + return NAME_FIELD.getPreferredName(); + } +} diff --git a/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/query/CorrelationQueryFactory.java b/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/query/CorrelationQueryFactory.java new file mode 100644 index 0000000000000..d5db299bfa3a5 --- /dev/null +++ b/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/query/CorrelationQueryFactory.java @@ -0,0 +1,142 @@ +/* + * 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.plugin.correlation.core.index.query; + +import org.apache.lucene.search.KnnFloatVectorQuery; +import org.apache.lucene.search.Query; +import org.opensearch.index.query.QueryBuilder; +import org.opensearch.index.query.QueryShardContext; + +import java.io.IOException; +import java.util.Optional; + +/** + * CorrelationQueryFactory util class is used to construct a Lucene KnnFloatVectorQuery. + * + * @opensearch.internal + */ +public class CorrelationQueryFactory { + + /** + * static method which takes input params to construct a Lucene KnnFloatVectorQuery. + * @param createQueryRequest object parameter containing inputs for constructing Lucene KnnFloatVectorQuery. + * @return generic Lucene Query object + */ + public static Query create(CreateQueryRequest createQueryRequest) { + final String indexName = createQueryRequest.getIndexName(); + final String fieldName = createQueryRequest.getFieldName(); + final int k = createQueryRequest.getK(); + final float[] vector = createQueryRequest.getVector(); + + if (createQueryRequest.getFilter().isPresent()) { + final QueryShardContext context = createQueryRequest.getContext() + .orElseThrow(() -> new RuntimeException("Shard context cannot be null")); + + try { + final Query filterQuery = createQueryRequest.getFilter().get().toQuery(context); + return new KnnFloatVectorQuery(fieldName, vector, k, filterQuery); + } catch (IOException ex) { + throw new RuntimeException("Cannot create knn query with filter", ex); + } + } + return new KnnFloatVectorQuery(fieldName, vector, k); + } + + /** + * class containing params to construct a Lucene KnnFloatVectorQuery. + * + * @opensearch.internal + */ + public static class CreateQueryRequest { + private String indexName; + + private String fieldName; + + private float[] vector; + + private int k; + + private QueryBuilder filter; + + private QueryShardContext context; + + /** + * Parameterized ctor for CreateQueryRequest + * @param indexName index name + * @param fieldName field name + * @param vector query vector + * @param k number of nearby neighbors + * @param filter additional filter query + * @param context QueryShardContext + */ + public CreateQueryRequest( + String indexName, + String fieldName, + float[] vector, + int k, + QueryBuilder filter, + QueryShardContext context + ) { + this.indexName = indexName; + this.fieldName = fieldName; + this.vector = vector; + this.k = k; + this.filter = filter; + this.context = context; + } + + /** + * get index name + * @return get index name + */ + public String getIndexName() { + return indexName; + } + + /** + * get field name + * @return get field name + */ + public String getFieldName() { + return fieldName; + } + + /** + * get vector + * @return get vector + */ + public float[] getVector() { + return vector; + } + + /** + * get number of nearby neighbors + * @return number of nearby neighbors + */ + public int getK() { + return k; + } + + /** + * get optional filter query + * @return get optional filter query + */ + public Optional getFilter() { + return Optional.ofNullable(filter); + } + + /** + * get optional query shard context + * @return get optional query shard context + */ + public Optional getContext() { + return Optional.ofNullable(context); + } + } +} diff --git a/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/query/package-info.java b/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/query/package-info.java new file mode 100644 index 0000000000000..2cf5db786a60f --- /dev/null +++ b/plugins/events-correlation-engine/src/main/java/org/opensearch/plugin/correlation/core/index/query/package-info.java @@ -0,0 +1,12 @@ +/* + * 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. + */ + +/** + * correlation query builder package + */ +package org.opensearch.plugin.correlation.core.index.query; diff --git a/plugins/events-correlation-engine/src/main/resources/META-INF/services/org.apache.lucene.codecs.Codec b/plugins/events-correlation-engine/src/main/resources/META-INF/services/org.apache.lucene.codecs.Codec new file mode 100644 index 0000000000000..598a3b6af73c2 --- /dev/null +++ b/plugins/events-correlation-engine/src/main/resources/META-INF/services/org.apache.lucene.codecs.Codec @@ -0,0 +1 @@ +org.opensearch.plugin.correlation.core.index.codec.correlation950.CorrelationCodec diff --git a/plugins/events-correlation-engine/src/test/java/org/opensearch/plugin/correlation/core/index/CorrelationParamsContextTests.java b/plugins/events-correlation-engine/src/test/java/org/opensearch/plugin/correlation/core/index/CorrelationParamsContextTests.java new file mode 100644 index 0000000000000..7b9d4e31d7e3a --- /dev/null +++ b/plugins/events-correlation-engine/src/test/java/org/opensearch/plugin/correlation/core/index/CorrelationParamsContextTests.java @@ -0,0 +1,170 @@ +/* + * 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.plugin.correlation.core.index; + +import org.apache.lucene.index.VectorSimilarityFunction; +import org.junit.Assert; +import org.opensearch.common.bytes.BytesReference; +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.common.xcontent.XContentFactory; +import org.opensearch.common.xcontent.XContentHelper; +import org.opensearch.core.xcontent.ToXContent; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.index.mapper.MapperParsingException; +import org.opensearch.test.OpenSearchTestCase; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import static org.opensearch.plugin.correlation.core.index.CorrelationParamsContext.PARAMETERS; +import static org.opensearch.plugin.correlation.core.index.CorrelationParamsContext.VECTOR_SIMILARITY_FUNCTION; + +/** + * Unit tests for CorrelationsParamsContext + */ +public class CorrelationParamsContextTests extends OpenSearchTestCase { + + /** + * Test reading from and writing to streams + */ + public void testStreams() throws IOException { + int efConstruction = 321; + int m = 12; + + Map parameters = new HashMap<>(); + parameters.put("m", m); + parameters.put("ef_construction", efConstruction); + + CorrelationParamsContext context = new CorrelationParamsContext(VectorSimilarityFunction.EUCLIDEAN, parameters); + + BytesStreamOutput streamOutput = new BytesStreamOutput(); + context.writeTo(streamOutput); + + CorrelationParamsContext copy = new CorrelationParamsContext(streamOutput.bytes().streamInput()); + Assert.assertEquals(context.getSimilarityFunction(), copy.getSimilarityFunction()); + Assert.assertEquals(context.getParameters(), copy.getParameters()); + } + + /** + * test get vector similarity function + */ + public void testVectorSimilarityFunction() { + int efConstruction = 321; + int m = 12; + + Map parameters = new HashMap<>(); + parameters.put("m", m); + parameters.put("ef_construction", efConstruction); + + CorrelationParamsContext context = new CorrelationParamsContext(VectorSimilarityFunction.EUCLIDEAN, parameters); + Assert.assertEquals(VectorSimilarityFunction.EUCLIDEAN, context.getSimilarityFunction()); + } + + /** + * test get parameters + */ + public void testParameters() { + int efConstruction = 321; + int m = 12; + + Map parameters = new HashMap<>(); + parameters.put("m", m); + parameters.put("ef_construction", efConstruction); + + CorrelationParamsContext context = new CorrelationParamsContext(VectorSimilarityFunction.EUCLIDEAN, parameters); + Assert.assertEquals(parameters, context.getParameters()); + } + + /** + * test parse method with invalid input + * @throws IOException IOException + */ + public void testParse_Invalid() throws IOException { + // Invalid input type + Integer invalidIn = 12; + expectThrows(MapperParsingException.class, () -> CorrelationParamsContext.parse(invalidIn)); + + // Invalid vector similarity function + XContentBuilder xContentBuilder = XContentFactory.jsonBuilder() + .startObject() + .field(CorrelationParamsContext.VECTOR_SIMILARITY_FUNCTION, 0) + .endObject(); + + final Map in2 = xContentBuilderToMap(xContentBuilder); + expectThrows(MapperParsingException.class, () -> CorrelationParamsContext.parse(in2)); + + // Invalid parameters + xContentBuilder = XContentFactory.jsonBuilder().startObject().field(PARAMETERS, 0).endObject(); + + final Map in4 = xContentBuilderToMap(xContentBuilder); + expectThrows(MapperParsingException.class, () -> CorrelationParamsContext.parse(in4)); + } + + /** + * test parse with null parameters + * @throws IOException IOException + */ + public void testParse_NullParameters() throws IOException { + XContentBuilder xContentBuilder = XContentFactory.jsonBuilder() + .startObject() + .field(VECTOR_SIMILARITY_FUNCTION, VectorSimilarityFunction.EUCLIDEAN) + .field(PARAMETERS, (String) null) + .endObject(); + Map in = xContentBuilderToMap(xContentBuilder); + Assert.assertThrows(MapperParsingException.class, () -> { CorrelationParamsContext.parse(in); }); + } + + /** + * test parse method + * @throws IOException IOException + */ + public void testParse_Valid() throws IOException { + XContentBuilder xContentBuilder = XContentFactory.jsonBuilder() + .startObject() + .field(VECTOR_SIMILARITY_FUNCTION, VectorSimilarityFunction.EUCLIDEAN) + .startObject(PARAMETERS) + .field("m", 2) + .field("ef_construction", 128) + .endObject() + .endObject(); + + Map in = xContentBuilderToMap(xContentBuilder); + CorrelationParamsContext context = CorrelationParamsContext.parse(in); + Assert.assertEquals(VectorSimilarityFunction.EUCLIDEAN, context.getSimilarityFunction()); + Assert.assertEquals(Map.of("m", 2, "ef_construction", 128), context.getParameters()); + } + + /** + * test toXContent method + * @throws IOException IOException + */ + public void testToXContent() throws IOException { + XContentBuilder xContentBuilder = XContentFactory.jsonBuilder() + .startObject() + .field(VECTOR_SIMILARITY_FUNCTION, VectorSimilarityFunction.EUCLIDEAN) + .startObject(PARAMETERS) + .field("m", 2) + .field("ef_construction", 128) + .endObject() + .endObject(); + + Map in = xContentBuilderToMap(xContentBuilder); + CorrelationParamsContext context = CorrelationParamsContext.parse(in); + XContentBuilder builder = XContentFactory.jsonBuilder(); + builder = context.toXContent(builder, ToXContent.EMPTY_PARAMS); + + Map out = xContentBuilderToMap(builder); + Assert.assertEquals(VectorSimilarityFunction.EUCLIDEAN.name(), out.get(VECTOR_SIMILARITY_FUNCTION)); + } + + private Map xContentBuilderToMap(XContentBuilder xContentBuilder) { + return XContentHelper.convertToMap(BytesReference.bytes(xContentBuilder), true, xContentBuilder.contentType()).v2(); + } +} diff --git a/plugins/events-correlation-engine/src/test/java/org/opensearch/plugin/correlation/core/index/VectorFieldTests.java b/plugins/events-correlation-engine/src/test/java/org/opensearch/plugin/correlation/core/index/VectorFieldTests.java new file mode 100644 index 0000000000000..00cec228de720 --- /dev/null +++ b/plugins/events-correlation-engine/src/test/java/org/opensearch/plugin/correlation/core/index/VectorFieldTests.java @@ -0,0 +1,83 @@ +/* + * 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.plugin.correlation.core.index; + +import org.apache.lucene.document.FieldType; +import org.junit.Assert; +import org.opensearch.ExceptionsHelper; +import org.opensearch.OpenSearchException; +import org.opensearch.common.Randomness; +import org.opensearch.common.io.stream.BytesStreamInput; +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.test.OpenSearchTestCase; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Random; + +/** + * Unit tests for VectorField + */ +public class VectorFieldTests extends OpenSearchTestCase { + + private final Random random = Randomness.get(); + + /** + * test VectorField ctor + */ + public void testVectorField_ctor() { + VectorField field = new VectorField("test-field", new float[] { 1.0f, 1.0f }, new FieldType()); + Assert.assertEquals("test-field", field.name()); + } + + /** + * test float vector to array serializer + * @throws IOException IOException + */ + public void testVectorAsArraySerializer() throws IOException { + final float[] vector = getArrayOfRandomFloats(20); + + final BytesStreamOutput objectStream = new BytesStreamOutput(); + objectStream.writeFloatArray(vector); + final byte[] serializedVector = objectStream.bytes().toBytesRef().bytes; + + final byte[] actualSerializedVector = VectorField.floatToByteArray(vector); + + Assert.assertNotNull(actualSerializedVector); + Assert.assertArrayEquals(serializedVector, actualSerializedVector); + + final float[] actualDeserializedVector = byteToFloatArray(actualSerializedVector); + Assert.assertNotNull(actualDeserializedVector); + Assert.assertArrayEquals(vector, actualDeserializedVector, 0.1f); + } + + /** + * test byte array to float vector failures + */ + public void testByteToFloatArrayFailures() { + final byte[] serializedVector = "test-dummy".getBytes(StandardCharsets.UTF_8); + expectThrows(OpenSearchException.class, () -> { byteToFloatArray(serializedVector); }); + } + + private float[] getArrayOfRandomFloats(int length) { + float[] vector = new float[length]; + for (int i = 0; i < 20; ++i) { + vector[i] = random.nextFloat(); + } + return vector; + } + + private static float[] byteToFloatArray(byte[] byteStream) { + try (BytesStreamInput objectStream = new BytesStreamInput(byteStream)) { + return objectStream.readFloatArray(); + } catch (IOException ex) { + throw ExceptionsHelper.convertToOpenSearchException(ex); + } + } +} diff --git a/plugins/events-correlation-engine/src/test/java/org/opensearch/plugin/correlation/core/index/codec/correlation950/CorrelationCodecTests.java b/plugins/events-correlation-engine/src/test/java/org/opensearch/plugin/correlation/core/index/codec/correlation950/CorrelationCodecTests.java new file mode 100644 index 0000000000000..ac859773f6350 --- /dev/null +++ b/plugins/events-correlation-engine/src/test/java/org/opensearch/plugin/correlation/core/index/codec/correlation950/CorrelationCodecTests.java @@ -0,0 +1,120 @@ +/* + * 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.plugin.correlation.core.index.codec.correlation950; + +import org.apache.lucene.codecs.Codec; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.FieldType; +import org.apache.lucene.document.KnnFloatVectorField; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.SerialMergeScheduler; +import org.apache.lucene.index.VectorSimilarityFunction; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.Query; +import org.apache.lucene.store.Directory; +import org.apache.lucene.tests.index.RandomIndexWriter; +import org.opensearch.index.mapper.MapperService; +import org.opensearch.plugin.correlation.core.index.CorrelationParamsContext; +import org.opensearch.plugin.correlation.core.index.mapper.VectorFieldMapper; +import org.opensearch.plugin.correlation.core.index.query.CorrelationQueryFactory; +import org.opensearch.test.OpenSearchTestCase; + +import java.util.Map; +import java.util.Optional; +import java.util.function.Function; + +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.spy; +import static org.opensearch.plugin.correlation.core.index.codec.BasePerFieldCorrelationVectorsFormat.METHOD_PARAMETER_EF_CONSTRUCTION; +import static org.opensearch.plugin.correlation.core.index.codec.BasePerFieldCorrelationVectorsFormat.METHOD_PARAMETER_M; +import static org.opensearch.plugin.correlation.core.index.codec.CorrelationCodecVersion.V_9_5_0; + +/** + * Unit tests for custom correlation codec + */ +public class CorrelationCodecTests extends OpenSearchTestCase { + + private static final String FIELD_NAME_ONE = "test_vector_one"; + private static final String FIELD_NAME_TWO = "test_vector_two"; + + /** + * test correlation vector index + * @throws Exception Exception + */ + public void testCorrelationVectorIndex() throws Exception { + Function perFieldCorrelationVectorsProvider = + mapperService -> new PerFieldCorrelationVectorsFormat(Optional.of(mapperService)); + Function correlationCodecProvider = (correlationVectorsFormat -> new CorrelationCodec( + V_9_5_0.getDefaultCodecDelegate(), + correlationVectorsFormat + )); + testCorrelationVectorIndex(correlationCodecProvider, perFieldCorrelationVectorsProvider); + } + + private void testCorrelationVectorIndex( + final Function codecProvider, + final Function perFieldCorrelationVectorsProvider + ) throws Exception { + final MapperService mapperService = mock(MapperService.class); + final CorrelationParamsContext correlationParamsContext = new CorrelationParamsContext( + VectorSimilarityFunction.EUCLIDEAN, + Map.of(METHOD_PARAMETER_M, 16, METHOD_PARAMETER_EF_CONSTRUCTION, 256) + ); + + final VectorFieldMapper.CorrelationVectorFieldType mappedFieldType1 = new VectorFieldMapper.CorrelationVectorFieldType( + FIELD_NAME_ONE, + Map.of(), + 3, + correlationParamsContext + ); + final VectorFieldMapper.CorrelationVectorFieldType mappedFieldType2 = new VectorFieldMapper.CorrelationVectorFieldType( + FIELD_NAME_TWO, + Map.of(), + 2, + correlationParamsContext + ); + when(mapperService.fieldType(eq(FIELD_NAME_ONE))).thenReturn(mappedFieldType1); + when(mapperService.fieldType(eq(FIELD_NAME_TWO))).thenReturn(mappedFieldType2); + + var perFieldCorrelationVectorsFormatSpy = spy(perFieldCorrelationVectorsProvider.apply(mapperService)); + final Codec codec = codecProvider.apply(perFieldCorrelationVectorsFormatSpy); + + Directory dir = newFSDirectory(createTempDir()); + IndexWriterConfig iwc = newIndexWriterConfig(); + iwc.setMergeScheduler(new SerialMergeScheduler()); + iwc.setCodec(codec); + + final FieldType luceneFieldType = KnnFloatVectorField.createFieldType(3, VectorSimilarityFunction.EUCLIDEAN); + float[] array = { 1.0f, 3.0f, 4.0f }; + KnnFloatVectorField vectorField = new KnnFloatVectorField(FIELD_NAME_ONE, array, luceneFieldType); + RandomIndexWriter writer = new RandomIndexWriter(random(), dir, iwc); + Document doc = new Document(); + doc.add(vectorField); + writer.addDocument(doc); + writer.commit(); + IndexReader reader = writer.getReader(); + writer.close(); + + verify(perFieldCorrelationVectorsFormatSpy).getKnnVectorsFormatForField(eq(FIELD_NAME_ONE)); + + IndexSearcher searcher = new IndexSearcher(reader); + Query query = CorrelationQueryFactory.create( + new CorrelationQueryFactory.CreateQueryRequest("dummy", FIELD_NAME_ONE, new float[] { 1.0f, 0.0f, 0.0f }, 1, null, null) + ); + + assertEquals(1, searcher.count(query)); + + reader.close(); + dir.close(); + } +} diff --git a/plugins/events-correlation-engine/src/test/java/org/opensearch/plugin/correlation/core/index/mapper/CorrelationVectorFieldMapperTests.java b/plugins/events-correlation-engine/src/test/java/org/opensearch/plugin/correlation/core/index/mapper/CorrelationVectorFieldMapperTests.java new file mode 100644 index 0000000000000..82c2cc1a07aaf --- /dev/null +++ b/plugins/events-correlation-engine/src/test/java/org/opensearch/plugin/correlation/core/index/mapper/CorrelationVectorFieldMapperTests.java @@ -0,0 +1,309 @@ +/* + * 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.plugin.correlation.core.index.mapper; + +import org.apache.lucene.document.KnnFloatVectorField; +import org.apache.lucene.index.IndexableField; +import org.apache.lucene.index.VectorSimilarityFunction; +import org.apache.lucene.search.FieldExistsQuery; +import org.junit.Assert; +import org.mockito.Mockito; +import org.opensearch.Version; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.Explicit; +import org.opensearch.common.bytes.BytesReference; +import org.opensearch.common.settings.IndexScopedSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.xcontent.XContentFactory; +import org.opensearch.common.xcontent.XContentHelper; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.index.IndexSettings; +import org.opensearch.index.mapper.ContentPath; +import org.opensearch.index.mapper.FieldMapper; +import org.opensearch.index.mapper.Mapper; +import org.opensearch.index.mapper.MapperParsingException; +import org.opensearch.index.mapper.MapperService; +import org.opensearch.index.mapper.ParseContext; +import org.opensearch.index.query.QueryShardContext; +import org.opensearch.index.query.QueryShardException; +import org.opensearch.plugin.correlation.core.index.CorrelationParamsContext; +import org.opensearch.search.lookup.SearchLookup; +import org.opensearch.test.OpenSearchTestCase; + +import java.io.IOException; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +/** + * Unit tests for correlation vector field mapper + */ +public class CorrelationVectorFieldMapperTests extends OpenSearchTestCase { + + private static final String CORRELATION_VECTOR_TYPE = "correlation_vector"; + private static final String DIMENSION_FIELD_NAME = "dimension"; + private static final String TYPE_FIELD_NAME = "type"; + + /** + * test builder construction from parse of correlation params context + * @throws IOException IOException + */ + public void testBuilder_parse_fromCorrelationParamsContext() throws IOException { + String fieldName = "test-field-name"; + String indexName = "test-index-name"; + Settings settings = Settings.builder().put(settings(Version.CURRENT).build()).build(); + + VectorFieldMapper.TypeParser typeParser = new VectorFieldMapper.TypeParser(); + + int efConstruction = 321; + int m = 12; + int dimension = 10; + XContentBuilder xContentBuilder = XContentFactory.jsonBuilder() + .startObject() + .field(TYPE_FIELD_NAME, CORRELATION_VECTOR_TYPE) + .field(DIMENSION_FIELD_NAME, dimension) + .startObject("correlation_ctx") + .field("similarityFunction", VectorSimilarityFunction.EUCLIDEAN.name()) + .startObject("parameters") + .field("m", m) + .field("ef_construction", efConstruction) + .endObject() + .endObject() + .endObject(); + + VectorFieldMapper.Builder builder = (VectorFieldMapper.Builder) typeParser.parse( + fieldName, + XContentHelper.convertToMap(BytesReference.bytes(xContentBuilder), true, xContentBuilder.contentType()).v2(), + buildParserContext(indexName, settings) + ); + Mapper.BuilderContext builderContext = new Mapper.BuilderContext(settings, new ContentPath()); + builder.build(builderContext); + + Assert.assertEquals(VectorSimilarityFunction.EUCLIDEAN, builder.correlationParamsContext.getValue().getSimilarityFunction()); + Assert.assertEquals(321, builder.correlationParamsContext.getValue().getParameters().get("ef_construction")); + + XContentBuilder xContentBuilderEmptyParams = XContentFactory.jsonBuilder() + .startObject() + .field(TYPE_FIELD_NAME, CORRELATION_VECTOR_TYPE) + .field(DIMENSION_FIELD_NAME, dimension) + .startObject("correlation_ctx") + .field("similarityFunction", VectorSimilarityFunction.EUCLIDEAN.name()) + .endObject() + .endObject(); + + VectorFieldMapper.Builder builderEmptyParams = (VectorFieldMapper.Builder) typeParser.parse( + fieldName, + XContentHelper.convertToMap(BytesReference.bytes(xContentBuilderEmptyParams), true, xContentBuilderEmptyParams.contentType()) + .v2(), + buildParserContext(indexName, settings) + ); + + Assert.assertEquals( + VectorSimilarityFunction.EUCLIDEAN, + builderEmptyParams.correlationParamsContext.getValue().getSimilarityFunction() + ); + Assert.assertTrue(builderEmptyParams.correlationParamsContext.getValue().getParameters().isEmpty()); + } + + /** + * test type parser construction throw error for invalid dimension of correlation vectors + * @throws IOException IOException + */ + public void testTypeParser_parse_fromCorrelationParamsContext_InvalidDimension() throws IOException { + String fieldName = "test-field-name"; + String indexName = "test-index-name"; + Settings settings = Settings.builder().put(settings(Version.CURRENT).build()).build(); + + VectorFieldMapper.TypeParser typeParser = new VectorFieldMapper.TypeParser(); + + int efConstruction = 321; + int m = 12; + XContentBuilder xContentBuilder = XContentFactory.jsonBuilder() + .startObject() + .field(TYPE_FIELD_NAME, CORRELATION_VECTOR_TYPE) + .field(DIMENSION_FIELD_NAME, 2000) + .startObject("correlation_ctx") + .field("similarityFunction", VectorSimilarityFunction.EUCLIDEAN.name()) + .startObject("parameters") + .field("m", m) + .field("ef_construction", efConstruction) + .endObject() + .endObject() + .endObject(); + + VectorFieldMapper.Builder builder = (VectorFieldMapper.Builder) typeParser.parse( + fieldName, + XContentHelper.convertToMap(BytesReference.bytes(xContentBuilder), true, xContentBuilder.contentType()).v2(), + buildParserContext(indexName, settings) + ); + + expectThrows(IllegalArgumentException.class, () -> builder.build(new Mapper.BuilderContext(settings, new ContentPath()))); + } + + /** + * test type parser construction error for invalid vector similarity function + * @throws IOException IOException + */ + public void testTypeParser_parse_fromCorrelationParamsContext_InvalidVectorSimilarityFunction() throws IOException { + String fieldName = "test-field-name"; + String indexName = "test-index-name"; + Settings settings = Settings.builder().put(settings(Version.CURRENT).build()).build(); + + VectorFieldMapper.TypeParser typeParser = new VectorFieldMapper.TypeParser(); + + int efConstruction = 321; + int m = 12; + XContentBuilder xContentBuilder = XContentFactory.jsonBuilder() + .startObject() + .field(TYPE_FIELD_NAME, CORRELATION_VECTOR_TYPE) + .field(DIMENSION_FIELD_NAME, 2000) + .startObject("correlation_ctx") + .field("similarityFunction", "invalid") + .startObject("parameters") + .field("m", m) + .field("ef_construction", efConstruction) + .endObject() + .endObject() + .endObject(); + + expectThrows( + MapperParsingException.class, + () -> typeParser.parse( + fieldName, + XContentHelper.convertToMap(BytesReference.bytes(xContentBuilder), true, xContentBuilder.contentType()).v2(), + buildParserContext(indexName, settings) + ) + ); + } + + /** + * test parseCreateField in CorrelationVectorFieldMapper + * @throws IOException + */ + public void testCorrelationVectorFieldMapper_parseCreateField() throws IOException { + String fieldName = "test-field-name"; + int dimension = 10; + float[] testVector = createInitializedFloatArray(dimension, 1.0f); + CorrelationParamsContext correlationParamsContext = new CorrelationParamsContext(VectorSimilarityFunction.EUCLIDEAN, Map.of()); + + VectorFieldMapper.CorrelationVectorFieldType correlationVectorFieldType = new VectorFieldMapper.CorrelationVectorFieldType( + fieldName, + Map.of(), + dimension, + correlationParamsContext + ); + + CorrelationVectorFieldMapper.CreateLuceneFieldMapperInput input = new CorrelationVectorFieldMapper.CreateLuceneFieldMapperInput( + fieldName, + correlationVectorFieldType, + FieldMapper.MultiFields.empty(), + FieldMapper.CopyTo.empty(), + new Explicit<>(true, true), + false, + false, + correlationParamsContext + ); + + ParseContext.Document document = new ParseContext.Document(); + ContentPath contentPath = new ContentPath(); + ParseContext parseContext = mock(ParseContext.class); + when(parseContext.doc()).thenReturn(document); + when(parseContext.path()).thenReturn(contentPath); + + CorrelationVectorFieldMapper correlationVectorFieldMapper = Mockito.spy(new CorrelationVectorFieldMapper(input)); + doReturn(Optional.of(testVector)).when(correlationVectorFieldMapper).getFloatsFromContext(parseContext, dimension); + + correlationVectorFieldMapper.parseCreateField(parseContext, dimension); + + List fields = document.getFields(); + assertEquals(1, fields.size()); + IndexableField field = fields.get(0); + + Assert.assertTrue(field instanceof KnnFloatVectorField); + KnnFloatVectorField knnFloatVectorField = (KnnFloatVectorField) field; + Assert.assertArrayEquals(testVector, knnFloatVectorField.vectorValue(), 0.001f); + } + + /** + * test CorrelationVectorFieldType subclass + */ + public void testCorrelationVectorFieldType() { + String fieldName = "test-field-name"; + int dimension = 10; + QueryShardContext context = mock(QueryShardContext.class); + SearchLookup searchLookup = mock(SearchLookup.class); + + VectorFieldMapper.CorrelationVectorFieldType correlationVectorFieldType = new VectorFieldMapper.CorrelationVectorFieldType( + fieldName, + Map.of(), + dimension + ); + Assert.assertThrows(QueryShardException.class, () -> { correlationVectorFieldType.termQuery(new Object(), context); }); + Assert.assertThrows( + UnsupportedOperationException.class, + () -> { correlationVectorFieldType.valueFetcher(context, searchLookup, ""); } + ); + Assert.assertTrue(correlationVectorFieldType.existsQuery(context) instanceof FieldExistsQuery); + Assert.assertEquals(VectorFieldMapper.CONTENT_TYPE, correlationVectorFieldType.typeName()); + } + + /** + * test constants in VectorFieldMapper + */ + public void testVectorFieldMapperConstants() { + Assert.assertNotNull(VectorFieldMapper.Defaults.IGNORE_MALFORMED); + Assert.assertNotNull(VectorFieldMapper.Names.IGNORE_MALFORMED); + } + + private IndexMetadata buildIndexMetaData(String index, Settings settings) { + return IndexMetadata.builder(index) + .settings(settings) + .numberOfShards(1) + .numberOfReplicas(0) + .version(7) + .mappingVersion(0) + .settingsVersion(0) + .aliasesVersion(0) + .creationDate(0) + .build(); + } + + private Mapper.TypeParser.ParserContext buildParserContext(String index, Settings settings) { + IndexSettings indexSettings = new IndexSettings( + buildIndexMetaData(index, settings), + Settings.EMPTY, + new IndexScopedSettings(Settings.EMPTY, new HashSet<>(IndexScopedSettings.BUILT_IN_INDEX_SETTINGS)) + ); + + MapperService mapperService = mock(MapperService.class); + when(mapperService.getIndexSettings()).thenReturn(indexSettings); + + return new Mapper.TypeParser.ParserContext( + null, + mapperService, + type -> new VectorFieldMapper.TypeParser(), + Version.CURRENT, + null, + null, + null + ); + } + + private static float[] createInitializedFloatArray(int dimension, float value) { + float[] array = new float[dimension]; + Arrays.fill(array, value); + return array; + } +} diff --git a/plugins/events-correlation-engine/src/test/java/org/opensearch/plugin/correlation/core/index/query/CorrelationQueryBuilderTests.java b/plugins/events-correlation-engine/src/test/java/org/opensearch/plugin/correlation/core/index/query/CorrelationQueryBuilderTests.java new file mode 100644 index 0000000000000..fd3c7220aad74 --- /dev/null +++ b/plugins/events-correlation-engine/src/test/java/org/opensearch/plugin/correlation/core/index/query/CorrelationQueryBuilderTests.java @@ -0,0 +1,268 @@ +/* + * 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.plugin.correlation.core.index.query; + +import org.apache.lucene.search.KnnFloatVectorQuery; +import org.junit.Assert; +import org.opensearch.Version; +import org.opensearch.cluster.ClusterModule; +import org.opensearch.common.Strings; +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.common.io.stream.NamedWriteableAwareStreamInput; +import org.opensearch.common.io.stream.NamedWriteableRegistry; +import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.common.xcontent.XContentFactory; +import org.opensearch.common.xcontent.XContentType; +import org.opensearch.common.xcontent.json.JsonXContent; +import org.opensearch.core.xcontent.NamedXContentRegistry; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParser; +import org.opensearch.index.Index; +import org.opensearch.index.mapper.NumberFieldMapper; +import org.opensearch.index.query.QueryBuilder; +import org.opensearch.index.query.QueryBuilders; +import org.opensearch.index.query.QueryShardContext; +import org.opensearch.index.query.TermQueryBuilder; +import org.opensearch.plugin.correlation.core.index.mapper.VectorFieldMapper; +import org.opensearch.plugins.SearchPlugin; +import org.opensearch.test.OpenSearchTestCase; + +import java.io.IOException; +import java.util.List; +import java.util.Optional; + +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +/** + * Unit tests for Correlation Query Builder + */ +public class CorrelationQueryBuilderTests extends OpenSearchTestCase { + + private static final String FIELD_NAME = "myvector"; + private static final int K = 1; + private static final TermQueryBuilder TERM_QUERY = QueryBuilders.termQuery("field", "value"); + private static final float[] QUERY_VECTOR = new float[] { 1.0f, 2.0f, 3.0f, 4.0f }; + + /** + * test invalid number of nearby neighbors + */ + public void testInvalidK() { + float[] queryVector = { 1.0f, 1.0f }; + + expectThrows(IllegalArgumentException.class, () -> new CorrelationQueryBuilder(FIELD_NAME, queryVector, -K)); + expectThrows(IllegalArgumentException.class, () -> new CorrelationQueryBuilder(FIELD_NAME, queryVector, 0)); + expectThrows( + IllegalArgumentException.class, + () -> new CorrelationQueryBuilder(FIELD_NAME, queryVector, CorrelationQueryBuilder.K_MAX + 1) + ); + } + + /** + * test empty vector scenario + */ + public void testEmptyVector() { + final float[] queryVector = null; + expectThrows(IllegalArgumentException.class, () -> new CorrelationQueryBuilder(FIELD_NAME, queryVector, 1)); + final float[] queryVector1 = new float[] {}; + expectThrows(IllegalArgumentException.class, () -> new CorrelationQueryBuilder(FIELD_NAME, queryVector1, 1)); + } + + /** + * test serde with xcontent + * @throws IOException IOException + */ + public void testFromXContent() throws IOException { + CorrelationQueryBuilder correlationQueryBuilder = new CorrelationQueryBuilder(FIELD_NAME, QUERY_VECTOR, K); + XContentBuilder builder = XContentFactory.jsonBuilder(); + builder.startObject(); + builder.startObject(correlationQueryBuilder.fieldName()); + builder.field(CorrelationQueryBuilder.VECTOR_FIELD.getPreferredName(), correlationQueryBuilder.vector()); + builder.field(CorrelationQueryBuilder.K_FIELD.getPreferredName(), correlationQueryBuilder.getK()); + builder.endObject(); + builder.endObject(); + XContentParser contentParser = createParser(builder); + contentParser.nextToken(); + CorrelationQueryBuilder actualBuilder = CorrelationQueryBuilder.parse(contentParser); + Assert.assertEquals(actualBuilder, correlationQueryBuilder); + } + + /** + * test serde with xcontent + * @throws IOException IOException + */ + public void testFromXContentFromString() throws IOException { + String correlationQuery = "{\n" + + " \"myvector\" : {\n" + + " \"vector\" : [\n" + + " 1.0,\n" + + " 2.0,\n" + + " 3.0,\n" + + " 4.0\n" + + " ],\n" + + " \"k\" : 1,\n" + + " \"boost\" : 1.0\n" + + " }\n" + + "}"; + XContentParser contentParser = createParser(JsonXContent.jsonXContent, correlationQuery); + contentParser.nextToken(); + CorrelationQueryBuilder actualBuilder = CorrelationQueryBuilder.parse(contentParser); + Assert.assertEquals(correlationQuery.replace("\n", "").replace(" ", ""), Strings.toString(XContentType.JSON, actualBuilder)); + } + + /** + * test serde with xcontent with filters + * @throws IOException IOException + */ + public void testFromXContentWithFilters() throws IOException { + CorrelationQueryBuilder correlationQueryBuilder = new CorrelationQueryBuilder(FIELD_NAME, QUERY_VECTOR, K, TERM_QUERY); + XContentBuilder builder = XContentFactory.jsonBuilder(); + builder.startObject(); + builder.startObject(correlationQueryBuilder.fieldName()); + builder.field(CorrelationQueryBuilder.VECTOR_FIELD.getPreferredName(), correlationQueryBuilder.vector()); + builder.field(CorrelationQueryBuilder.K_FIELD.getPreferredName(), correlationQueryBuilder.getK()); + builder.field(CorrelationQueryBuilder.FILTER_FIELD.getPreferredName(), correlationQueryBuilder.getFilter()); + builder.endObject(); + builder.endObject(); + XContentParser contentParser = createParser(builder); + contentParser.nextToken(); + CorrelationQueryBuilder actualBuilder = CorrelationQueryBuilder.parse(contentParser); + Assert.assertEquals(actualBuilder, correlationQueryBuilder); + } + + /** + * test conversion o KnnFloatVectorQuery logic + * @throws IOException IOException + */ + public void testDoToQuery() throws IOException { + CorrelationQueryBuilder correlationQueryBuilder = new CorrelationQueryBuilder(FIELD_NAME, QUERY_VECTOR, K); + Index dummyIndex = new Index("dummy", "dummy"); + QueryShardContext mockQueryShardContext = mock(QueryShardContext.class); + VectorFieldMapper.CorrelationVectorFieldType mockCorrVectorField = mock(VectorFieldMapper.CorrelationVectorFieldType.class); + when(mockQueryShardContext.index()).thenReturn(dummyIndex); + when(mockCorrVectorField.getDimension()).thenReturn(4); + when(mockQueryShardContext.fieldMapper(anyString())).thenReturn(mockCorrVectorField); + KnnFloatVectorQuery query = (KnnFloatVectorQuery) correlationQueryBuilder.doToQuery(mockQueryShardContext); + Assert.assertEquals(FIELD_NAME, query.getField()); + Assert.assertArrayEquals(QUERY_VECTOR, query.getTargetCopy(), 0.1f); + Assert.assertEquals(K, query.getK()); + } + + /** + * test conversion o KnnFloatVectorQuery logic with filter + * @throws IOException IOException + */ + public void testDoToQueryWithFilter() throws IOException { + CorrelationQueryBuilder correlationQueryBuilder = new CorrelationQueryBuilder(FIELD_NAME, QUERY_VECTOR, K, TERM_QUERY); + Index dummyIndex = new Index("dummy", "dummy"); + QueryShardContext mockQueryShardContext = mock(QueryShardContext.class); + VectorFieldMapper.CorrelationVectorFieldType mockCorrVectorField = mock(VectorFieldMapper.CorrelationVectorFieldType.class); + when(mockQueryShardContext.index()).thenReturn(dummyIndex); + when(mockCorrVectorField.getDimension()).thenReturn(4); + when(mockQueryShardContext.fieldMapper(anyString())).thenReturn(mockCorrVectorField); + KnnFloatVectorQuery query = (KnnFloatVectorQuery) correlationQueryBuilder.doToQuery(mockQueryShardContext); + Assert.assertEquals(FIELD_NAME, query.getField()); + Assert.assertArrayEquals(QUERY_VECTOR, query.getTargetCopy(), 0.1f); + Assert.assertEquals(K, query.getK()); + Assert.assertEquals(TERM_QUERY.toQuery(mockQueryShardContext), query.getFilter()); + } + + /** + * test conversion o KnnFloatVectorQuery logic failure with invalid dimensions + */ + public void testDoToQueryInvalidDimensions() { + CorrelationQueryBuilder correlationQueryBuilder = new CorrelationQueryBuilder(FIELD_NAME, QUERY_VECTOR, K); + Index dummyIndex = new Index("dummy", "dummy"); + QueryShardContext mockQueryShardContext = mock(QueryShardContext.class); + VectorFieldMapper.CorrelationVectorFieldType mockCorrVectorField = mock(VectorFieldMapper.CorrelationVectorFieldType.class); + when(mockQueryShardContext.index()).thenReturn(dummyIndex); + when(mockCorrVectorField.getDimension()).thenReturn(400); + when(mockQueryShardContext.fieldMapper(anyString())).thenReturn(mockCorrVectorField); + expectThrows(IllegalArgumentException.class, () -> correlationQueryBuilder.doToQuery(mockQueryShardContext)); + } + + /** + * test conversion o KnnFloatVectorQuery logic failure with invalid field type + */ + public void testDoToQueryInvalidFieldType() { + CorrelationQueryBuilder correlationQueryBuilder = new CorrelationQueryBuilder(FIELD_NAME, QUERY_VECTOR, K); + Index dummyIndex = new Index("dummy", "dummy"); + QueryShardContext mockQueryShardContext = mock(QueryShardContext.class); + NumberFieldMapper.NumberFieldType mockCorrVectorField = mock(NumberFieldMapper.NumberFieldType.class); + when(mockQueryShardContext.index()).thenReturn(dummyIndex); + when(mockQueryShardContext.fieldMapper(anyString())).thenReturn(mockCorrVectorField); + expectThrows(IllegalArgumentException.class, () -> correlationQueryBuilder.doToQuery(mockQueryShardContext)); + } + + /** + * test serialization of Correlation Query Builder + * @throws Exception + */ + public void testSerialization() throws Exception { + assertSerialization(Optional.empty()); + assertSerialization(Optional.of(TERM_QUERY)); + } + + private void assertSerialization(final Optional queryBuilderOptional) throws IOException { + final CorrelationQueryBuilder builder = queryBuilderOptional.isPresent() + ? new CorrelationQueryBuilder(FIELD_NAME, QUERY_VECTOR, K, queryBuilderOptional.get()) + : new CorrelationQueryBuilder(FIELD_NAME, QUERY_VECTOR, K); + + try (BytesStreamOutput output = new BytesStreamOutput()) { + output.setVersion(Version.CURRENT); + output.writeNamedWriteable(builder); + + try (StreamInput in = new NamedWriteableAwareStreamInput(output.bytes().streamInput(), writableRegistry())) { + in.setVersion(Version.CURRENT); + final QueryBuilder deserializedQuery = in.readNamedWriteable(QueryBuilder.class); + + assertNotNull(deserializedQuery); + assertTrue(deserializedQuery instanceof CorrelationQueryBuilder); + final CorrelationQueryBuilder deserializedKnnQueryBuilder = (CorrelationQueryBuilder) deserializedQuery; + assertEquals(FIELD_NAME, deserializedKnnQueryBuilder.fieldName()); + assertArrayEquals(QUERY_VECTOR, (float[]) deserializedKnnQueryBuilder.vector(), 0.0f); + assertEquals(K, deserializedKnnQueryBuilder.getK()); + if (queryBuilderOptional.isPresent()) { + assertNotNull(deserializedKnnQueryBuilder.getFilter()); + assertEquals(queryBuilderOptional.get(), deserializedKnnQueryBuilder.getFilter()); + } else { + assertNull(deserializedKnnQueryBuilder.getFilter()); + } + } + } + } + + @Override + protected NamedXContentRegistry xContentRegistry() { + List list = ClusterModule.getNamedXWriteables(); + SearchPlugin.QuerySpec spec = new SearchPlugin.QuerySpec<>( + TermQueryBuilder.NAME, + TermQueryBuilder::new, + TermQueryBuilder::fromXContent + ); + list.add(new NamedXContentRegistry.Entry(QueryBuilder.class, spec.getName(), (p, c) -> spec.getParser().fromXContent(p))); + NamedXContentRegistry registry = new NamedXContentRegistry(list); + return registry; + } + + @Override + protected NamedWriteableRegistry writableRegistry() { + final List entries = ClusterModule.getNamedWriteables(); + entries.add( + new NamedWriteableRegistry.Entry( + QueryBuilder.class, + CorrelationQueryBuilder.NAME_FIELD.getPreferredName(), + CorrelationQueryBuilder::new + ) + ); + entries.add(new NamedWriteableRegistry.Entry(QueryBuilder.class, TermQueryBuilder.NAME, TermQueryBuilder::new)); + return new NamedWriteableRegistry(entries); + } +} diff --git a/plugins/events-correlation-engine/src/test/java/org/opensearch/plugin/correlation/settings/EventsCorrelationSettingsTests.java b/plugins/events-correlation-engine/src/test/java/org/opensearch/plugin/correlation/settings/EventsCorrelationSettingsTests.java new file mode 100644 index 0000000000000..faec6057b04c6 --- /dev/null +++ b/plugins/events-correlation-engine/src/test/java/org/opensearch/plugin/correlation/settings/EventsCorrelationSettingsTests.java @@ -0,0 +1,58 @@ +/* + * 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.plugin.correlation.settings; + +import org.junit.Assert; +import org.junit.Before; +import org.opensearch.common.settings.Setting; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.plugin.correlation.EventsCorrelationPlugin; +import org.opensearch.test.OpenSearchTestCase; + +import java.util.List; +import java.util.concurrent.TimeUnit; + +/** + * Unit tests for Correlation Engine settings + */ +public class EventsCorrelationSettingsTests extends OpenSearchTestCase { + + private EventsCorrelationPlugin plugin; + + @Before + public void setup() { + plugin = new EventsCorrelationPlugin(); + } + + /** + * test all plugin settings returned + */ + public void testAllPluginSettingsReturned() { + List expectedSettings = List.of( + EventsCorrelationSettings.IS_CORRELATION_INDEX_SETTING, + EventsCorrelationSettings.CORRELATION_TIME_WINDOW + ); + + List> settings = plugin.getSettings(); + Assert.assertTrue(settings.containsAll(expectedSettings)); + } + + /** + * test settings get value + */ + public void testSettingsGetValue() { + Settings settings = Settings.builder().put("index.correlation", true).build(); + Assert.assertEquals(EventsCorrelationSettings.IS_CORRELATION_INDEX_SETTING.get(settings), true); + settings = Settings.builder() + .put("plugins.security_analytics.correlation_time_window", new TimeValue(10, TimeUnit.MINUTES)) + .build(); + Assert.assertEquals(EventsCorrelationSettings.CORRELATION_TIME_WINDOW.get(settings), new TimeValue(10, TimeUnit.MINUTES)); + } +} From f21d73ef553c1986872bf973ee70da219a7204c7 Mon Sep 17 00:00:00 2001 From: Owais Kazi Date: Fri, 23 Jun 2023 14:43:50 -0700 Subject: [PATCH 061/109] [Extensions] Fixed duplicate extension unique id (#8228) * Fixes duplicate extension unique id Signed-off-by: Owais Kazi * Handled different extension unique id Signed-off-by: Owais Kazi --------- Signed-off-by: Owais Kazi --- .../main/java/org/opensearch/transport/TransportService.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/opensearch/transport/TransportService.java b/server/src/main/java/org/opensearch/transport/TransportService.java index f41112d69196b..021bf39fc5a1f 100644 --- a/server/src/main/java/org/opensearch/transport/TransportService.java +++ b/server/src/main/java/org/opensearch/transport/TransportService.java @@ -544,7 +544,10 @@ public ConnectionManager.ConnectionValidator connectionValidatorForExtensionConn ) { return (newConnection, actualProfile, listener) -> { // We don't validate cluster names to allow for CCS connections. - threadPool.getThreadContext().putHeader("extension_unique_id", extensionUniqueId); + String currentId = threadPool.getThreadContext().getHeader("extension_unique_id"); + if (Strings.isNullOrEmpty(currentId) || !extensionUniqueId.equals(currentId)) { + threadPool.getThreadContext().putHeader("extension_unique_id", extensionUniqueId); + } handshake(newConnection, actualProfile.getHandshakeTimeout().millis(), cn -> true, ActionListener.map(listener, resp -> { final DiscoveryNode remote = resp.discoveryNode; From 3b04ef48e3054f75ff563418d7e1ab9566ea4d6f Mon Sep 17 00:00:00 2001 From: Rishikesh Pasham <62345295+Rishikesh1159@users.noreply.github.com> Date: Fri, 23 Jun 2023 14:59:29 -0700 Subject: [PATCH 062/109] [Segment Replication + Remote Store] Removing exception throw when remote store is used with system indices (#8235) * Removing throwing of exception when remote store is used with system indices. Signed-off-by: Rishikesh1159 * Add check of feature flag enabled. Signed-off-by: Rishikesh1159 * Add unit test to improve code coverage. Signed-off-by: Rishikesh1159 * Add warning log. Signed-off-by: Rishikesh1159 --------- Signed-off-by: Rishikesh1159 --- .../remotestore/CreateRemoteIndexIT.java | 41 ++++++++++++------- .../metadata/MetadataCreateIndexService.java | 27 ++++++------ .../MetadataCreateIndexServiceTests.java | 30 ++++++++++++++ 3 files changed, 71 insertions(+), 27 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/CreateRemoteIndexIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/CreateRemoteIndexIT.java index cf53fa7fac2cd..46966e289e75e 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/CreateRemoteIndexIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/CreateRemoteIndexIT.java @@ -12,6 +12,8 @@ import org.junit.Before; import org.opensearch.action.admin.indices.get.GetIndexRequest; import org.opensearch.action.admin.indices.get.GetIndexResponse; +import org.opensearch.action.admin.indices.settings.get.GetSettingsRequest; +import org.opensearch.action.admin.indices.settings.get.GetSettingsResponse; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; @@ -131,21 +133,30 @@ public void testDefaultRemoteStoreNoUserOverride() throws Exception { private static final String SYSTEM_INDEX_NAME = ".test-system-index"; public void testSystemIndexWithRemoteStoreClusterSetting() throws Exception { - IllegalArgumentException illegalArgumentException = expectThrows( - IllegalArgumentException.class, - () -> createIndex(SYSTEM_INDEX_NAME) - ); - assertThat( - illegalArgumentException.getMessage(), - containsString( - "Cannot enable [" - + SETTING_REMOTE_STORE_ENABLED - + "] when [" - + SETTING_REPLICATION_TYPE - + "] is " - + ReplicationType.DOCUMENT - ) - ); + createIndex(SYSTEM_INDEX_NAME); + ensureGreen(SYSTEM_INDEX_NAME); + final GetSettingsResponse response = client().admin() + .indices() + .getSettings(new GetSettingsRequest().indices(SYSTEM_INDEX_NAME).includeDefaults(true)) + .actionGet(); + // Verify that Document replication strategy is used + assertEquals(response.getSetting(SYSTEM_INDEX_NAME, SETTING_REPLICATION_TYPE), ReplicationType.DOCUMENT.toString()); + assertEquals(response.getSetting(SYSTEM_INDEX_NAME, SETTING_REMOTE_STORE_ENABLED), "false"); + } + + public void testSystemIndexWithRemoteStoreIndexSettings() throws Exception { + prepareCreate( + SYSTEM_INDEX_NAME, + Settings.builder().put(SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT).put(SETTING_REMOTE_STORE_ENABLED, true) + ).get(); + ensureGreen(SYSTEM_INDEX_NAME); + final GetSettingsResponse response = client().admin() + .indices() + .getSettings(new GetSettingsRequest().indices(SYSTEM_INDEX_NAME).includeDefaults(true)) + .actionGet(); + // Verify that Document replication strategy is used + assertEquals(response.getSetting(SYSTEM_INDEX_NAME, SETTING_REPLICATION_TYPE), ReplicationType.DOCUMENT.toString()); + assertEquals(response.getSetting(SYSTEM_INDEX_NAME, SETTING_REMOTE_STORE_ENABLED), "false"); } public void testRemoteStoreDisabledByUser() throws Exception { 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 a2a11b6bc9271..0a49feec61621 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java @@ -72,6 +72,7 @@ import org.opensearch.common.settings.IndexScopedSettings; import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.FeatureFlags; import org.opensearch.common.xcontent.XContentHelper; import org.opensearch.core.common.Strings; import org.opensearch.core.xcontent.NamedXContentRegistry; @@ -136,6 +137,7 @@ import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REPLICATION_TYPE; import static org.opensearch.cluster.metadata.Metadata.DEFAULT_REPLICA_COUNT_SETTING; +import static org.opensearch.common.util.FeatureFlags.REMOTE_STORE; import static org.opensearch.indices.IndicesService.CLUSTER_REMOTE_STORE_REPOSITORY_SETTING; import static org.opensearch.indices.IndicesService.CLUSTER_REMOTE_TRANSLOG_REPOSITORY_SETTING; import static org.opensearch.indices.IndicesService.CLUSTER_REMOTE_STORE_ENABLED_SETTING; @@ -898,8 +900,18 @@ static Settings aggregateIndexSettings( indexSettingsBuilder.put(IndexMetadata.SETTING_INDEX_PROVIDED_NAME, request.getProvidedName()); indexSettingsBuilder.put(SETTING_INDEX_UUID, UUIDs.randomBase64UUID()); - updateReplicationStrategy(indexSettingsBuilder, request.settings(), settings, isSystemIndex); - updateRemoteStoreSettings(indexSettingsBuilder, request.settings(), settings); + if (isSystemIndex || IndexMetadata.INDEX_HIDDEN_SETTING.get(request.settings())) { + logger.warn( + "Setting replication.type: DOCUMENT will be used for Index until Segment Replication supports System and Hidden indices" + ); + indexSettingsBuilder.put(SETTING_REPLICATION_TYPE, ReplicationType.DOCUMENT); + if (FeatureFlags.isEnabled(REMOTE_STORE)) { + indexSettingsBuilder.put(SETTING_REMOTE_STORE_ENABLED, false); + } + } else { + updateReplicationStrategy(indexSettingsBuilder, request.settings(), settings); + updateRemoteStoreSettings(indexSettingsBuilder, request.settings(), settings); + } if (sourceMetadata != null) { assert request.resizeType() != null; @@ -939,16 +951,7 @@ static Settings aggregateIndexSettings( * @param requestSettings settings passed in during index create request * @param clusterSettings cluster level settings */ - private static void updateReplicationStrategy( - Settings.Builder settingsBuilder, - Settings requestSettings, - Settings clusterSettings, - boolean isSystemIndex - ) { - if (isSystemIndex || IndexMetadata.INDEX_HIDDEN_SETTING.get(requestSettings)) { - settingsBuilder.put(SETTING_REPLICATION_TYPE, ReplicationType.DOCUMENT); - return; - } + private static void updateReplicationStrategy(Settings.Builder settingsBuilder, Settings requestSettings, Settings clusterSettings) { if (CLUSTER_REPLICATION_TYPE_SETTING.exists(clusterSettings) && INDEX_REPLICATION_TYPE_SETTING.exists(requestSettings) == false) { settingsBuilder.put(SETTING_REPLICATION_TYPE, CLUSTER_REPLICATION_TYPE_SETTING.get(clusterSettings)); return; diff --git a/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java b/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java index 206c8914a61ee..94a1ca9992cb6 100644 --- a/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java +++ b/server/src/test/java/org/opensearch/cluster/metadata/MetadataCreateIndexServiceTests.java @@ -1715,6 +1715,36 @@ public void testSystemIndexUsesDocumentReplication() { assertEquals(ReplicationType.DOCUMENT.toString(), indexSettings.get(SETTING_REPLICATION_TYPE)); } + public void testRemoteStoreDisabledForSystemIndices() { + Settings settings = Settings.builder() + .put(CLUSTER_REPLICATION_TYPE_SETTING.getKey(), ReplicationType.SEGMENT) + .put(CLUSTER_REMOTE_STORE_ENABLED_SETTING.getKey(), true) + .put(CLUSTER_REMOTE_STORE_REPOSITORY_SETTING.getKey(), "my-segment-repo-1") + .put(CLUSTER_REMOTE_TRANSLOG_STORE_ENABLED_SETTING.getKey(), true) + .put(CLUSTER_REMOTE_TRANSLOG_REPOSITORY_SETTING.getKey(), "my-translog-repo-1") + .build(); + FeatureFlagSetter.set(FeatureFlags.REMOTE_STORE); + + request = new CreateIndexClusterStateUpdateRequest("create index", "test", "test"); + final Settings.Builder requestSettings = Settings.builder(); + request.settings(requestSettings.build()); + // set isSystemIndex parameter as true + Settings indexSettings = aggregateIndexSettings( + ClusterState.EMPTY_STATE, + request, + Settings.EMPTY, + null, + settings, + IndexScopedSettings.DEFAULT_SCOPED_SETTINGS, + randomShardLimitService(), + Collections.emptySet(), + true + ); + // Verify that remote store is disabled. + assertEquals(indexSettings.get(SETTING_REMOTE_STORE_ENABLED), "false"); + assertEquals(ReplicationType.DOCUMENT.toString(), indexSettings.get(SETTING_REPLICATION_TYPE)); + } + private IndexTemplateMetadata addMatchingTemplate(Consumer configurator) { IndexTemplateMetadata.Builder builder = templateMetadataBuilder("template1", "te*"); configurator.accept(builder); From 2407e4665259da4122dede50d8c133429acb953d Mon Sep 17 00:00:00 2001 From: Marc Handalian Date: Fri, 23 Jun 2023 20:57:05 -0700 Subject: [PATCH 063/109] Mute Flaky Test SegmentReplicationAllocationIT.testPerIndexPrimaryAllocation (#8244) Signed-off-by: Marc Handalian --- .../indices/replication/SegmentReplicationAllocationIT.java | 1 + 1 file changed, 1 insertion(+) diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationAllocationIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationAllocationIT.java index 5056adea32f83..9f382d6a2e9ef 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationAllocationIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationAllocationIT.java @@ -96,6 +96,7 @@ public void testGlobalPrimaryAllocation() throws Exception { * This test in general passes without primary shard balance as well due to nature of allocation algorithm which * assigns all primary shards first followed by replica copies. */ + @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/7751") public void testPerIndexPrimaryAllocation() throws Exception { internalCluster().startClusterManagerOnlyNode(); final int maxReplicaCount = 2; From b9edb5ac0804081ee2fe6203f732bbabd55c7aeb Mon Sep 17 00:00:00 2001 From: Suraj Singh Date: Sat, 24 Jun 2023 10:13:07 -0700 Subject: [PATCH 064/109] [Segment Replication] Update segrep bwc tests to verify replica checkpoints and skip tests for 1.x bwc versions (#8203) * [Segment Replication] Verify segment replication stats in bwc test Signed-off-by: Suraj Singh * Log cleanup Signed-off-by: Suraj Singh * Spotless check Signed-off-by: Suraj Singh * Add version check to skip test for 1.x bwc branches Signed-off-by: Suraj Singh * Add version check to skip test for 1.x bwc branches for mixed clusters Signed-off-by: Suraj Singh * Add version string in build to identify bwc version Signed-off-by: Suraj Singh * Use correct bwc version string Signed-off-by: Suraj Singh * Address review comments from https://github.com/opensearch-project/OpenSearch/pull/7626 Signed-off-by: Suraj Singh --------- Signed-off-by: Suraj Singh --- qa/mixed-cluster/build.gradle | 2 ++ .../org/opensearch/backwards/IndexingIT.java | 17 ++++++++-- .../org/opensearch/upgrades/IndexingIT.java | 32 ++++++++++++++++--- 3 files changed, 43 insertions(+), 8 deletions(-) diff --git a/qa/mixed-cluster/build.gradle b/qa/mixed-cluster/build.gradle index 55f900c52f2c2..d64bf245dbf8f 100644 --- a/qa/mixed-cluster/build.gradle +++ b/qa/mixed-cluster/build.gradle @@ -55,6 +55,7 @@ for (Version bwcVersion : BuildParams.bwcVersions.wireCompatible) { } String baseName = "v${bwcVersion}" + String bwcVersionStr = "${bwcVersion}" /* This project runs the core REST tests against a 4 node cluster where two of the nodes has a different minor. */ @@ -86,6 +87,7 @@ for (Version bwcVersion : BuildParams.bwcVersions.wireCompatible) { nonInputProperties.systemProperty('tests.rest.cluster', "${-> testClusters."${baseName}".allHttpSocketURI.join(",")}") nonInputProperties.systemProperty('tests.clustername', "${-> testClusters."${baseName}".getName()}") } + systemProperty 'tests.upgrade_from_version', bwcVersionStr systemProperty 'tests.path.repo', "${buildDir}/cluster/shared/repo/${baseName}" onlyIf { project.bwc_tests_enabled } } diff --git a/qa/mixed-cluster/src/test/java/org/opensearch/backwards/IndexingIT.java b/qa/mixed-cluster/src/test/java/org/opensearch/backwards/IndexingIT.java index a6675a6d0ddb5..b867b90af333c 100644 --- a/qa/mixed-cluster/src/test/java/org/opensearch/backwards/IndexingIT.java +++ b/qa/mixed-cluster/src/test/java/org/opensearch/backwards/IndexingIT.java @@ -66,6 +66,9 @@ public class IndexingIT extends OpenSearchRestTestCase { + protected static final Version UPGRADE_FROM_VERSION = Version.fromString(System.getProperty("tests.upgrade_from_version")); + + private int indexDocs(String index, final int idStart, final int numDocs) throws IOException { for (int i = 0; i < numDocs; i++) { final int id = idStart + i; @@ -114,12 +117,16 @@ private void printClusterRouting() throws IOException, ParseException { * @throws Exception */ public void testIndexingWithPrimaryOnBwcNodes() throws Exception { + if (UPGRADE_FROM_VERSION.before(Version.V_2_4_0)) { + logger.info("--> Skip test for version {} where segment replication feature is not available", UPGRADE_FROM_VERSION); + return; + } Nodes nodes = buildNodeAndVersions(); assumeFalse("new nodes is empty", nodes.getNewNodes().isEmpty()); logger.info("cluster discovered:\n {}", nodes.toString()); final List bwcNamesList = nodes.getBWCNodes().stream().map(Node::getNodeName).collect(Collectors.toList()); final String bwcNames = bwcNamesList.stream().collect(Collectors.joining(",")); - // Exclude bwc nodes from allocation so that primaries gets allocated on current version + // Update allocation settings so that primaries gets allocated only on nodes running on older version Settings.Builder settings = Settings.builder() .put(IndexMetadata.INDEX_NUMBER_OF_SHARDS_SETTING.getKey(), 1) .put(IndexMetadata.INDEX_NUMBER_OF_REPLICAS_SETTING.getKey(), 0) @@ -133,7 +140,7 @@ public void testIndexingWithPrimaryOnBwcNodes() throws Exception { try (RestClient nodeClient = buildClient(restClientSettings(), nodes.getNewNodes().stream().map(Node::getPublishAddress).toArray(HttpHost[]::new))) { - logger.info("allowing replica shards assignment on bwc nodes"); + logger.info("Remove allocation include settings so that shards can be allocated on current version nodes"); updateIndexSettings(index, Settings.builder().putNull("index.routing.allocation.include._name")); // Add replicas so that it can be assigned on higher OS version nodes. updateIndexSettings(index, Settings.builder().put("index.number_of_replicas", 2)); @@ -154,13 +161,17 @@ public void testIndexingWithPrimaryOnBwcNodes() throws Exception { /** - * This test creates a cluster with primary on older version but due to {@link org.opensearch.cluster.routing.allocation.decider.NodeVersionAllocationDecider}; + * This test creates a cluster with primary on higher version but due to {@link org.opensearch.cluster.routing.allocation.decider.NodeVersionAllocationDecider}; * replica shard allocation on lower OpenSearch version is prevented. Thus, this test though cover the use case where * primary shard containing nodes are running on higher OS version while replicas are unassigned. * * @throws Exception */ public void testIndexingWithReplicaOnBwcNodes() throws Exception { + if (UPGRADE_FROM_VERSION.before(Version.V_2_4_0)) { + logger.info("--> Skip test for version {} where segment replication feature is not available", UPGRADE_FROM_VERSION); + return; + } Nodes nodes = buildNodeAndVersions(); assumeFalse("new nodes is empty", nodes.getNewNodes().isEmpty()); logger.info("cluster discovered:\n {}", nodes.toString()); diff --git a/qa/rolling-upgrade/src/test/java/org/opensearch/upgrades/IndexingIT.java b/qa/rolling-upgrade/src/test/java/org/opensearch/upgrades/IndexingIT.java index a758b8e4ccd72..173aa9f6557d2 100644 --- a/qa/rolling-upgrade/src/test/java/org/opensearch/upgrades/IndexingIT.java +++ b/qa/rolling-upgrade/src/test/java/org/opensearch/upgrades/IndexingIT.java @@ -38,13 +38,17 @@ import org.opensearch.client.Response; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.Booleans; +import org.opensearch.common.io.Streams; import org.opensearch.common.settings.Settings; +import org.opensearch.index.codec.CodecService; +import org.opensearch.index.engine.EngineConfig; import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.test.rest.yaml.ObjectPath; import java.io.IOException; import java.net.URISyntaxException; import java.nio.charset.StandardCharsets; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -90,6 +94,7 @@ private void waitForSearchableDocs(String index, int shardCount) throws Exceptio waitForClusterHealthWithNoShardMigration(index, "green"); logger.info("--> _cat/shards before search \n{}", EntityUtils.toString(client().performRequest(new Request("GET", "/_cat/shards?v")).getEntity())); + verifySegmentStats(index); Request request = new Request("GET", index + "/_stats"); request.addParameter("level", "shards"); Response response = client().performRequest(request); @@ -109,14 +114,12 @@ private void waitForSearchableDocs(String index, int shardCount) throws Exceptio logger.info("--> replicaShardToNodeIDMap {}", replicaShardToNodeIDMap); for (int shardNumber = 0; shardNumber < shardCount; shardNumber++) { - logger.info("--> Verify doc count for shard number {}", shardNumber); Request searchTestIndexRequest = new Request("POST", "/" + index + "/_search"); searchTestIndexRequest.addParameter(TOTAL_HITS_AS_INT_PARAM, "true"); searchTestIndexRequest.addParameter("filter_path", "hits.total"); searchTestIndexRequest.addParameter("preference", "_shards:" + shardNumber + "|_only_nodes:" + primaryShardToNodeIDMap.get(shardNumber)); Response searchTestIndexResponse = client().performRequest(searchTestIndexRequest); final int primaryHits = ObjectPath.createFromResponse(searchTestIndexResponse).evaluate("hits.total"); - logger.info("--> primaryHits {}", primaryHits); final int shardNum = shardNumber; // Verify replica shard doc count only when available. if (replicaShardToNodeIDMap.get(shardNum) != null) { @@ -127,8 +130,7 @@ private void waitForSearchableDocs(String index, int shardCount) throws Exceptio replicaRequest.addParameter("preference", "_shards:" + shardNum + "|_only_nodes:" + replicaShardToNodeIDMap.get(shardNum)); Response replicaResponse = client().performRequest(replicaRequest); int replicaHits = ObjectPath.createFromResponse(replicaResponse).evaluate("hits.total"); - logger.info("--> ReplicaHits {}", replicaHits); - assertEquals(primaryHits, replicaHits); + assertEquals("Doc count mismatch for shard " + shardNum + ". Primary hits " + primaryHits + " Replica hits " + replicaHits, primaryHits, replicaHits); }, 1, TimeUnit.MINUTES); } } @@ -145,6 +147,18 @@ private void waitForClusterHealthWithNoShardMigration(String indexName, String s client().performRequest(waitForStatus); } + private void verifySegmentStats(String indexName) throws Exception { + assertBusy(() -> { + Request segrepStatsRequest = new Request("GET", "/_cat/segment_replication/" + indexName); + segrepStatsRequest.addParameter("h", "shardId,target_node,checkpoints_behind"); + Response segrepStatsResponse = client().performRequest(segrepStatsRequest); + for (String statLine : Streams.readAllLines(segrepStatsResponse.getEntity().getContent())) { + String[] elements = statLine.split(" +"); + assertEquals("Replica shard " + elements[0] + "not upto date with primary ", 0, Integer.parseInt(elements[2])); + } + }); + } + public void testIndexing() throws IOException, ParseException { switch (CLUSTER_TYPE) { case OLD: @@ -239,9 +253,13 @@ public void testIndexing() throws IOException, ParseException { * @throws Exception */ public void testIndexingWithSegRep() throws Exception { + if (UPGRADE_FROM_VERSION.before(Version.V_2_4_0)) { + logger.info("--> Skip test for version {} where segment replication feature is not available", UPGRADE_FROM_VERSION); + return; + } final String indexName = "test-index-segrep"; final int shardCount = 3; - final int replicaCount = 1; + final int replicaCount = 2; logger.info("--> Case {}", CLUSTER_TYPE); printClusterNodes(); logger.info("--> _cat/shards before test execution \n{}", EntityUtils.toString(client().performRequest(new Request("GET", "/_cat/shards?v")).getEntity())); @@ -251,6 +269,10 @@ public void testIndexingWithSegRep() throws Exception { .put(IndexMetadata.INDEX_NUMBER_OF_SHARDS_SETTING.getKey(), shardCount) .put(IndexMetadata.INDEX_NUMBER_OF_REPLICAS_SETTING.getKey(), replicaCount) .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) + .put( + EngineConfig.INDEX_CODEC_SETTING.getKey(), + randomFrom(CodecService.DEFAULT_CODEC, CodecService.BEST_COMPRESSION_CODEC, CodecService.LUCENE_DEFAULT_CODEC) + ) .put(INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), "100ms"); createIndex(indexName, settings.build()); waitForClusterHealthWithNoShardMigration(indexName, "green"); From 9aad3a3b7f753e2ab69c1d3f834e4e21af372380 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 26 Jun 2023 13:21:52 -0400 Subject: [PATCH 065/109] Bump org.apache.hadoop:hadoop-minicluster from 3.3.5 to 3.3.6 in /test/fixtures/hdfs-fixture (#8257) * Bump org.apache.hadoop:hadoop-minicluster in /test/fixtures/hdfs-fixture Bumps org.apache.hadoop:hadoop-minicluster from 3.3.5 to 3.3.6. --- updated-dependencies: - dependency-name: org.apache.hadoop:hadoop-minicluster dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] * Update changelog Signed-off-by: dependabot[bot] --------- Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: dependabot[bot] --- CHANGELOG.md | 3 ++- test/fixtures/hdfs-fixture/build.gradle | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 16c794135cb23..8f830334f7897 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -110,6 +110,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `commons-io:commons-io` from 2.12.0 to 2.13.0 in /plugins/discovery-azure-classic ([#8140](https://github.com/opensearch-project/OpenSearch/pull/8140)) - Bump `mockito` from 5.2.0 to 5.4.0 ([#8181](https://github.com/opensearch-project/OpenSearch/pull/8181)) - Bump `netty` from 4.1.93.Final to 4.1.94.Final ([#8191](https://github.com/opensearch-project/OpenSearch/pull/8191)) +- Bump `org.apache.hadoop:hadoop-minicluster` from 3.3.5 to 3.3.6 (#8257) ### Changed - Replace jboss-annotations-api_1.2_spec with jakarta.annotation-api ([#7836](https://github.com/opensearch-project/OpenSearch/pull/7836)) @@ -139,4 +140,4 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), ### Security [Unreleased 3.0]: https://github.com/opensearch-project/OpenSearch/compare/2.x...HEAD -[Unreleased 2.x]: https://github.com/opensearch-project/OpenSearch/compare/2.8...2.x +[Unreleased 2.x]: https://github.com/opensearch-project/OpenSearch/compare/2.8...2.x \ No newline at end of file diff --git a/test/fixtures/hdfs-fixture/build.gradle b/test/fixtures/hdfs-fixture/build.gradle index 70b84a405c9c6..de6f69a4fd4ce 100644 --- a/test/fixtures/hdfs-fixture/build.gradle +++ b/test/fixtures/hdfs-fixture/build.gradle @@ -37,7 +37,7 @@ versions << [ ] dependencies { - api("org.apache.hadoop:hadoop-minicluster:3.3.5") { + api("org.apache.hadoop:hadoop-minicluster:3.3.6") { exclude module: 'websocket-client' exclude module: 'jettison' exclude module: 'netty' From 3b2a93f9fc89cdf5b19c8af0381e19ff030d4374 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 26 Jun 2023 14:10:51 -0400 Subject: [PATCH 066/109] Bump com.networknt:json-schema-validator from 1.0.84 to 1.0.85 in /buildSrc (#8255) * Bump com.networknt:json-schema-validator in /buildSrc Bumps [com.networknt:json-schema-validator](https://github.com/networknt/json-schema-validator) from 1.0.84 to 1.0.85. - [Release notes](https://github.com/networknt/json-schema-validator/releases) - [Changelog](https://github.com/networknt/json-schema-validator/blob/master/CHANGELOG.md) - [Commits](https://github.com/networknt/json-schema-validator/compare/1.0.84...1.0.85) --- updated-dependencies: - dependency-name: com.networknt:json-schema-validator dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] * Update changelog Signed-off-by: dependabot[bot] --------- Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: dependabot[bot] --- CHANGELOG.md | 2 +- buildSrc/build.gradle | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 8f830334f7897..cb2a75da2454d 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -95,7 +95,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `netty` from 4.1.91.Final to 4.1.93.Final ([#7901](https://github.com/opensearch-project/OpenSearch/pull/7901)) - Bump `com.amazonaws` 1.12.270 to `software.amazon.awssdk` 2.20.55 ([7372](https://github.com/opensearch-project/OpenSearch/pull/7372/)) - Add `org.reactivestreams` 1.0.4 ([7372](https://github.com/opensearch-project/OpenSearch/pull/7372/)) -- Bump `com.networknt:json-schema-validator` from 1.0.81 to 1.0.83 ([7968](https://github.com/opensearch-project/OpenSearch/pull/7968)) +- Bump `com.networknt:json-schema-validator` from 1.0.81 to 1.0.85 ([7968], #8255) - Bump `com.netflix.nebula:gradle-extra-configurations-plugin` from 9.0.0 to 10.0.0 in /buildSrc ([#7068](https://github.com/opensearch-project/OpenSearch/pull/7068)) - Bump `com.google.guava:guava` from 32.0.0-jre to 32.0.1-jre (#8009) - Bump `commons-io:commons-io` from 2.12.0 to 2.13.0 (#8014, #8013, #8010) diff --git a/buildSrc/build.gradle b/buildSrc/build.gradle index abbdc6f6a570e..eca536e6e90cf 100644 --- a/buildSrc/build.gradle +++ b/buildSrc/build.gradle @@ -118,7 +118,7 @@ dependencies { api 'com.avast.gradle:gradle-docker-compose-plugin:0.16.12' api "org.yaml:snakeyaml:${props.getProperty('snakeyaml')}" api 'org.apache.maven:maven-model:3.9.2' - api 'com.networknt:json-schema-validator:1.0.84' + api 'com.networknt:json-schema-validator:1.0.85' api 'org.jruby.jcodings:jcodings:1.0.58' api 'org.jruby.joni:joni:2.1.48' api "com.fasterxml.jackson.core:jackson-databind:${props.getProperty('jackson_databind')}" From c29e4aa774199b007acffd17bb016f4bfc681874 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 26 Jun 2023 14:52:52 -0400 Subject: [PATCH 067/109] Bump io.projectreactor.netty:reactor-netty-http from 1.1.7 to 1.1.8 in /plugins/repository-azure (#8256) * Bump io.projectreactor.netty:reactor-netty-http Bumps [io.projectreactor.netty:reactor-netty-http](https://github.com/reactor/reactor-netty) from 1.1.7 to 1.1.8. - [Release notes](https://github.com/reactor/reactor-netty/releases) - [Commits](https://github.com/reactor/reactor-netty/compare/v1.1.7...v1.1.8) --- updated-dependencies: - dependency-name: io.projectreactor.netty:reactor-netty-http dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] * Updating SHAs Signed-off-by: dependabot[bot] * Update changelog Signed-off-by: dependabot[bot] --------- Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: dependabot[bot] --- CHANGELOG.md | 1 + plugins/repository-azure/build.gradle | 2 +- .../repository-azure/licenses/reactor-netty-http-1.1.7.jar.sha1 | 1 - .../repository-azure/licenses/reactor-netty-http-1.1.8.jar.sha1 | 1 + 4 files changed, 3 insertions(+), 2 deletions(-) delete mode 100644 plugins/repository-azure/licenses/reactor-netty-http-1.1.7.jar.sha1 create mode 100644 plugins/repository-azure/licenses/reactor-netty-http-1.1.8.jar.sha1 diff --git a/CHANGELOG.md b/CHANGELOG.md index cb2a75da2454d..12d57af9018f5 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -111,6 +111,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `mockito` from 5.2.0 to 5.4.0 ([#8181](https://github.com/opensearch-project/OpenSearch/pull/8181)) - Bump `netty` from 4.1.93.Final to 4.1.94.Final ([#8191](https://github.com/opensearch-project/OpenSearch/pull/8191)) - Bump `org.apache.hadoop:hadoop-minicluster` from 3.3.5 to 3.3.6 (#8257) +- Bump `io.projectreactor.netty:reactor-netty-http` from 1.1.7 to 1.1.8 (#8256) ### Changed - Replace jboss-annotations-api_1.2_spec with jakarta.annotation-api ([#7836](https://github.com/opensearch-project/OpenSearch/pull/7836)) diff --git a/plugins/repository-azure/build.gradle b/plugins/repository-azure/build.gradle index 48a49af165542..e67ea7ab0a11e 100644 --- a/plugins/repository-azure/build.gradle +++ b/plugins/repository-azure/build.gradle @@ -60,7 +60,7 @@ dependencies { api 'io.projectreactor:reactor-core:3.5.6' api 'io.projectreactor.netty:reactor-netty:1.1.7' api 'io.projectreactor.netty:reactor-netty-core:1.1.7' - api 'io.projectreactor.netty:reactor-netty-http:1.1.7' + api 'io.projectreactor.netty:reactor-netty-http:1.1.8' api "org.slf4j:slf4j-api:${versions.slf4j}" api "com.fasterxml.jackson.core:jackson-annotations:${versions.jackson}" api "com.fasterxml.jackson.core:jackson-databind:${versions.jackson_databind}" diff --git a/plugins/repository-azure/licenses/reactor-netty-http-1.1.7.jar.sha1 b/plugins/repository-azure/licenses/reactor-netty-http-1.1.7.jar.sha1 deleted file mode 100644 index 33bf2aabcfc9b..0000000000000 --- a/plugins/repository-azure/licenses/reactor-netty-http-1.1.7.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -39d7c0a13afa471b426a30bcf82664496ad34723 \ No newline at end of file diff --git a/plugins/repository-azure/licenses/reactor-netty-http-1.1.8.jar.sha1 b/plugins/repository-azure/licenses/reactor-netty-http-1.1.8.jar.sha1 new file mode 100644 index 0000000000000..5092608c90eba --- /dev/null +++ b/plugins/repository-azure/licenses/reactor-netty-http-1.1.8.jar.sha1 @@ -0,0 +1 @@ +696ea25658295e49906c6aad13fa70acbdeb2359 \ No newline at end of file From e802b538beb4f0071c6d28ceae655d2b26c70236 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 26 Jun 2023 13:13:43 -0700 Subject: [PATCH 068/109] Bump org.jruby.joni:joni from 2.1.48 to 2.2.1 (#8254) Bumps [org.jruby.joni:joni](https://github.com/jruby/joni) from 2.1.48 to 2.2.1. - [Commits](https://github.com/jruby/joni/compare/joni-2.1.48...joni-2.2.1) --- updated-dependencies: - dependency-name: org.jruby.joni:joni dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Signed-off-by: Andriy Redko Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- CHANGELOG.md | 2 +- buildSrc/build.gradle | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 12d57af9018f5..f4db81ecf16e5 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -101,7 +101,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `commons-io:commons-io` from 2.12.0 to 2.13.0 (#8014, #8013, #8010) - Bump `com.diffplug.spotless` from 6.18.0 to 6.19.0 (#8007) - Bump `'com.azure:azure-storage-blob` to 12.22.2 from 12.21.1 ([#8043](https://github.com/opensearch-project/OpenSearch/pull/8043)) -- Bump `org.jruby.joni:joni` from 2.1.48 to 2.2.1 (#8015) +- Bump `org.jruby.joni:joni` from 2.1.48 to 2.2.1 (#8015, #8254) - Bump `com.google.guava:guava` from 32.0.0-jre to 32.0.1-jre ([#8011](https://github.com/opensearch-project/OpenSearch/pull/8011), [#8012](https://github.com/opensearch-project/OpenSearch/pull/8012), [#8107](https://github.com/opensearch-project/OpenSearch/pull/8107)) - Bump `io.projectreactor:reactor-core` from 3.4.18 to 3.5.6 in /plugins/repository-azure ([#8016](https://github.com/opensearch-project/OpenSearch/pull/8016)) - Bump `spock-core` from 2.1-groovy-3.0 to 2.3-groovy-3.0 ([#8122](https://github.com/opensearch-project/OpenSearch/pull/8122)) diff --git a/buildSrc/build.gradle b/buildSrc/build.gradle index eca536e6e90cf..35f3fb87560e7 100644 --- a/buildSrc/build.gradle +++ b/buildSrc/build.gradle @@ -120,7 +120,7 @@ dependencies { api 'org.apache.maven:maven-model:3.9.2' api 'com.networknt:json-schema-validator:1.0.85' api 'org.jruby.jcodings:jcodings:1.0.58' - api 'org.jruby.joni:joni:2.1.48' + api 'org.jruby.joni:joni:2.2.1' api "com.fasterxml.jackson.core:jackson-databind:${props.getProperty('jackson_databind')}" api "org.ajoberstar.grgit:grgit-core:5.2.0" From 57798de012135d9c223957933c3ce207297ba13f Mon Sep 17 00:00:00 2001 From: Varun Bansal Date: Tue, 27 Jun 2023 04:54:15 +0530 Subject: [PATCH 069/109] Add integ tests for remote store stats api (#8135) Signed-off-by: bansvaru --- .../RemoteStoreBaseIntegTestCase.java | 22 +++- .../remotestore/RemoteStoreStatsIT.java | 115 ++++++++++++++---- 2 files changed, 106 insertions(+), 31 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java index 0ffa5ab23e0b6..d226d0d757638 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java @@ -62,22 +62,36 @@ private Settings defaultIndexSettings() { .build(); } - protected Settings remoteStoreIndexSettings(int numberOfReplicas) { + protected Settings remoteStoreIndexSettings(int numberOfReplicas, int numberOfShards) { return Settings.builder() .put(defaultIndexSettings()) - .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, numberOfShards) .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, numberOfReplicas) .build(); } - protected Settings remoteTranslogIndexSettings(int numberOfReplicas) { + protected Settings remoteStoreIndexSettings(int numberOfReplicas) { + return remoteStoreIndexSettings(numberOfReplicas, 1); + } + + protected Settings remoteTranslogIndexSettings(int numberOfReplicas, int numberOfShards) { return Settings.builder() - .put(remoteStoreIndexSettings(numberOfReplicas)) + .put(remoteStoreIndexSettings(numberOfReplicas, numberOfShards)) .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_ENABLED, true) .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, REPOSITORY_NAME) .build(); } + protected Settings remoteTranslogIndexSettings(int numberOfReplicas) { + return remoteTranslogIndexSettings(numberOfReplicas, 1); + } + + protected void putRepository(Path path) { + assertAcked( + clusterAdmin().preparePutRepository(REPOSITORY_NAME).setType("fs").setSettings(Settings.builder().put("location", path)) + ); + } + @Before public void setup() { internalCluster().startClusterManagerOnlyNode(); diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreStatsIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreStatsIT.java index 3c5853f9a64e9..0ea87d106c14e 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreStatsIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreStatsIT.java @@ -9,18 +9,21 @@ package org.opensearch.remotestore; import org.opensearch.action.admin.cluster.remotestore.stats.RemoteStoreStats; +import org.opensearch.action.admin.cluster.remotestore.stats.RemoteStoreStatsRequestBuilder; import org.opensearch.action.admin.cluster.remotestore.stats.RemoteStoreStatsResponse; import org.opensearch.action.index.IndexResponse; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.common.UUIDs; import org.opensearch.index.remote.RemoteRefreshSegmentTracker; +import org.opensearch.test.OpenSearchIntegTestCase; import java.util.Arrays; import java.util.List; import java.util.Locale; import java.util.stream.Collectors; +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 3) public class RemoteStoreStatsIT extends RemoteStoreBaseIntegTestCase { private static final String INDEX_NAME = "remote-store-test-idx-1"; @@ -29,7 +32,6 @@ public void testStatsResponseFromAllNodes() { // Step 1 - We create cluster, create an index, and then index documents into. We also do multiple refreshes/flushes // during this time frame. This ensures that the segment upload has started. - internalCluster().startDataOnlyNodes(3); if (randomBoolean()) { createIndex(INDEX_NAME, remoteTranslogIndexSettings(0)); } else { @@ -38,18 +40,7 @@ public void testStatsResponseFromAllNodes() { ensureYellowAndNoInitializingShards(INDEX_NAME); ensureGreen(INDEX_NAME); - // Indexing documents along with refreshes and flushes. - for (int i = 0; i < randomIntBetween(5, 10); i++) { - if (randomBoolean()) { - flush(INDEX_NAME); - } else { - refresh(INDEX_NAME); - } - int numberOfOperations = randomIntBetween(20, 50); - for (int j = 0; j < numberOfOperations; j++) { - indexSingleDoc(); - } - } + indexDocs(); // Step 2 - We find all the nodes that are present in the cluster. We make the remote store stats api call from // each of the node in the cluster and check that the response is coming as expected. @@ -66,23 +57,93 @@ public void testStatsResponseFromAllNodes() { .collect(Collectors.toList()); assertEquals(1, matches.size()); RemoteRefreshSegmentTracker.Stats stats = matches.get(0).getStats(); - assertEquals(0, stats.refreshTimeLagMs); - assertEquals(stats.localRefreshNumber, stats.remoteRefreshNumber); - assertTrue(stats.uploadBytesStarted > 0); - assertEquals(0, stats.uploadBytesFailed); - assertTrue(stats.uploadBytesSucceeded > 0); - assertTrue(stats.totalUploadsStarted > 0); - assertEquals(0, stats.totalUploadsFailed); - assertTrue(stats.totalUploadsSucceeded > 0); - assertEquals(0, stats.rejectionCount); - assertEquals(0, stats.consecutiveFailuresCount); - assertEquals(0, stats.bytesLag); - assertTrue(stats.uploadBytesMovingAverage > 0); - assertTrue(stats.uploadBytesPerSecMovingAverage > 0); - assertTrue(stats.uploadTimeMovingAverage > 0); + assertResponseStats(stats); + } + } + + public void testStatsResponseAllShards() { + + // Step 1 - We create cluster, create an index, and then index documents into. We also do multiple refreshes/flushes + // during this time frame. This ensures that the segment upload has started. + createIndex(INDEX_NAME, remoteTranslogIndexSettings(0, 3)); + ensureYellowAndNoInitializingShards(INDEX_NAME); + ensureGreen(INDEX_NAME); + + indexDocs(); + + // Step 2 - We find all the nodes that are present in the cluster. We make the remote store stats api call from + // each of the node in the cluster and check that the response is coming as expected. + ClusterState state = getClusterState(); + String node = state.nodes().getDataNodes().values().stream().map(DiscoveryNode::getName).findFirst().get(); + RemoteStoreStatsRequestBuilder remoteStoreStatsRequestBuilder = client(node).admin() + .cluster() + .prepareRemoteStoreStats(INDEX_NAME, null); + RemoteStoreStatsResponse response = remoteStoreStatsRequestBuilder.get(); + assertTrue(response.getSuccessfulShards() == 3); + assertTrue(response.getShards() != null && response.getShards().length == 3); + RemoteRefreshSegmentTracker.Stats stats = response.getShards()[0].getStats(); + assertResponseStats(stats); + } + + public void testStatsResponseFromLocalNode() { + + // Step 1 - We create cluster, create an index, and then index documents into. We also do multiple refreshes/flushes + // during this time frame. This ensures that the segment upload has started. + createIndex(INDEX_NAME, remoteTranslogIndexSettings(0, 3)); + ensureYellowAndNoInitializingShards(INDEX_NAME); + ensureGreen(INDEX_NAME); + + indexDocs(); + + // Step 2 - We find a data node in the cluster. We make the remote store stats api call from + // each of the data node in the cluster and check that only local shards are returned. + ClusterState state = getClusterState(); + List nodes = state.nodes().getDataNodes().values().stream().map(DiscoveryNode::getName).collect(Collectors.toList()); + for (String node : nodes) { + RemoteStoreStatsRequestBuilder remoteStoreStatsRequestBuilder = client(node).admin() + .cluster() + .prepareRemoteStoreStats(INDEX_NAME, null); + remoteStoreStatsRequestBuilder.setLocal(true); + RemoteStoreStatsResponse response = remoteStoreStatsRequestBuilder.get(); + assertTrue(response.getSuccessfulShards() == 1); + assertTrue(response.getShards() != null && response.getShards().length == 1); + RemoteRefreshSegmentTracker.Stats stats = response.getShards()[0].getStats(); + assertResponseStats(stats); + } + } + + private void indexDocs() { + // Indexing documents along with refreshes and flushes. + for (int i = 0; i < randomIntBetween(5, 10); i++) { + if (randomBoolean()) { + flush(INDEX_NAME); + } else { + refresh(INDEX_NAME); + } + int numberOfOperations = randomIntBetween(20, 50); + for (int j = 0; j < numberOfOperations; j++) { + indexSingleDoc(); + } } } + private void assertResponseStats(RemoteRefreshSegmentTracker.Stats stats) { + assertEquals(0, stats.refreshTimeLagMs); + assertEquals(stats.localRefreshNumber, stats.remoteRefreshNumber); + assertTrue(stats.uploadBytesStarted > 0); + assertEquals(0, stats.uploadBytesFailed); + assertTrue(stats.uploadBytesSucceeded > 0); + assertTrue(stats.totalUploadsStarted > 0); + assertEquals(0, stats.totalUploadsFailed); + assertTrue(stats.totalUploadsSucceeded > 0); + assertEquals(0, stats.rejectionCount); + assertEquals(0, stats.consecutiveFailuresCount); + assertEquals(0, stats.bytesLag); + assertTrue(stats.uploadBytesMovingAverage > 0); + assertTrue(stats.uploadBytesPerSecMovingAverage > 0); + assertTrue(stats.uploadTimeMovingAverage > 0); + } + private IndexResponse indexSingleDoc() { return client().prepareIndex(INDEX_NAME) .setId(UUIDs.randomBase64UUID()) From 45a934d38ad1c7350010a4537322d03bbea9826f Mon Sep 17 00:00:00 2001 From: Nick Knize Date: Mon, 26 Jun 2023 19:07:43 -0500 Subject: [PATCH 070/109] [Upgrade] Lucene 9.7.0 release (#8272) * [Upgrade] Lucene 9.7.0 release Upgrades to the official 9.7.0 release of lucene Signed-off-by: Nicholas Walter Knize * update changelog Signed-off-by: Nicholas Walter Knize --------- Signed-off-by: Nicholas Walter Knize --- CHANGELOG.md | 3 ++- buildSrc/version.properties | 2 +- libs/core/licenses/lucene-core-9.7.0-snapshot-204acc3.jar.sha1 | 1 - libs/core/licenses/lucene-core-9.7.0.jar.sha1 | 1 + .../lucene-expressions-9.7.0-snapshot-204acc3.jar.sha1 | 1 - .../lang-expression/licenses/lucene-expressions-9.7.0.jar.sha1 | 1 + .../lucene-analysis-icu-9.7.0-snapshot-204acc3.jar.sha1 | 1 - .../analysis-icu/licenses/lucene-analysis-icu-9.7.0.jar.sha1 | 1 + .../lucene-analysis-kuromoji-9.7.0-snapshot-204acc3.jar.sha1 | 1 - .../licenses/lucene-analysis-kuromoji-9.7.0.jar.sha1 | 1 + .../lucene-analysis-nori-9.7.0-snapshot-204acc3.jar.sha1 | 1 - .../analysis-nori/licenses/lucene-analysis-nori-9.7.0.jar.sha1 | 1 + .../lucene-analysis-phonetic-9.7.0-snapshot-204acc3.jar.sha1 | 1 - .../licenses/lucene-analysis-phonetic-9.7.0.jar.sha1 | 1 + .../lucene-analysis-smartcn-9.7.0-snapshot-204acc3.jar.sha1 | 1 - .../licenses/lucene-analysis-smartcn-9.7.0.jar.sha1 | 1 + .../lucene-analysis-stempel-9.7.0-snapshot-204acc3.jar.sha1 | 1 - .../licenses/lucene-analysis-stempel-9.7.0.jar.sha1 | 1 + .../lucene-analysis-morfologik-9.7.0-snapshot-204acc3.jar.sha1 | 1 - .../licenses/lucene-analysis-morfologik-9.7.0.jar.sha1 | 1 + .../lucene-analysis-common-9.7.0-snapshot-204acc3.jar.sha1 | 1 - server/licenses/lucene-analysis-common-9.7.0.jar.sha1 | 1 + .../lucene-backward-codecs-9.7.0-snapshot-204acc3.jar.sha1 | 1 - server/licenses/lucene-backward-codecs-9.7.0.jar.sha1 | 1 + server/licenses/lucene-core-9.7.0-snapshot-204acc3.jar.sha1 | 1 - server/licenses/lucene-core-9.7.0.jar.sha1 | 1 + .../licenses/lucene-grouping-9.7.0-snapshot-204acc3.jar.sha1 | 1 - server/licenses/lucene-grouping-9.7.0.jar.sha1 | 1 + .../lucene-highlighter-9.7.0-snapshot-204acc3.jar.sha1 | 1 - server/licenses/lucene-highlighter-9.7.0.jar.sha1 | 1 + server/licenses/lucene-join-9.7.0-snapshot-204acc3.jar.sha1 | 1 - server/licenses/lucene-join-9.7.0.jar.sha1 | 1 + server/licenses/lucene-memory-9.7.0-snapshot-204acc3.jar.sha1 | 1 - server/licenses/lucene-memory-9.7.0.jar.sha1 | 1 + server/licenses/lucene-misc-9.7.0-snapshot-204acc3.jar.sha1 | 1 - server/licenses/lucene-misc-9.7.0.jar.sha1 | 1 + server/licenses/lucene-queries-9.7.0-snapshot-204acc3.jar.sha1 | 1 - server/licenses/lucene-queries-9.7.0.jar.sha1 | 1 + .../lucene-queryparser-9.7.0-snapshot-204acc3.jar.sha1 | 1 - server/licenses/lucene-queryparser-9.7.0.jar.sha1 | 1 + server/licenses/lucene-sandbox-9.7.0-snapshot-204acc3.jar.sha1 | 1 - server/licenses/lucene-sandbox-9.7.0.jar.sha1 | 1 + .../lucene-spatial-extras-9.7.0-snapshot-204acc3.jar.sha1 | 1 - server/licenses/lucene-spatial-extras-9.7.0.jar.sha1 | 1 + .../licenses/lucene-spatial3d-9.7.0-snapshot-204acc3.jar.sha1 | 1 - server/licenses/lucene-spatial3d-9.7.0.jar.sha1 | 1 + server/licenses/lucene-suggest-9.7.0-snapshot-204acc3.jar.sha1 | 1 - server/licenses/lucene-suggest-9.7.0.jar.sha1 | 1 + .../src/main/java/org/opensearch/index/search/MatchQuery.java | 1 - .../main/java/org/opensearch/index/search/MultiMatchQuery.java | 1 - 50 files changed, 26 insertions(+), 27 deletions(-) delete mode 100644 libs/core/licenses/lucene-core-9.7.0-snapshot-204acc3.jar.sha1 create mode 100644 libs/core/licenses/lucene-core-9.7.0.jar.sha1 delete mode 100644 modules/lang-expression/licenses/lucene-expressions-9.7.0-snapshot-204acc3.jar.sha1 create mode 100644 modules/lang-expression/licenses/lucene-expressions-9.7.0.jar.sha1 delete mode 100644 plugins/analysis-icu/licenses/lucene-analysis-icu-9.7.0-snapshot-204acc3.jar.sha1 create mode 100644 plugins/analysis-icu/licenses/lucene-analysis-icu-9.7.0.jar.sha1 delete mode 100644 plugins/analysis-kuromoji/licenses/lucene-analysis-kuromoji-9.7.0-snapshot-204acc3.jar.sha1 create mode 100644 plugins/analysis-kuromoji/licenses/lucene-analysis-kuromoji-9.7.0.jar.sha1 delete mode 100644 plugins/analysis-nori/licenses/lucene-analysis-nori-9.7.0-snapshot-204acc3.jar.sha1 create mode 100644 plugins/analysis-nori/licenses/lucene-analysis-nori-9.7.0.jar.sha1 delete mode 100644 plugins/analysis-phonetic/licenses/lucene-analysis-phonetic-9.7.0-snapshot-204acc3.jar.sha1 create mode 100644 plugins/analysis-phonetic/licenses/lucene-analysis-phonetic-9.7.0.jar.sha1 delete mode 100644 plugins/analysis-smartcn/licenses/lucene-analysis-smartcn-9.7.0-snapshot-204acc3.jar.sha1 create mode 100644 plugins/analysis-smartcn/licenses/lucene-analysis-smartcn-9.7.0.jar.sha1 delete mode 100644 plugins/analysis-stempel/licenses/lucene-analysis-stempel-9.7.0-snapshot-204acc3.jar.sha1 create mode 100644 plugins/analysis-stempel/licenses/lucene-analysis-stempel-9.7.0.jar.sha1 delete mode 100644 plugins/analysis-ukrainian/licenses/lucene-analysis-morfologik-9.7.0-snapshot-204acc3.jar.sha1 create mode 100644 plugins/analysis-ukrainian/licenses/lucene-analysis-morfologik-9.7.0.jar.sha1 delete mode 100644 server/licenses/lucene-analysis-common-9.7.0-snapshot-204acc3.jar.sha1 create mode 100644 server/licenses/lucene-analysis-common-9.7.0.jar.sha1 delete mode 100644 server/licenses/lucene-backward-codecs-9.7.0-snapshot-204acc3.jar.sha1 create mode 100644 server/licenses/lucene-backward-codecs-9.7.0.jar.sha1 delete mode 100644 server/licenses/lucene-core-9.7.0-snapshot-204acc3.jar.sha1 create mode 100644 server/licenses/lucene-core-9.7.0.jar.sha1 delete mode 100644 server/licenses/lucene-grouping-9.7.0-snapshot-204acc3.jar.sha1 create mode 100644 server/licenses/lucene-grouping-9.7.0.jar.sha1 delete mode 100644 server/licenses/lucene-highlighter-9.7.0-snapshot-204acc3.jar.sha1 create mode 100644 server/licenses/lucene-highlighter-9.7.0.jar.sha1 delete mode 100644 server/licenses/lucene-join-9.7.0-snapshot-204acc3.jar.sha1 create mode 100644 server/licenses/lucene-join-9.7.0.jar.sha1 delete mode 100644 server/licenses/lucene-memory-9.7.0-snapshot-204acc3.jar.sha1 create mode 100644 server/licenses/lucene-memory-9.7.0.jar.sha1 delete mode 100644 server/licenses/lucene-misc-9.7.0-snapshot-204acc3.jar.sha1 create mode 100644 server/licenses/lucene-misc-9.7.0.jar.sha1 delete mode 100644 server/licenses/lucene-queries-9.7.0-snapshot-204acc3.jar.sha1 create mode 100644 server/licenses/lucene-queries-9.7.0.jar.sha1 delete mode 100644 server/licenses/lucene-queryparser-9.7.0-snapshot-204acc3.jar.sha1 create mode 100644 server/licenses/lucene-queryparser-9.7.0.jar.sha1 delete mode 100644 server/licenses/lucene-sandbox-9.7.0-snapshot-204acc3.jar.sha1 create mode 100644 server/licenses/lucene-sandbox-9.7.0.jar.sha1 delete mode 100644 server/licenses/lucene-spatial-extras-9.7.0-snapshot-204acc3.jar.sha1 create mode 100644 server/licenses/lucene-spatial-extras-9.7.0.jar.sha1 delete mode 100644 server/licenses/lucene-spatial3d-9.7.0-snapshot-204acc3.jar.sha1 create mode 100644 server/licenses/lucene-spatial3d-9.7.0.jar.sha1 delete mode 100644 server/licenses/lucene-suggest-9.7.0-snapshot-204acc3.jar.sha1 create mode 100644 server/licenses/lucene-suggest-9.7.0.jar.sha1 diff --git a/CHANGELOG.md b/CHANGELOG.md index f4db81ecf16e5..f5bc63794e46d 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -112,6 +112,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `netty` from 4.1.93.Final to 4.1.94.Final ([#8191](https://github.com/opensearch-project/OpenSearch/pull/8191)) - Bump `org.apache.hadoop:hadoop-minicluster` from 3.3.5 to 3.3.6 (#8257) - Bump `io.projectreactor.netty:reactor-netty-http` from 1.1.7 to 1.1.8 (#8256) +- [Upgrade] Lucene 9.7.0 release (#8272) ### Changed - Replace jboss-annotations-api_1.2_spec with jakarta.annotation-api ([#7836](https://github.com/opensearch-project/OpenSearch/pull/7836)) @@ -141,4 +142,4 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), ### Security [Unreleased 3.0]: https://github.com/opensearch-project/OpenSearch/compare/2.x...HEAD -[Unreleased 2.x]: https://github.com/opensearch-project/OpenSearch/compare/2.8...2.x \ No newline at end of file +[Unreleased 2.x]: https://github.com/opensearch-project/OpenSearch/compare/2.8...2.x diff --git a/buildSrc/version.properties b/buildSrc/version.properties index d3dbae38c2615..377421703b892 100644 --- a/buildSrc/version.properties +++ b/buildSrc/version.properties @@ -1,5 +1,5 @@ opensearch = 3.0.0 -lucene = 9.7.0-snapshot-204acc3 +lucene = 9.7.0 bundled_jdk_vendor = adoptium bundled_jdk = 20.0.1+9 diff --git a/libs/core/licenses/lucene-core-9.7.0-snapshot-204acc3.jar.sha1 b/libs/core/licenses/lucene-core-9.7.0-snapshot-204acc3.jar.sha1 deleted file mode 100644 index 2afe66c03cf22..0000000000000 --- a/libs/core/licenses/lucene-core-9.7.0-snapshot-204acc3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -3aa698cf90f074cbf24acfd7feaaad84c5a6f829 \ No newline at end of file diff --git a/libs/core/licenses/lucene-core-9.7.0.jar.sha1 b/libs/core/licenses/lucene-core-9.7.0.jar.sha1 new file mode 100644 index 0000000000000..2b0f77275c0ab --- /dev/null +++ b/libs/core/licenses/lucene-core-9.7.0.jar.sha1 @@ -0,0 +1 @@ +ad391210ffd806931334be9670a35af00c56f959 \ No newline at end of file diff --git a/modules/lang-expression/licenses/lucene-expressions-9.7.0-snapshot-204acc3.jar.sha1 b/modules/lang-expression/licenses/lucene-expressions-9.7.0-snapshot-204acc3.jar.sha1 deleted file mode 100644 index f70c327ab5f6b..0000000000000 --- a/modules/lang-expression/licenses/lucene-expressions-9.7.0-snapshot-204acc3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -c31671978777d97c026b13decfbef1d7eeed4410 \ No newline at end of file diff --git a/modules/lang-expression/licenses/lucene-expressions-9.7.0.jar.sha1 b/modules/lang-expression/licenses/lucene-expressions-9.7.0.jar.sha1 new file mode 100644 index 0000000000000..ecf696b4b3b83 --- /dev/null +++ b/modules/lang-expression/licenses/lucene-expressions-9.7.0.jar.sha1 @@ -0,0 +1 @@ +297e1cfade4ef71466cc9d4f361d81807c8dc4c8 \ No newline at end of file diff --git a/plugins/analysis-icu/licenses/lucene-analysis-icu-9.7.0-snapshot-204acc3.jar.sha1 b/plugins/analysis-icu/licenses/lucene-analysis-icu-9.7.0-snapshot-204acc3.jar.sha1 deleted file mode 100644 index b5c59c8f5ed8c..0000000000000 --- a/plugins/analysis-icu/licenses/lucene-analysis-icu-9.7.0-snapshot-204acc3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -e31f7c161ad2b7652b1ac9100b4fa67d52be86eb \ No newline at end of file diff --git a/plugins/analysis-icu/licenses/lucene-analysis-icu-9.7.0.jar.sha1 b/plugins/analysis-icu/licenses/lucene-analysis-icu-9.7.0.jar.sha1 new file mode 100644 index 0000000000000..0ed030926ab93 --- /dev/null +++ b/plugins/analysis-icu/licenses/lucene-analysis-icu-9.7.0.jar.sha1 @@ -0,0 +1 @@ +94293b169fb8572f440a5a4a523320ecf9778ffe \ No newline at end of file diff --git a/plugins/analysis-kuromoji/licenses/lucene-analysis-kuromoji-9.7.0-snapshot-204acc3.jar.sha1 b/plugins/analysis-kuromoji/licenses/lucene-analysis-kuromoji-9.7.0-snapshot-204acc3.jar.sha1 deleted file mode 100644 index e3f1d7afdc72a..0000000000000 --- a/plugins/analysis-kuromoji/licenses/lucene-analysis-kuromoji-9.7.0-snapshot-204acc3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -ed13aaa5843c76ba8bb9fcd1cac6de8855ea7f1c \ No newline at end of file diff --git a/plugins/analysis-kuromoji/licenses/lucene-analysis-kuromoji-9.7.0.jar.sha1 b/plugins/analysis-kuromoji/licenses/lucene-analysis-kuromoji-9.7.0.jar.sha1 new file mode 100644 index 0000000000000..ddd67276606a5 --- /dev/null +++ b/plugins/analysis-kuromoji/licenses/lucene-analysis-kuromoji-9.7.0.jar.sha1 @@ -0,0 +1 @@ +2df800a38b64867b8dcd61fc2cd986114e4a80cb \ No newline at end of file diff --git a/plugins/analysis-nori/licenses/lucene-analysis-nori-9.7.0-snapshot-204acc3.jar.sha1 b/plugins/analysis-nori/licenses/lucene-analysis-nori-9.7.0-snapshot-204acc3.jar.sha1 deleted file mode 100644 index 12210d41b4e6f..0000000000000 --- a/plugins/analysis-nori/licenses/lucene-analysis-nori-9.7.0-snapshot-204acc3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -f8673a2b86d3868e9ef2745d1d3fa9ad5a63e84f \ No newline at end of file diff --git a/plugins/analysis-nori/licenses/lucene-analysis-nori-9.7.0.jar.sha1 b/plugins/analysis-nori/licenses/lucene-analysis-nori-9.7.0.jar.sha1 new file mode 100644 index 0000000000000..0cd68af98e724 --- /dev/null +++ b/plugins/analysis-nori/licenses/lucene-analysis-nori-9.7.0.jar.sha1 @@ -0,0 +1 @@ +a01e8153f34d72e8c8c0180c1dea5b10f677dd3a \ No newline at end of file diff --git a/plugins/analysis-phonetic/licenses/lucene-analysis-phonetic-9.7.0-snapshot-204acc3.jar.sha1 b/plugins/analysis-phonetic/licenses/lucene-analysis-phonetic-9.7.0-snapshot-204acc3.jar.sha1 deleted file mode 100644 index 307d90ca834b6..0000000000000 --- a/plugins/analysis-phonetic/licenses/lucene-analysis-phonetic-9.7.0-snapshot-204acc3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -f7a585d5f62033f19a5ec79b66a74cc24c9e9d10 \ No newline at end of file diff --git a/plugins/analysis-phonetic/licenses/lucene-analysis-phonetic-9.7.0.jar.sha1 b/plugins/analysis-phonetic/licenses/lucene-analysis-phonetic-9.7.0.jar.sha1 new file mode 100644 index 0000000000000..c7b4d2dc6da75 --- /dev/null +++ b/plugins/analysis-phonetic/licenses/lucene-analysis-phonetic-9.7.0.jar.sha1 @@ -0,0 +1 @@ +b7d47d54683b0b1e09b271c32d1b7d3eb1990f49 \ No newline at end of file diff --git a/plugins/analysis-smartcn/licenses/lucene-analysis-smartcn-9.7.0-snapshot-204acc3.jar.sha1 b/plugins/analysis-smartcn/licenses/lucene-analysis-smartcn-9.7.0-snapshot-204acc3.jar.sha1 deleted file mode 100644 index b4e595a07ffda..0000000000000 --- a/plugins/analysis-smartcn/licenses/lucene-analysis-smartcn-9.7.0-snapshot-204acc3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -40ba622c3aa91a0a8f2747617fef8f59bf212345 \ No newline at end of file diff --git a/plugins/analysis-smartcn/licenses/lucene-analysis-smartcn-9.7.0.jar.sha1 b/plugins/analysis-smartcn/licenses/lucene-analysis-smartcn-9.7.0.jar.sha1 new file mode 100644 index 0000000000000..8df7245044171 --- /dev/null +++ b/plugins/analysis-smartcn/licenses/lucene-analysis-smartcn-9.7.0.jar.sha1 @@ -0,0 +1 @@ +5e68b9816e6cff8ee15f5b350cf2ffa54f9828b7 \ No newline at end of file diff --git a/plugins/analysis-stempel/licenses/lucene-analysis-stempel-9.7.0-snapshot-204acc3.jar.sha1 b/plugins/analysis-stempel/licenses/lucene-analysis-stempel-9.7.0-snapshot-204acc3.jar.sha1 deleted file mode 100644 index 9b6dcc21465fa..0000000000000 --- a/plugins/analysis-stempel/licenses/lucene-analysis-stempel-9.7.0-snapshot-204acc3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -e08432a3db0bde29f425812896bd695c5477b05f \ No newline at end of file diff --git a/plugins/analysis-stempel/licenses/lucene-analysis-stempel-9.7.0.jar.sha1 b/plugins/analysis-stempel/licenses/lucene-analysis-stempel-9.7.0.jar.sha1 new file mode 100644 index 0000000000000..974e4202f5ffb --- /dev/null +++ b/plugins/analysis-stempel/licenses/lucene-analysis-stempel-9.7.0.jar.sha1 @@ -0,0 +1 @@ +d23b1f05b471e05d0d6068b3ece7c8c65672eae7 \ No newline at end of file diff --git a/plugins/analysis-ukrainian/licenses/lucene-analysis-morfologik-9.7.0-snapshot-204acc3.jar.sha1 b/plugins/analysis-ukrainian/licenses/lucene-analysis-morfologik-9.7.0-snapshot-204acc3.jar.sha1 deleted file mode 100644 index b70b4e8d6c35b..0000000000000 --- a/plugins/analysis-ukrainian/licenses/lucene-analysis-morfologik-9.7.0-snapshot-204acc3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -a44008e83286060f0265a32f9a9b586e86ec03b1 \ No newline at end of file diff --git a/plugins/analysis-ukrainian/licenses/lucene-analysis-morfologik-9.7.0.jar.sha1 b/plugins/analysis-ukrainian/licenses/lucene-analysis-morfologik-9.7.0.jar.sha1 new file mode 100644 index 0000000000000..dce408a7d40ef --- /dev/null +++ b/plugins/analysis-ukrainian/licenses/lucene-analysis-morfologik-9.7.0.jar.sha1 @@ -0,0 +1 @@ +dfb4313f3c68d337310522840d7144c1605d084a \ No newline at end of file diff --git a/server/licenses/lucene-analysis-common-9.7.0-snapshot-204acc3.jar.sha1 b/server/licenses/lucene-analysis-common-9.7.0-snapshot-204acc3.jar.sha1 deleted file mode 100644 index a770070d97c2d..0000000000000 --- a/server/licenses/lucene-analysis-common-9.7.0-snapshot-204acc3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -5aa5989b931c68eee90b22bca3f1f280a7d5c1ee \ No newline at end of file diff --git a/server/licenses/lucene-analysis-common-9.7.0.jar.sha1 b/server/licenses/lucene-analysis-common-9.7.0.jar.sha1 new file mode 100644 index 0000000000000..45d8f459573b1 --- /dev/null +++ b/server/licenses/lucene-analysis-common-9.7.0.jar.sha1 @@ -0,0 +1 @@ +27ba6caaa4587a982cd451f7217b5a982bcfc44a \ No newline at end of file diff --git a/server/licenses/lucene-backward-codecs-9.7.0-snapshot-204acc3.jar.sha1 b/server/licenses/lucene-backward-codecs-9.7.0-snapshot-204acc3.jar.sha1 deleted file mode 100644 index c35bc03c19097..0000000000000 --- a/server/licenses/lucene-backward-codecs-9.7.0-snapshot-204acc3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -2326fb4762cd14eff6bea770d5c85a848e7a2482 \ No newline at end of file diff --git a/server/licenses/lucene-backward-codecs-9.7.0.jar.sha1 b/server/licenses/lucene-backward-codecs-9.7.0.jar.sha1 new file mode 100644 index 0000000000000..3981ea4fa226e --- /dev/null +++ b/server/licenses/lucene-backward-codecs-9.7.0.jar.sha1 @@ -0,0 +1 @@ +6389463bfbfcf902c8d31d12e9513a6818ac9d5e \ No newline at end of file diff --git a/server/licenses/lucene-core-9.7.0-snapshot-204acc3.jar.sha1 b/server/licenses/lucene-core-9.7.0-snapshot-204acc3.jar.sha1 deleted file mode 100644 index 2afe66c03cf22..0000000000000 --- a/server/licenses/lucene-core-9.7.0-snapshot-204acc3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -3aa698cf90f074cbf24acfd7feaaad84c5a6f829 \ No newline at end of file diff --git a/server/licenses/lucene-core-9.7.0.jar.sha1 b/server/licenses/lucene-core-9.7.0.jar.sha1 new file mode 100644 index 0000000000000..2b0f77275c0ab --- /dev/null +++ b/server/licenses/lucene-core-9.7.0.jar.sha1 @@ -0,0 +1 @@ +ad391210ffd806931334be9670a35af00c56f959 \ No newline at end of file diff --git a/server/licenses/lucene-grouping-9.7.0-snapshot-204acc3.jar.sha1 b/server/licenses/lucene-grouping-9.7.0-snapshot-204acc3.jar.sha1 deleted file mode 100644 index c7aaff3a0b184..0000000000000 --- a/server/licenses/lucene-grouping-9.7.0-snapshot-204acc3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -b3251c58b21c5c205c63bbe618c0d3c1393908e1 \ No newline at end of file diff --git a/server/licenses/lucene-grouping-9.7.0.jar.sha1 b/server/licenses/lucene-grouping-9.7.0.jar.sha1 new file mode 100644 index 0000000000000..90acbf6dcee8d --- /dev/null +++ b/server/licenses/lucene-grouping-9.7.0.jar.sha1 @@ -0,0 +1 @@ +8e6f0c229f4861be641047c33b05067176e4279c \ No newline at end of file diff --git a/server/licenses/lucene-highlighter-9.7.0-snapshot-204acc3.jar.sha1 b/server/licenses/lucene-highlighter-9.7.0-snapshot-204acc3.jar.sha1 deleted file mode 100644 index 6f9bfe7604f7f..0000000000000 --- a/server/licenses/lucene-highlighter-9.7.0-snapshot-204acc3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -2313a0755a5774eb5dfcdc116dc15bd9367c8cc1 \ No newline at end of file diff --git a/server/licenses/lucene-highlighter-9.7.0.jar.sha1 b/server/licenses/lucene-highlighter-9.7.0.jar.sha1 new file mode 100644 index 0000000000000..bfcca0bc6cb5b --- /dev/null +++ b/server/licenses/lucene-highlighter-9.7.0.jar.sha1 @@ -0,0 +1 @@ +facb7c7ee0f75ed457a2d98f10d6430e25a53691 \ No newline at end of file diff --git a/server/licenses/lucene-join-9.7.0-snapshot-204acc3.jar.sha1 b/server/licenses/lucene-join-9.7.0-snapshot-204acc3.jar.sha1 deleted file mode 100644 index 00b5c33be3568..0000000000000 --- a/server/licenses/lucene-join-9.7.0-snapshot-204acc3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -25fddcaf47d5614f48dc264f79151eb87990abd9 \ No newline at end of file diff --git a/server/licenses/lucene-join-9.7.0.jar.sha1 b/server/licenses/lucene-join-9.7.0.jar.sha1 new file mode 100644 index 0000000000000..0dab3a7ddc41a --- /dev/null +++ b/server/licenses/lucene-join-9.7.0.jar.sha1 @@ -0,0 +1 @@ +d041bdc0947a14223cf68357407ee18b21027587 \ No newline at end of file diff --git a/server/licenses/lucene-memory-9.7.0-snapshot-204acc3.jar.sha1 b/server/licenses/lucene-memory-9.7.0-snapshot-204acc3.jar.sha1 deleted file mode 100644 index cf7eced4686aa..0000000000000 --- a/server/licenses/lucene-memory-9.7.0-snapshot-204acc3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -7c2aad4c1baa818a94bd787615e70824e6ae3a9d \ No newline at end of file diff --git a/server/licenses/lucene-memory-9.7.0.jar.sha1 b/server/licenses/lucene-memory-9.7.0.jar.sha1 new file mode 100644 index 0000000000000..357a9c4b2ea26 --- /dev/null +++ b/server/licenses/lucene-memory-9.7.0.jar.sha1 @@ -0,0 +1 @@ +0fade51ee353e15ddbbc45262aafe6f99ed020f1 \ No newline at end of file diff --git a/server/licenses/lucene-misc-9.7.0-snapshot-204acc3.jar.sha1 b/server/licenses/lucene-misc-9.7.0-snapshot-204acc3.jar.sha1 deleted file mode 100644 index b5ac7eb1f988b..0000000000000 --- a/server/licenses/lucene-misc-9.7.0-snapshot-204acc3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -56211acb2a31b1967568eba9953db502a53f9010 \ No newline at end of file diff --git a/server/licenses/lucene-misc-9.7.0.jar.sha1 b/server/licenses/lucene-misc-9.7.0.jar.sha1 new file mode 100644 index 0000000000000..da5e1921626b2 --- /dev/null +++ b/server/licenses/lucene-misc-9.7.0.jar.sha1 @@ -0,0 +1 @@ +7fcf451e2376526c3a027958812866cc5b0ff13f \ No newline at end of file diff --git a/server/licenses/lucene-queries-9.7.0-snapshot-204acc3.jar.sha1 b/server/licenses/lucene-queries-9.7.0-snapshot-204acc3.jar.sha1 deleted file mode 100644 index 2aa628ebc9573..0000000000000 --- a/server/licenses/lucene-queries-9.7.0-snapshot-204acc3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -c45b1847150e2e2e6302177b41faf95df2ca4fbf \ No newline at end of file diff --git a/server/licenses/lucene-queries-9.7.0.jar.sha1 b/server/licenses/lucene-queries-9.7.0.jar.sha1 new file mode 100644 index 0000000000000..fa82e95a7e19f --- /dev/null +++ b/server/licenses/lucene-queries-9.7.0.jar.sha1 @@ -0,0 +1 @@ +126989d4622419aa06fcbf3a342e859cab8c8799 \ No newline at end of file diff --git a/server/licenses/lucene-queryparser-9.7.0-snapshot-204acc3.jar.sha1 b/server/licenses/lucene-queryparser-9.7.0-snapshot-204acc3.jar.sha1 deleted file mode 100644 index 5e7b5d9da9c0d..0000000000000 --- a/server/licenses/lucene-queryparser-9.7.0-snapshot-204acc3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -d40fe6400f21564ccb50b6716ec42d8dcd526b9d \ No newline at end of file diff --git a/server/licenses/lucene-queryparser-9.7.0.jar.sha1 b/server/licenses/lucene-queryparser-9.7.0.jar.sha1 new file mode 100644 index 0000000000000..438db0aea66e1 --- /dev/null +++ b/server/licenses/lucene-queryparser-9.7.0.jar.sha1 @@ -0,0 +1 @@ +6e77bde908ff698354e4a2149e6dd4658b56d7b0 \ No newline at end of file diff --git a/server/licenses/lucene-sandbox-9.7.0-snapshot-204acc3.jar.sha1 b/server/licenses/lucene-sandbox-9.7.0-snapshot-204acc3.jar.sha1 deleted file mode 100644 index e45e9b929836b..0000000000000 --- a/server/licenses/lucene-sandbox-9.7.0-snapshot-204acc3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -e9347773d0b269768c58dd0b438bdc4b450f9185 \ No newline at end of file diff --git a/server/licenses/lucene-sandbox-9.7.0.jar.sha1 b/server/licenses/lucene-sandbox-9.7.0.jar.sha1 new file mode 100644 index 0000000000000..38b0b1cccbc29 --- /dev/null +++ b/server/licenses/lucene-sandbox-9.7.0.jar.sha1 @@ -0,0 +1 @@ +9f3e8e1947f2f1c5784132444af51a060ff0b4bf \ No newline at end of file diff --git a/server/licenses/lucene-spatial-extras-9.7.0-snapshot-204acc3.jar.sha1 b/server/licenses/lucene-spatial-extras-9.7.0-snapshot-204acc3.jar.sha1 deleted file mode 100644 index a697cb9fa2dff..0000000000000 --- a/server/licenses/lucene-spatial-extras-9.7.0-snapshot-204acc3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -7055502c9a77b9a93d1b96c99397fd765dd7891f \ No newline at end of file diff --git a/server/licenses/lucene-spatial-extras-9.7.0.jar.sha1 b/server/licenses/lucene-spatial-extras-9.7.0.jar.sha1 new file mode 100644 index 0000000000000..48679df469fd1 --- /dev/null +++ b/server/licenses/lucene-spatial-extras-9.7.0.jar.sha1 @@ -0,0 +1 @@ +01b0bc7a407d8c35a70a1adf7966bb3e7caae928 \ No newline at end of file diff --git a/server/licenses/lucene-spatial3d-9.7.0-snapshot-204acc3.jar.sha1 b/server/licenses/lucene-spatial3d-9.7.0-snapshot-204acc3.jar.sha1 deleted file mode 100644 index 3b96e448add5b..0000000000000 --- a/server/licenses/lucene-spatial3d-9.7.0-snapshot-204acc3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -a2c5a7be5887dacf89c251936c3f3388ca20d28e \ No newline at end of file diff --git a/server/licenses/lucene-spatial3d-9.7.0.jar.sha1 b/server/licenses/lucene-spatial3d-9.7.0.jar.sha1 new file mode 100644 index 0000000000000..55d4d217fa6b9 --- /dev/null +++ b/server/licenses/lucene-spatial3d-9.7.0.jar.sha1 @@ -0,0 +1 @@ +7c6b1b6e0a70c9cd177371e648648c2f896742a2 \ No newline at end of file diff --git a/server/licenses/lucene-suggest-9.7.0-snapshot-204acc3.jar.sha1 b/server/licenses/lucene-suggest-9.7.0-snapshot-204acc3.jar.sha1 deleted file mode 100644 index 0460861900897..0000000000000 --- a/server/licenses/lucene-suggest-9.7.0-snapshot-204acc3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -1a076a8db84fa85634cc4894ee4fc78b5e69525c \ No newline at end of file diff --git a/server/licenses/lucene-suggest-9.7.0.jar.sha1 b/server/licenses/lucene-suggest-9.7.0.jar.sha1 new file mode 100644 index 0000000000000..d4d7e6cd6bed9 --- /dev/null +++ b/server/licenses/lucene-suggest-9.7.0.jar.sha1 @@ -0,0 +1 @@ +5c37fd9a5d71dc87fe1cd4c18ff295ec8cfac170 \ No newline at end of file diff --git a/server/src/main/java/org/opensearch/index/search/MatchQuery.java b/server/src/main/java/org/opensearch/index/search/MatchQuery.java index 2c8b091fef8ef..91a4b456bfa2a 100644 --- a/server/src/main/java/org/opensearch/index/search/MatchQuery.java +++ b/server/src/main/java/org/opensearch/index/search/MatchQuery.java @@ -56,7 +56,6 @@ import org.apache.lucene.search.Query; import org.apache.lucene.search.TermQuery; import org.apache.lucene.util.QueryBuilder; -import org.apache.lucene.util.TermAndBoost; import org.apache.lucene.util.graph.GraphTokenStreamFiniteStrings; import org.opensearch.OpenSearchException; import org.opensearch.common.io.stream.StreamInput; diff --git a/server/src/main/java/org/opensearch/index/search/MultiMatchQuery.java b/server/src/main/java/org/opensearch/index/search/MultiMatchQuery.java index 554d9714409b7..241f05af2c512 100644 --- a/server/src/main/java/org/opensearch/index/search/MultiMatchQuery.java +++ b/server/src/main/java/org/opensearch/index/search/MultiMatchQuery.java @@ -42,7 +42,6 @@ import org.apache.lucene.search.Query; import org.apache.lucene.search.TermQuery; import org.apache.lucene.util.BytesRef; -import org.apache.lucene.util.TermAndBoost; import org.opensearch.common.lucene.search.Queries; import org.opensearch.index.mapper.MappedFieldType; import org.opensearch.index.query.AbstractQueryBuilder; From 0c7ba945806893ab07c705324de890f21bd623cc Mon Sep 17 00:00:00 2001 From: Sachin Kale Date: Tue, 27 Jun 2023 08:04:23 +0530 Subject: [PATCH 071/109] Fix SegmentReplication flaky integ tests (#8134) Signed-off-by: Sachin Kale --- .../replication/SegmentReplicationIT.java | 91 ++++++++++--------- .../SegmentReplicationRemoteStoreIT.java | 4 +- .../opensearch/index/shard/IndexShard.java | 30 ++++-- .../RemoteStoreRefreshListenerTests.java | 1 - 4 files changed, 70 insertions(+), 56 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationIT.java index 0a593e6149ddd..ce5e0989b622f 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationIT.java @@ -20,6 +20,7 @@ import org.apache.lucene.index.StandardDirectoryReader; import org.apache.lucene.tests.util.TestUtil; import org.apache.lucene.util.BytesRef; +import org.junit.Before; import org.opensearch.action.ActionFuture; import org.opensearch.action.admin.indices.flush.FlushRequest; import org.opensearch.action.admin.indices.stats.IndicesStatsRequest; @@ -95,11 +96,16 @@ @OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) public class SegmentReplicationIT extends SegmentReplicationBaseIT { + @Before + private void setup() { + internalCluster().startClusterManagerOnlyNode(); + } + public void testPrimaryStopped_ReplicaPromoted() throws Exception { - final String primary = internalCluster().startNode(); + final String primary = internalCluster().startDataOnlyNode(); createIndex(INDEX_NAME); ensureYellowAndNoInitializingShards(INDEX_NAME); - final String replica = internalCluster().startNode(); + final String replica = internalCluster().startDataOnlyNode(); ensureGreen(INDEX_NAME); client().prepareIndex(INDEX_NAME).setId("1").setSource("foo", "bar").setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE).get(); @@ -125,7 +131,7 @@ public void testPrimaryStopped_ReplicaPromoted() throws Exception { assertHitCount(client(replica).prepareSearch(INDEX_NAME).setSize(0).setPreference("_only_local").get(), 3); // start another node, index another doc and replicate. - String nodeC = internalCluster().startNode(); + String nodeC = internalCluster().startDataOnlyNode(); ensureGreen(INDEX_NAME); client().prepareIndex(INDEX_NAME).setId("4").setSource("baz", "baz").get(); refresh(INDEX_NAME); @@ -134,10 +140,10 @@ public void testPrimaryStopped_ReplicaPromoted() throws Exception { } public void testRestartPrimary() throws Exception { - final String primary = internalCluster().startNode(); + final String primary = internalCluster().startDataOnlyNode(); createIndex(INDEX_NAME); ensureYellowAndNoInitializingShards(INDEX_NAME); - final String replica = internalCluster().startNode(); + final String replica = internalCluster().startDataOnlyNode(); ensureGreen(INDEX_NAME); assertEquals(getNodeContainingPrimaryShard().getName(), primary); @@ -160,10 +166,10 @@ public void testRestartPrimary() throws Exception { public void testCancelPrimaryAllocation() throws Exception { // this test cancels allocation on the primary - promoting the new replica and recreating the former primary as a replica. - final String primary = internalCluster().startNode(); + final String primary = internalCluster().startDataOnlyNode(); createIndex(INDEX_NAME); ensureYellowAndNoInitializingShards(INDEX_NAME); - final String replica = internalCluster().startNode(); + final String replica = internalCluster().startDataOnlyNode(); ensureGreen(INDEX_NAME); final int initialDocCount = 1; @@ -190,8 +196,8 @@ public void testCancelPrimaryAllocation() throws Exception { } public void testReplicationAfterPrimaryRefreshAndFlush() throws Exception { - final String nodeA = internalCluster().startNode(); - final String nodeB = internalCluster().startNode(); + final String nodeA = internalCluster().startDataOnlyNode(); + final String nodeB = internalCluster().startDataOnlyNode(); final Settings settings = Settings.builder() .put(indexSettings()) .put( @@ -233,8 +239,8 @@ public void testReplicationAfterPrimaryRefreshAndFlush() throws Exception { } public void testIndexReopenClose() throws Exception { - final String primary = internalCluster().startNode(); - final String replica = internalCluster().startNode(); + final String primary = internalCluster().startDataOnlyNode(); + final String replica = internalCluster().startDataOnlyNode(); createIndex(INDEX_NAME); ensureGreen(INDEX_NAME); @@ -274,8 +280,8 @@ public void testMultipleShards() throws Exception { .put(IndexModule.INDEX_QUERY_CACHE_ENABLED_SETTING.getKey(), false) .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) .build(); - final String nodeA = internalCluster().startNode(); - final String nodeB = internalCluster().startNode(); + final String nodeA = internalCluster().startDataOnlyNode(); + final String nodeB = internalCluster().startDataOnlyNode(); createIndex(INDEX_NAME, indexSettings); ensureGreen(INDEX_NAME); @@ -310,8 +316,8 @@ public void testMultipleShards() throws Exception { } public void testReplicationAfterForceMerge() throws Exception { - final String nodeA = internalCluster().startNode(); - final String nodeB = internalCluster().startNode(); + final String nodeA = internalCluster().startDataOnlyNode(); + final String nodeB = internalCluster().startDataOnlyNode(); createIndex(INDEX_NAME); ensureGreen(INDEX_NAME); @@ -351,14 +357,13 @@ public void testReplicationAfterForceMerge() throws Exception { * This test verifies that segment replication does not fail for closed indices */ public void testClosedIndices() { - internalCluster().startClusterManagerOnlyNode(); List nodes = new ArrayList<>(); // start 1st node so that it contains the primary - nodes.add(internalCluster().startNode()); + nodes.add(internalCluster().startDataOnlyNode()); createIndex(INDEX_NAME, super.indexSettings()); ensureYellowAndNoInitializingShards(INDEX_NAME); // start 2nd node so that it contains the replica - nodes.add(internalCluster().startNode()); + nodes.add(internalCluster().startDataOnlyNode()); ensureGreen(INDEX_NAME); logger.info("--> Close index"); @@ -373,8 +378,7 @@ public void testClosedIndices() { * @throws Exception when issue is encountered */ public void testNodeDropWithOngoingReplication() throws Exception { - internalCluster().startClusterManagerOnlyNode(); - final String primaryNode = internalCluster().startNode(); + final String primaryNode = internalCluster().startDataOnlyNode(); createIndex( INDEX_NAME, Settings.builder() @@ -385,7 +389,7 @@ public void testNodeDropWithOngoingReplication() throws Exception { .build() ); ensureYellow(INDEX_NAME); - final String replicaNode = internalCluster().startNode(); + final String replicaNode = internalCluster().startDataOnlyNode(); ensureGreen(INDEX_NAME); ClusterState state = client().admin().cluster().prepareState().execute().actionGet().getState(); // Get replica allocation id @@ -447,11 +451,11 @@ public void testNodeDropWithOngoingReplication() throws Exception { } public void testCancellation() throws Exception { - final String primaryNode = internalCluster().startNode(); + final String primaryNode = internalCluster().startDataOnlyNode(); createIndex(INDEX_NAME, Settings.builder().put(indexSettings()).put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1).build()); ensureYellow(INDEX_NAME); - final String replicaNode = internalCluster().startNode(); + final String replicaNode = internalCluster().startDataOnlyNode(); final SegmentReplicationSourceService segmentReplicationSourceService = internalCluster().getInstance( SegmentReplicationSourceService.class, @@ -506,7 +510,7 @@ public void testCancellation() throws Exception { } public void testStartReplicaAfterPrimaryIndexesDocs() throws Exception { - final String primaryNode = internalCluster().startNode(); + final String primaryNode = internalCluster().startDataOnlyNode(); createIndex(INDEX_NAME, Settings.builder().put(indexSettings()).put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0).build()); ensureGreen(INDEX_NAME); @@ -529,7 +533,7 @@ public void testStartReplicaAfterPrimaryIndexesDocs() throws Exception { .prepareUpdateSettings(INDEX_NAME) .setSettings(Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1)) ); - final String replicaNode = internalCluster().startNode(); + final String replicaNode = internalCluster().startDataOnlyNode(); ensureGreen(INDEX_NAME); assertHitCount(client(primaryNode).prepareSearch(INDEX_NAME).setSize(0).setPreference("_only_local").get(), 2); @@ -544,8 +548,8 @@ public void testStartReplicaAfterPrimaryIndexesDocs() throws Exception { } public void testDeleteOperations() throws Exception { - final String nodeA = internalCluster().startNode(); - final String nodeB = internalCluster().startNode(); + final String nodeA = internalCluster().startDataOnlyNode(); + final String nodeB = internalCluster().startDataOnlyNode(); createIndex(INDEX_NAME); ensureGreen(INDEX_NAME); @@ -591,9 +595,9 @@ public void testDeleteOperations() throws Exception { */ public void testReplicationPostDeleteAndForceMerge() throws Exception { assumeFalse("Skipping the test with Remote store as its flaky.", segmentReplicationWithRemoteEnabled()); - final String primary = internalCluster().startNode(); + final String primary = internalCluster().startDataOnlyNode(); createIndex(INDEX_NAME); - final String replica = internalCluster().startNode(); + final String replica = internalCluster().startDataOnlyNode(); ensureGreen(INDEX_NAME); final int initialDocCount = scaledRandomIntBetween(10, 200); for (int i = 0; i < initialDocCount; i++) { @@ -648,7 +652,6 @@ public void testReplicationPostDeleteAndForceMerge() throws Exception { } public void testUpdateOperations() throws Exception { - internalCluster().startClusterManagerOnlyNode(); final String primary = internalCluster().startDataOnlyNode(); createIndex(INDEX_NAME); ensureYellow(INDEX_NAME); @@ -702,7 +705,6 @@ public void testDropPrimaryDuringReplication() throws Exception { .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, replica_count) .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) .build(); - final String clusterManagerNode = internalCluster().startClusterManagerOnlyNode(); final String primaryNode = internalCluster().startDataOnlyNode(); createIndex(INDEX_NAME, settings); final List dataNodes = internalCluster().startDataOnlyNodes(6); @@ -742,11 +744,10 @@ public void testDropPrimaryDuringReplication() throws Exception { } public void testReplicaHasDiffFilesThanPrimary() throws Exception { - internalCluster().startClusterManagerOnlyNode(); - final String primaryNode = internalCluster().startNode(); + final String primaryNode = internalCluster().startDataOnlyNode(); createIndex(INDEX_NAME, Settings.builder().put(indexSettings()).put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 1).build()); ensureYellow(INDEX_NAME); - final String replicaNode = internalCluster().startNode(); + final String replicaNode = internalCluster().startDataOnlyNode(); ensureGreen(INDEX_NAME); final IndexShard replicaShard = getIndexShard(replicaNode, INDEX_NAME); @@ -796,9 +797,9 @@ public void testReplicaHasDiffFilesThanPrimary() throws Exception { } public void testPressureServiceStats() throws Exception { - final String primaryNode = internalCluster().startNode(); + final String primaryNode = internalCluster().startDataOnlyNode(); createIndex(INDEX_NAME); - final String replicaNode = internalCluster().startNode(); + final String replicaNode = internalCluster().startDataOnlyNode(); ensureGreen(INDEX_NAME); int initialDocCount = scaledRandomIntBetween(100, 200); @@ -848,7 +849,7 @@ public void testPressureServiceStats() throws Exception { assertEquals(0, replicaNode_service.nodeStats().getShardStats().get(primaryShard.shardId()).getReplicaStats().size()); // start another replica. - String replicaNode_2 = internalCluster().startNode(); + String replicaNode_2 = internalCluster().startDataOnlyNode(); ensureGreen(INDEX_NAME); String docId = String.valueOf(initialDocCount + 1); client().prepareIndex(INDEX_NAME).setId(docId).setSource("foo", "bar").get(); @@ -887,10 +888,10 @@ public void testPressureServiceStats() throws Exception { public void testScrollCreatedOnReplica() throws Exception { assumeFalse("Skipping the test with Remote store as its flaky.", segmentReplicationWithRemoteEnabled()); // create the cluster with one primary node containing primary shard and replica node containing replica shard - final String primary = internalCluster().startNode(); + final String primary = internalCluster().startDataOnlyNode(); createIndex(INDEX_NAME); ensureYellowAndNoInitializingShards(INDEX_NAME); - final String replica = internalCluster().startNode(); + final String replica = internalCluster().startDataOnlyNode(); ensureGreen(INDEX_NAME); // index 100 docs @@ -981,7 +982,7 @@ public void testScrollWithOngoingSegmentReplication() throws Exception { ); // create the cluster with one primary node containing primary shard and replica node containing replica shard - final String primary = internalCluster().startNode(); + final String primary = internalCluster().startDataOnlyNode(); prepareCreate( INDEX_NAME, Settings.builder() @@ -989,7 +990,7 @@ public void testScrollWithOngoingSegmentReplication() throws Exception { .put("index.refresh_interval", -1) ).get(); ensureYellowAndNoInitializingShards(INDEX_NAME); - final String replica = internalCluster().startNode(); + final String replica = internalCluster().startDataOnlyNode(); ensureGreen(INDEX_NAME); final int initialDocCount = 10; @@ -1104,10 +1105,10 @@ public void testScrollWithOngoingSegmentReplication() throws Exception { } public void testPitCreatedOnReplica() throws Exception { - final String primary = internalCluster().startNode(); + final String primary = internalCluster().startDataOnlyNode(); createIndex(INDEX_NAME); ensureYellowAndNoInitializingShards(INDEX_NAME); - final String replica = internalCluster().startNode(); + final String replica = internalCluster().startDataOnlyNode(); ensureGreen(INDEX_NAME); client().prepareIndex(INDEX_NAME) .setId("1") @@ -1234,13 +1235,13 @@ public void testPitCreatedOnReplica() throws Exception { */ public void testPrimaryReceivesDocsDuringReplicaRecovery() throws Exception { final List nodes = new ArrayList<>(); - final String primaryNode = internalCluster().startNode(); + final String primaryNode = internalCluster().startDataOnlyNode(); nodes.add(primaryNode); final Settings settings = Settings.builder().put(indexSettings()).put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0).build(); createIndex(INDEX_NAME, settings); ensureGreen(INDEX_NAME); // start a replica node, initially will be empty with no shard assignment. - final String replicaNode = internalCluster().startNode(); + final String replicaNode = internalCluster().startDataOnlyNode(); nodes.add(replicaNode); // index a doc. diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationRemoteStoreIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationRemoteStoreIT.java index ab0c0cc3aec77..7e79812fcfeea 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationRemoteStoreIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationRemoteStoreIT.java @@ -8,7 +8,6 @@ package org.opensearch.remotestore; -import org.apache.lucene.tests.util.LuceneTestCase; import org.junit.After; import org.junit.Before; import org.opensearch.cluster.metadata.IndexMetadata; @@ -26,7 +25,6 @@ * This makes sure that the constructs/flows that are being tested with Segment Replication, holds true after enabling * remote store. */ -@LuceneTestCase.AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/7643") @OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) public class SegmentReplicationRemoteStoreIT extends SegmentReplicationIT { @@ -49,7 +47,7 @@ protected Settings featureFlagSettings() { } @Before - public void setup() { + private void setup() { internalCluster().startClusterManagerOnlyNode(); Path absolutePath = randomRepoPath().toAbsolutePath(); assertAcked( diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index 93d4d4bbfec8b..9938d11caca13 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -1579,13 +1579,21 @@ public Tuple, ReplicationCheckpoint> getLatestSegme if (indexSettings.isSegRepEnabled() == false) { return null; } + + Tuple, ReplicationCheckpoint> nullSegmentInfosEmptyCheckpoint = new Tuple<>( + new GatedCloseable<>(null, () -> {}), + ReplicationCheckpoint.empty(shardId, getDefaultCodecName()) + ); + if (getEngineOrNull() == null) { - return new Tuple<>(new GatedCloseable<>(null, () -> {}), ReplicationCheckpoint.empty(shardId, getDefaultCodecName())); + return nullSegmentInfosEmptyCheckpoint; } // do not close the snapshot - caller will close it. - final GatedCloseable snapshot = getSegmentInfosSnapshot(); - return Optional.ofNullable(snapshot.get()).map(segmentInfos -> { - try { + GatedCloseable snapshot = null; + try { + snapshot = getSegmentInfosSnapshot(); + if (snapshot.get() != null) { + SegmentInfos segmentInfos = snapshot.get(); return new Tuple<>( snapshot, new ReplicationCheckpoint( @@ -1601,10 +1609,18 @@ public Tuple, ReplicationCheckpoint> getLatestSegme getEngine().config().getCodec().getName() ) ); - } catch (IOException e) { - throw new OpenSearchException("Error Fetching SegmentInfos and latest checkpoint", e); } - }).orElseGet(() -> new Tuple<>(new GatedCloseable<>(null, () -> {}), ReplicationCheckpoint.empty(shardId, getDefaultCodecName()))); + } catch (IOException | AlreadyClosedException e) { + logger.error("Error Fetching SegmentInfos and latest checkpoint", e); + if (snapshot != null) { + try { + snapshot.close(); + } catch (IOException ex) { + throw new OpenSearchException("Error Closing SegmentInfos Snapshot", e); + } + } + } + return nullSegmentInfosEmptyCheckpoint; } /** 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 1c2ddf43f8274..688f29fa1f4bf 100644 --- a/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java +++ b/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java @@ -282,7 +282,6 @@ public void testRefreshSuccessOnSecondAttempt() throws Exception { /** * Tests retry flow after snapshot and metadata files have been uploaded to remote store in the failed attempt. * Snapshot and metadata files created in failed attempt should not break retry. - * @throws Exception */ public void testRefreshSuccessAfterFailureInFirstAttemptAfterSnapshotAndMetadataUpload() throws Exception { int succeedOnAttempt = 1; From 9d9a143ff7a896bcfb116fb133f84718b73dec8b Mon Sep 17 00:00:00 2001 From: Ashish Date: Tue, 27 Jun 2023 15:16:21 +0530 Subject: [PATCH 072/109] [Remote Store] Add remote segment upload backpressure integ tests (#8197) Signed-off-by: Ashish Singh --- ...emoteStoreMockRepositoryIntegTestCase.java | 5 +- .../RemoteStoreBackpressureIT.java | 117 ++++++++++++++++-- .../snapshots/mockstore/MockRepository.java | 6 +- 3 files changed, 115 insertions(+), 13 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/AbstractRemoteStoreMockRepositoryIntegTestCase.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/AbstractRemoteStoreMockRepositoryIntegTestCase.java index 2bcbf3f5b614d..f57c312aa2cd0 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/AbstractRemoteStoreMockRepositoryIntegTestCase.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/AbstractRemoteStoreMockRepositoryIntegTestCase.java @@ -70,7 +70,7 @@ protected void deleteRepo() { assertAcked(clusterAdmin().prepareDeleteRepository(REPOSITORY_NAME)); } - protected void setup(Path repoLocation, double ioFailureRate, String skipExceptionBlobList, long maxFailure) { + protected String setup(Path repoLocation, double ioFailureRate, String skipExceptionBlobList, long maxFailure) { logger.info("--> Creating repository={} at the path={}", REPOSITORY_NAME, repoLocation); // The random_control_io_exception_rate setting ensures that 10-25% of all operations to remote store results in /// IOException. skip_exception_on_verification_file & skip_exception_on_list_blobs settings ensures that the @@ -88,13 +88,14 @@ protected void setup(Path repoLocation, double ioFailureRate, String skipExcepti .put("max_failure_number", maxFailure) ); - internalCluster().startDataOnlyNodes(1); + String dataNodeName = internalCluster().startDataOnlyNodes(1).get(0); createIndex(INDEX_NAME); logger.info("--> Created index={}", INDEX_NAME); ensureYellowAndNoInitializingShards(INDEX_NAME); logger.info("--> Cluster is yellow with no initializing shards"); ensureGreen(INDEX_NAME); logger.info("--> Cluster is green"); + return dataNodeName; } /** diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBackpressureIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBackpressureIT.java index c46eab6468c6b..64d5f06f061a9 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBackpressureIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBackpressureIT.java @@ -11,40 +11,111 @@ import org.opensearch.action.admin.cluster.remotestore.stats.RemoteStoreStats; import org.opensearch.action.admin.cluster.remotestore.stats.RemoteStoreStatsResponse; import org.opensearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse; +import org.opensearch.common.bytes.BytesArray; +import org.opensearch.common.bytes.BytesReference; import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.ByteSizeUnit; +import org.opensearch.common.xcontent.XContentType; import org.opensearch.core.concurrency.OpenSearchRejectedExecutionException; import org.opensearch.index.remote.RemoteRefreshSegmentTracker; +import org.opensearch.repositories.RepositoriesService; +import org.opensearch.snapshots.mockstore.MockRepository; import org.opensearch.test.OpenSearchIntegTestCase; import java.nio.file.Path; import java.util.Arrays; import java.util.List; import java.util.Locale; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +import static org.opensearch.index.remote.RemoteRefreshSegmentPressureSettings.MIN_CONSECUTIVE_FAILURES_LIMIT; import static org.opensearch.index.remote.RemoteRefreshSegmentPressureSettings.REMOTE_REFRESH_SEGMENT_PRESSURE_ENABLED; @OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) public class RemoteStoreBackpressureIT extends AbstractRemoteStoreMockRepositoryIntegTestCase { + public void testWritesRejectedDueToConsecutiveFailureBreach() throws Exception { + // Here the doc size of the request remains same throughout the test. After initial indexing, all remote store interactions + // fail leading to consecutive failure limit getting exceeded and leading to rejections. + validateBackpressure(ByteSizeUnit.KB.toIntBytes(1), 10, ByteSizeUnit.KB.toIntBytes(1), 15, "failure_streak_count"); + } + + public void testWritesRejectedDueToBytesLagBreach() throws Exception { + // Initially indexing happens with doc size of 2 bytes, then all remote store interactions start failing. Now, the + // indexing happens with doc size of 1KB leading to bytes lag limit getting exceeded and leading to rejections. + validateBackpressure(ByteSizeUnit.BYTES.toIntBytes(2), 30, ByteSizeUnit.KB.toIntBytes(1), 15, "bytes_lag"); + } - public void testWritesRejected() { + public void testWritesRejectedDueToTimeLagBreach() throws Exception { + // Initially indexing happens with doc size of 1KB, then all remote store interactions start failing. Now, the + // indexing happens with doc size of 1 byte leading to time lag limit getting exceeded and leading to rejections. + validateBackpressure(ByteSizeUnit.KB.toIntBytes(1), 20, ByteSizeUnit.BYTES.toIntBytes(1), 15, "time_lag"); + } + + private void validateBackpressure( + int initialDocSize, + int initialDocsToIndex, + int onFailureDocSize, + int onFailureDocsToIndex, + String breachMode + ) throws Exception { Path location = randomRepoPath().toAbsolutePath(); - setup(location, 1d, "metadata", Long.MAX_VALUE); + String dataNodeName = setup(location, 0d, "metadata", Long.MAX_VALUE); - Settings request = Settings.builder().put(REMOTE_REFRESH_SEGMENT_PRESSURE_ENABLED.getKey(), true).build(); + Settings request = Settings.builder() + .put(REMOTE_REFRESH_SEGMENT_PRESSURE_ENABLED.getKey(), true) + .put(MIN_CONSECUTIVE_FAILURES_LIMIT.getKey(), 10) + .build(); ClusterUpdateSettingsResponse clusterUpdateResponse = client().admin() .cluster() .prepareUpdateSettings() .setPersistentSettings(request) .get(); assertEquals(clusterUpdateResponse.getPersistentSettings().get(REMOTE_REFRESH_SEGMENT_PRESSURE_ENABLED.getKey()), "true"); + assertEquals(clusterUpdateResponse.getPersistentSettings().get(MIN_CONSECUTIVE_FAILURES_LIMIT.getKey()), "10"); logger.info("--> Indexing data"); + + String jsonString = generateString(initialDocSize); + BytesReference initialSource = new BytesArray(jsonString); + indexDocAndRefresh(initialSource, initialDocsToIndex); + + ((MockRepository) internalCluster().getInstance(RepositoriesService.class, dataNodeName).repository(REPOSITORY_NAME)) + .setRandomControlIOExceptionRate(1d); + + jsonString = generateString(onFailureDocSize); + BytesReference onFailureSource = new BytesArray(jsonString); OpenSearchRejectedExecutionException ex = assertThrows( OpenSearchRejectedExecutionException.class, - () -> indexData(randomIntBetween(10, 20), randomBoolean()) + () -> indexDocAndRefresh(onFailureSource, onFailureDocsToIndex) ); assertTrue(ex.getMessage().contains("rejected execution on primary shard")); + assertTrue(ex.getMessage().contains(breachMode)); + + RemoteRefreshSegmentTracker.Stats stats = stats(); + assertTrue(stats.bytesLag > 0); + assertTrue(stats.refreshTimeLagMs > 0); + assertTrue(stats.localRefreshNumber - stats.remoteRefreshNumber > 0); + assertTrue(stats.rejectionCount > 0); + + ((MockRepository) internalCluster().getInstance(RepositoriesService.class, dataNodeName).repository(REPOSITORY_NAME)) + .setRandomControlIOExceptionRate(0d); + + assertBusy(() -> { + RemoteRefreshSegmentTracker.Stats finalStats = stats(); + assertEquals(0, finalStats.bytesLag); + assertEquals(0, finalStats.refreshTimeLagMs); + assertEquals(0, finalStats.localRefreshNumber - finalStats.remoteRefreshNumber); + }, 30, TimeUnit.SECONDS); + + long rejectionCount = stats.rejectionCount; + stats = stats(); + indexDocAndRefresh(initialSource, initialDocsToIndex); + assertEquals(rejectionCount, stats.rejectionCount); + deleteRepo(); + } + + private RemoteRefreshSegmentTracker.Stats stats() { String shardId = "0"; RemoteStoreStatsResponse response = client().admin().cluster().prepareRemoteStoreStats(INDEX_NAME, shardId).get(); final String indexShardId = String.format(Locale.ROOT, "[%s][%s]", INDEX_NAME, shardId); @@ -52,11 +123,37 @@ public void testWritesRejected() { .filter(stat -> indexShardId.equals(stat.getStats().shardId.toString())) .collect(Collectors.toList()); assertEquals(1, matches.size()); - RemoteRefreshSegmentTracker.Stats stats = matches.get(0).getStats(); - assertTrue(stats.bytesLag > 0); - assertTrue(stats.refreshTimeLagMs > 0); - assertTrue(stats.localRefreshNumber - stats.remoteRefreshNumber > 0); - assertTrue(stats.rejectionCount > 0); - deleteRepo(); + return matches.get(0).getStats(); + } + + private void indexDocAndRefresh(BytesReference source, int iterations) { + for (int i = 0; i < iterations; i++) { + client().prepareIndex(INDEX_NAME).setSource(source, XContentType.JSON).get(); + refresh(INDEX_NAME); + } + } + + /** + * Generates string of given sizeInBytes + * + * @param sizeInBytes size of the string + * @return the generated string + */ + private String generateString(int sizeInBytes) { + StringBuilder sb = new StringBuilder(); + sb.append("{"); + int i = 0; + // Based on local tests, 1 char is occupying 1 byte + while (sb.length() < sizeInBytes) { + String key = "field" + i; + String value = "value" + i; + sb.append("\"").append(key).append("\":\"").append(value).append("\","); + i++; + } + if (sb.length() > 1 && sb.charAt(sb.length() - 1) == ',') { + sb.setLength(sb.length() - 1); + } + sb.append("}"); + return sb.toString(); } } diff --git a/test/framework/src/main/java/org/opensearch/snapshots/mockstore/MockRepository.java b/test/framework/src/main/java/org/opensearch/snapshots/mockstore/MockRepository.java index fcaf9f6c900d3..7a7c4bd448c55 100644 --- a/test/framework/src/main/java/org/opensearch/snapshots/mockstore/MockRepository.java +++ b/test/framework/src/main/java/org/opensearch/snapshots/mockstore/MockRepository.java @@ -114,7 +114,7 @@ public long getFailureCount() { return failureCounter.get(); } - private final double randomControlIOExceptionRate; + private volatile double randomControlIOExceptionRate; private final double randomDataFileIOExceptionRate; @@ -246,6 +246,10 @@ public synchronized void unblock() { this.notifyAll(); } + public void setRandomControlIOExceptionRate(double randomControlIOExceptionRate) { + this.randomControlIOExceptionRate = randomControlIOExceptionRate; + } + public void blockOnDataFiles(boolean blocked) { blockOnDataFiles = blocked; } From 246b922fc1a15985254d38222fc6c62570f16cf3 Mon Sep 17 00:00:00 2001 From: Thomas Farr Date: Wed, 28 Jun 2023 02:33:13 +1200 Subject: [PATCH 073/109] Bump `resteasy-jackson2-provider` from 3.0.26.Final to 6.2.4.Final in /qa/wildfly (#8209) Signed-off-by: Thomas Farr --- CHANGELOG.md | 1 + buildSrc/version.properties | 2 + qa/wildfly/build.gradle | 37 +++++++++++-------- qa/wildfly/docker-compose.yml | 2 +- .../opensearch/wildfly/model/Employee.java | 4 -- .../RestHighLevelClientActivator.java | 12 +----- .../RestHighLevelClientEmployeeResource.java | 18 +++++---- .../RestHighLevelClientProducer.java | 3 +- .../RestHighLevelJacksonJsonProvider.java | 2 +- .../WEB-INF/jboss-deployment-structure.xml | 3 -- 10 files changed, 40 insertions(+), 44 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index f5bc63794e46d..50e1fe78daf5d 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -113,6 +113,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `org.apache.hadoop:hadoop-minicluster` from 3.3.5 to 3.3.6 (#8257) - Bump `io.projectreactor.netty:reactor-netty-http` from 1.1.7 to 1.1.8 (#8256) - [Upgrade] Lucene 9.7.0 release (#8272) +- Bump `org.jboss.resteasy:resteasy-jackson2-provider` from 3.0.26.Final to 6.2.4.Final in /qa/wildfly ([#8209](https://github.com/opensearch-project/OpenSearch/pull/8209)) ### Changed - Replace jboss-annotations-api_1.2_spec with jakarta.annotation-api ([#7836](https://github.com/opensearch-project/OpenSearch/pull/7836)) diff --git a/buildSrc/version.properties b/buildSrc/version.properties index 377421703b892..dd64569259c2d 100644 --- a/buildSrc/version.properties +++ b/buildSrc/version.properties @@ -64,3 +64,5 @@ jmh = 1.35 zstd = 1.5.5-3 jzlib = 1.1.3 + +resteasy = 6.2.4.Final diff --git a/qa/wildfly/build.gradle b/qa/wildfly/build.gradle index a2a13165ca10c..391d2c78b489b 100644 --- a/qa/wildfly/build.gradle +++ b/qa/wildfly/build.gradle @@ -40,25 +40,32 @@ apply plugin: 'opensearch.internal-distribution-download' testFixtures.useFixture() dependencies { - providedCompile 'javax.enterprise:cdi-api:2.0' - providedCompile "jakarta.annotation:jakarta.annotation-api:${versions.jakarta_annotation}" - providedCompile 'jakarta.ws.rs:jakarta.ws.rs-api:2.1.3' - api('org.jboss.resteasy:resteasy-jackson2-provider:3.0.26.Final') { - exclude module: 'jackson-annotations' - exclude module: 'jackson-core' - exclude module: 'jackson-databind' - exclude module: 'jackson-jaxrs-json-provider' + providedCompile('jakarta.enterprise:jakarta.enterprise.cdi-api:4.0.1') { + exclude module: 'jakarta.annotation-api' + } + providedCompile 'jakarta.ws.rs:jakarta.ws.rs-api:3.1.0' + providedCompile "org.jboss.resteasy:resteasy-core:${versions.resteasy}" + providedCompile "org.jboss.resteasy:resteasy-core-spi:${versions.resteasy}" + api("org.jboss.resteasy:resteasy-jackson2-provider:${versions.resteasy}") { + exclude module: 'jakarta.activation-api' + exclude group: 'com.fasterxml.jackson' + exclude group: 'com.fasterxml.jackson.core' + exclude group: 'com.fasterxml.jackson.dataformat' + exclude group: 'com.fasterxml.jackson.module' } api "com.fasterxml.jackson.core:jackson-annotations:${versions.jackson}" - api "com.fasterxml.jackson.core:jackson-core:${versions.jackson}" - api "com.fasterxml.jackson.core:jackson-databind:${versions.jackson_databind}" - api "com.fasterxml.jackson.jaxrs:jackson-jaxrs-json-provider:${versions.jackson}" - api "com.fasterxml.jackson.jaxrs:jackson-jaxrs-base:${versions.jackson}" - api "com.fasterxml.jackson.module:jackson-module-jaxb-annotations:${versions.jackson}" + api "com.fasterxml.jackson.core:jackson-databind:${versions.jackson}" + api "com.fasterxml.jackson.jakarta.rs:jackson-jakarta-rs-base:${versions.jackson}" + api "com.fasterxml.jackson.jakarta.rs:jackson-jakarta-rs-json-provider:${versions.jackson}" + api "com.github.fge:json-patch:1.9" api "org.apache.logging.log4j:log4j-api:${versions.log4j}" api "org.apache.logging.log4j:log4j-core:${versions.log4j}" - api project(path: ':client:rest-high-level') - testImplementation project(':test:framework') + api(project(path: ':client:rest-high-level')) { + exclude module: 'jakarta.annotation-api' + } + testImplementation(project(':test:framework')) { + exclude module: 'jakarta.annotation-api' + } } war { diff --git a/qa/wildfly/docker-compose.yml b/qa/wildfly/docker-compose.yml index 96f168ba3505c..b0f1609f01e72 100644 --- a/qa/wildfly/docker-compose.yml +++ b/qa/wildfly/docker-compose.yml @@ -2,7 +2,7 @@ version: '3.7' services: wildfly: - image: jboss/wildfly:18.0.1.Final + image: quay.io/wildfly/wildfly:28.0.1.Final-jdk11 environment: JAVA_OPTS: -Dopensearch.uri=opensearch:9200 -Djboss.http.port=8080 -Djava.net.preferIPv4Stack=true volumes: diff --git a/qa/wildfly/src/main/java/org/opensearch/wildfly/model/Employee.java b/qa/wildfly/src/main/java/org/opensearch/wildfly/model/Employee.java index 5cf40ce941636..8f8d6635d568d 100644 --- a/qa/wildfly/src/main/java/org/opensearch/wildfly/model/Employee.java +++ b/qa/wildfly/src/main/java/org/opensearch/wildfly/model/Employee.java @@ -34,12 +34,8 @@ import com.fasterxml.jackson.annotation.JsonProperty; -import javax.ws.rs.Consumes; -import javax.ws.rs.core.MediaType; - import java.util.List; -@Consumes(MediaType.APPLICATION_JSON) public class Employee { @JsonProperty(value = "first_name") diff --git a/qa/wildfly/src/main/java/org/opensearch/wildfly/transport/RestHighLevelClientActivator.java b/qa/wildfly/src/main/java/org/opensearch/wildfly/transport/RestHighLevelClientActivator.java index bf91346933e02..14b8cd6730531 100644 --- a/qa/wildfly/src/main/java/org/opensearch/wildfly/transport/RestHighLevelClientActivator.java +++ b/qa/wildfly/src/main/java/org/opensearch/wildfly/transport/RestHighLevelClientActivator.java @@ -32,18 +32,10 @@ package org.opensearch.wildfly.transport; -import javax.ws.rs.ApplicationPath; -import javax.ws.rs.core.Application; - -import java.util.Collections; -import java.util.Set; +import jakarta.ws.rs.ApplicationPath; +import jakarta.ws.rs.core.Application; @ApplicationPath("/transport") public class RestHighLevelClientActivator extends Application { - @Override - public Set> getClasses() { - return Collections.singleton(RestHighLevelClientEmployeeResource.class); - } - } diff --git a/qa/wildfly/src/main/java/org/opensearch/wildfly/transport/RestHighLevelClientEmployeeResource.java b/qa/wildfly/src/main/java/org/opensearch/wildfly/transport/RestHighLevelClientEmployeeResource.java index 036d782f6f5e8..432b40367c978 100644 --- a/qa/wildfly/src/main/java/org/opensearch/wildfly/transport/RestHighLevelClientEmployeeResource.java +++ b/qa/wildfly/src/main/java/org/opensearch/wildfly/transport/RestHighLevelClientEmployeeResource.java @@ -32,6 +32,15 @@ package org.opensearch.wildfly.transport; +import jakarta.inject.Inject; +import jakarta.ws.rs.Consumes; +import jakarta.ws.rs.GET; +import jakarta.ws.rs.PUT; +import jakarta.ws.rs.Path; +import jakarta.ws.rs.PathParam; +import jakarta.ws.rs.Produces; +import jakarta.ws.rs.core.MediaType; +import jakarta.ws.rs.core.Response; import org.opensearch.action.get.GetRequest; import org.opensearch.action.get.GetResponse; import org.opensearch.action.index.IndexRequest; @@ -41,14 +50,6 @@ import org.opensearch.core.xcontent.XContentBuilder; import org.opensearch.wildfly.model.Employee; -import javax.inject.Inject; -import javax.ws.rs.GET; -import javax.ws.rs.PUT; -import javax.ws.rs.Path; -import javax.ws.rs.PathParam; -import javax.ws.rs.Produces; -import javax.ws.rs.core.MediaType; -import javax.ws.rs.core.Response; import java.io.IOException; import java.net.URI; import java.net.URISyntaxException; @@ -88,6 +89,7 @@ public Response getEmployeeById(final @PathParam("id") Long id) throws IOExcepti @PUT @Path("/{id}") + @Consumes(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON) public Response putEmployeeById(final @PathParam("id") Long id, final Employee employee) throws URISyntaxException, IOException { Objects.requireNonNull(id); diff --git a/qa/wildfly/src/main/java/org/opensearch/wildfly/transport/RestHighLevelClientProducer.java b/qa/wildfly/src/main/java/org/opensearch/wildfly/transport/RestHighLevelClientProducer.java index 2b1abe45f7723..490ecd214c3f3 100644 --- a/qa/wildfly/src/main/java/org/opensearch/wildfly/transport/RestHighLevelClientProducer.java +++ b/qa/wildfly/src/main/java/org/opensearch/wildfly/transport/RestHighLevelClientProducer.java @@ -32,14 +32,13 @@ package org.opensearch.wildfly.transport; +import jakarta.enterprise.inject.Produces; import org.apache.hc.core5.http.HttpHost; import org.opensearch.client.RestClient; import org.opensearch.client.RestHighLevelClient; import org.opensearch.common.SuppressForbidden; import org.opensearch.common.io.PathUtils; -import javax.enterprise.inject.Produces; - import java.net.URISyntaxException; import java.nio.file.Path; diff --git a/qa/wildfly/src/main/java/org/opensearch/wildfly/transport/RestHighLevelJacksonJsonProvider.java b/qa/wildfly/src/main/java/org/opensearch/wildfly/transport/RestHighLevelJacksonJsonProvider.java index 604d975f53280..7989f0351daef 100644 --- a/qa/wildfly/src/main/java/org/opensearch/wildfly/transport/RestHighLevelJacksonJsonProvider.java +++ b/qa/wildfly/src/main/java/org/opensearch/wildfly/transport/RestHighLevelJacksonJsonProvider.java @@ -34,7 +34,7 @@ import org.jboss.resteasy.plugins.providers.jackson.ResteasyJackson2Provider; -import javax.ws.rs.ext.Provider; +import jakarta.ws.rs.ext.Provider; @Provider public class RestHighLevelJacksonJsonProvider extends ResteasyJackson2Provider { diff --git a/qa/wildfly/src/main/webapp/WEB-INF/jboss-deployment-structure.xml b/qa/wildfly/src/main/webapp/WEB-INF/jboss-deployment-structure.xml index 7191bfe1268aa..a08090100989a 100644 --- a/qa/wildfly/src/main/webapp/WEB-INF/jboss-deployment-structure.xml +++ b/qa/wildfly/src/main/webapp/WEB-INF/jboss-deployment-structure.xml @@ -1,9 +1,6 @@ - - - From f936b896ced3ec33f7116f52f45f1df6fd61fc52 Mon Sep 17 00:00:00 2001 From: Andriy Redko Date: Tue, 27 Jun 2023 13:22:54 -0400 Subject: [PATCH 074/109] Fix Apache Lucene verson for 2.9.0 release (#8290) Signed-off-by: Andriy Redko --- libs/core/src/main/java/org/opensearch/Version.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/libs/core/src/main/java/org/opensearch/Version.java b/libs/core/src/main/java/org/opensearch/Version.java index c399f7f3e1b1c..8c0e247bc3169 100644 --- a/libs/core/src/main/java/org/opensearch/Version.java +++ b/libs/core/src/main/java/org/opensearch/Version.java @@ -89,7 +89,7 @@ public class Version implements Comparable, ToXContentFragment { public static final Version V_2_7_1 = new Version(2070199, org.apache.lucene.util.Version.LUCENE_9_5_0); public static final Version V_2_8_0 = new Version(2080099, org.apache.lucene.util.Version.LUCENE_9_6_0); public static final Version V_2_8_1 = new Version(2080199, org.apache.lucene.util.Version.LUCENE_9_6_0); - public static final Version V_2_9_0 = new Version(2090099, org.apache.lucene.util.Version.LUCENE_9_6_0); + public static final Version V_2_9_0 = new Version(2090099, org.apache.lucene.util.Version.LUCENE_9_7_0); public static final Version V_3_0_0 = new Version(3000099, org.apache.lucene.util.Version.LUCENE_9_7_0); public static final Version CURRENT = V_3_0_0; From 317dd03ed83110a5c4ab665f3682a22a7c1c864a Mon Sep 17 00:00:00 2001 From: Harish Bhakuni Date: Tue, 27 Jun 2023 11:01:58 -0700 Subject: [PATCH 075/109] [Snapshot Interop] Add Changes in Restore Snapshot Flow for remote store Interoperability (#6788) * [Snapshot Interop] Add Changes in Restore Snapshot Flow for remote store interoperability. --------- Signed-off-by: Harish Bhakuni Co-authored-by: Harish Bhakuni --- .../rest-api-spec/api/snapshot.restore.json | 4 + .../snapshots/RestoreSnapshotIT.java | 513 ++++++++++++++++++ .../restore/RestoreSnapshotRequest.java | 42 +- .../RestoreSnapshotRequestBuilder.java | 8 + .../cluster/routing/RecoverySource.java | 50 +- .../opensearch/index/shard/IndexShard.java | 199 +++++-- .../shard/RemoteStoreRefreshListener.java | 15 +- .../opensearch/index/shard/StoreRecovery.java | 98 +++- .../store/RemoteSegmentStoreDirectory.java | 19 + .../RemoteSegmentStoreDirectoryFactory.java | 5 + .../metadata/RemoteSegmentMetadata.java | 11 +- .../index/translog/RemoteFsTranslog.java | 14 + .../transfer/TranslogTransferManager.java | 5 + .../cluster/IndicesClusterStateService.java | 15 +- .../repositories/FilterRepository.java | 10 + .../opensearch/repositories/Repository.java | 17 + .../blobstore/BlobStoreRepository.java | 10 + .../opensearch/snapshots/RestoreService.java | 15 +- .../restore/RestoreSnapshotRequestTests.java | 4 + .../cluster/routing/RecoverySourceTests.java | 24 + .../index/shard/IndexShardTests.java | 77 ++- .../RemoteSegmentStoreDirectoryTests.java | 18 +- .../RemoteSegmentMetadataHandlerTests.java | 4 + .../RepositoriesServiceTests.java | 10 + .../blobstore/BlobStoreRepositoryTests.java | 92 +++- .../index/shard/IndexShardTestCase.java | 32 +- .../blobstore/BlobStoreTestUtil.java | 31 +- .../AbstractSnapshotIntegTestCase.java | 48 +- 28 files changed, 1303 insertions(+), 87 deletions(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.restore.json b/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.restore.json index 87ab8117ec489..07148c7d261f4 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.restore.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/snapshot.restore.json @@ -42,6 +42,10 @@ "type":"boolean", "description":"Should this request wait until the operation has completed before returning", "default":false + }, + "source_remote_store_repository": { + "type":"string", + "description":"Remote Store Repository of Remote Store Indices" } }, "body":{ diff --git a/server/src/internalClusterTest/java/org/opensearch/snapshots/RestoreSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/snapshots/RestoreSnapshotIT.java index 8be14d1188db8..9f492bbaee01a 100644 --- a/server/src/internalClusterTest/java/org/opensearch/snapshots/RestoreSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/snapshots/RestoreSnapshotIT.java @@ -33,29 +33,44 @@ package org.opensearch.snapshots; import org.opensearch.action.ActionFuture; +import org.opensearch.action.DocWriteResponse; +import org.opensearch.action.admin.cluster.remotestore.restore.RestoreRemoteStoreRequest; import org.opensearch.action.admin.cluster.snapshots.create.CreateSnapshotResponse; import org.opensearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse; +import org.opensearch.action.admin.indices.get.GetIndexRequest; +import org.opensearch.action.admin.indices.get.GetIndexResponse; import org.opensearch.action.admin.indices.settings.get.GetSettingsResponse; import org.opensearch.action.admin.indices.template.delete.DeleteIndexTemplateRequestBuilder; import org.opensearch.action.admin.indices.template.get.GetIndexTemplatesResponse; +import org.opensearch.action.delete.DeleteResponse; import org.opensearch.action.index.IndexRequestBuilder; +import org.opensearch.action.support.PlainActionFuture; import org.opensearch.client.Client; +import org.opensearch.client.Requests; import org.opensearch.cluster.block.ClusterBlocks; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.MappingMetadata; +import org.opensearch.common.io.PathUtils; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.ByteSizeUnit; import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.util.FeatureFlags; import org.opensearch.common.xcontent.XContentFactory; +import org.opensearch.index.IndexSettings; import org.opensearch.indices.InvalidIndexNameException; +import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.repositories.RepositoriesService; import org.opensearch.rest.RestStatus; +import org.opensearch.test.InternalTestCluster; +import java.io.IOException; import java.nio.file.Path; import java.util.Collections; import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.Arrays; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -70,6 +85,8 @@ import static org.hamcrest.Matchers.nullValue; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_REPLICAS; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_NUMBER_OF_SHARDS; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_STORE_ENABLED; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_STORE_REPOSITORY; import static org.opensearch.index.IndexSettings.INDEX_REFRESH_INTERVAL_SETTING; import static org.opensearch.index.IndexSettings.INDEX_SOFT_DELETES_SETTING; import static org.opensearch.index.query.QueryBuilders.matchQuery; @@ -81,6 +98,10 @@ import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertRequestBuilderThrows; public class RestoreSnapshotIT extends AbstractSnapshotIntegTestCase { + @Override + protected Settings nodeSettings(int nodeOrdinal) { + return Settings.builder().put(super.nodeSettings(nodeOrdinal)).put(FeatureFlags.REMOTE_STORE, "true").build(); + } public void testParallelRestoreOperations() { String indexName1 = "testindex1"; @@ -152,6 +173,498 @@ public void testParallelRestoreOperations() { assertThat(client.prepareGet(restoredIndexName2, docId2).get().isExists(), equalTo(true)); } + public void testRestoreRemoteStoreIndicesWithRemoteTranslog() throws IOException, ExecutionException, InterruptedException { + testRestoreOperationsShallowCopyEnabled(true); + } + + public void testRestoreRemoteStoreIndicesWithoutRemoteTranslog() throws IOException, ExecutionException, InterruptedException { + testRestoreOperationsShallowCopyEnabled(false); + } + + public void testRestoreOperationsShallowCopyEnabled(boolean remoteTranslogEnabled) throws IOException, ExecutionException, + InterruptedException { + internalCluster().startClusterManagerOnlyNode(); + final String primaryNode = internalCluster().startNode(); + String indexName1 = "testindex1"; + String indexName2 = "testindex2"; + String snapshotRepoName = "test-restore-snapshot-repo"; + String remoteStoreRepoName = "test-rs-repo" + TEST_REMOTE_STORE_REPO_SUFFIX; + String snapshotName1 = "test-restore-snapshot1"; + String snapshotName2 = "test-restore-snapshot2"; + Path absolutePath1 = randomRepoPath().toAbsolutePath(); + Path absolutePath2 = randomRepoPath().toAbsolutePath(); + logger.info("Snapshot Path [{}]", absolutePath1); + logger.info("Remote Store Repo Path [{}]", absolutePath2); + String restoredIndexName1 = indexName1 + "-restored"; + String restoredIndexName1Seg = indexName1 + "-restored-seg"; + String restoredIndexName1Doc = indexName1 + "-restored-doc"; + String restoredIndexName2 = indexName2 + "-restored"; + + createRepository(snapshotRepoName, "fs", getRepositorySettings(absolutePath1, true)); + createRepository(remoteStoreRepoName, "fs", absolutePath2); + + Client client = client(); + Settings indexSettings = getIndexSettings(true, remoteTranslogEnabled, remoteStoreRepoName, 1, 0).build(); + createIndex(indexName1, indexSettings); + + Settings indexSettings2 = getIndexSettings(false, false, null, 1, 0).build(); + createIndex(indexName2, indexSettings2); + + final int numDocsInIndex1 = 5; + final int numDocsInIndex2 = 6; + indexDocuments(client, indexName1, numDocsInIndex1); + indexDocuments(client, indexName2, numDocsInIndex2); + ensureGreen(indexName1, indexName2); + + final String secondNode = internalCluster().startNode(); + logger.info("--> snapshot"); + CreateSnapshotResponse createSnapshotResponse = client.admin() + .cluster() + .prepareCreateSnapshot(snapshotRepoName, snapshotName1) + .setWaitForCompletion(true) + .setIndices(indexName1, indexName2) + .get(); + assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), greaterThan(0)); + assertThat( + createSnapshotResponse.getSnapshotInfo().successfulShards(), + equalTo(createSnapshotResponse.getSnapshotInfo().totalShards()) + ); + assertThat(createSnapshotResponse.getSnapshotInfo().state(), equalTo(SnapshotState.SUCCESS)); + + updateRepository(snapshotRepoName, "fs", getRepositorySettings(absolutePath1, false)); + CreateSnapshotResponse createSnapshotResponse2 = client.admin() + .cluster() + .prepareCreateSnapshot(snapshotRepoName, snapshotName2) + .setWaitForCompletion(true) + .setIndices(indexName1, indexName2) + .get(); + assertThat(createSnapshotResponse2.getSnapshotInfo().successfulShards(), greaterThan(0)); + assertThat( + createSnapshotResponse2.getSnapshotInfo().successfulShards(), + equalTo(createSnapshotResponse2.getSnapshotInfo().totalShards()) + ); + assertThat(createSnapshotResponse2.getSnapshotInfo().state(), equalTo(SnapshotState.SUCCESS)); + + DeleteResponse deleteResponse = client().prepareDelete(indexName1, "0").execute().actionGet(); + assertEquals(deleteResponse.getResult(), DocWriteResponse.Result.DELETED); + indexDocuments(client, indexName1, numDocsInIndex1, numDocsInIndex1 + randomIntBetween(2, 5)); + ensureGreen(indexName1); + + RestoreSnapshotResponse restoreSnapshotResponse1 = client.admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepoName, snapshotName1) + .setWaitForCompletion(false) + .setIndices(indexName1) + .setRenamePattern(indexName1) + .setRenameReplacement(restoredIndexName1) + .get(); + RestoreSnapshotResponse restoreSnapshotResponse2 = client.admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepoName, snapshotName2) + .setWaitForCompletion(false) + .setIndices(indexName2) + .setRenamePattern(indexName2) + .setRenameReplacement(restoredIndexName2) + .get(); + assertEquals(restoreSnapshotResponse1.status(), RestStatus.ACCEPTED); + assertEquals(restoreSnapshotResponse2.status(), RestStatus.ACCEPTED); + ensureGreen(restoredIndexName1, restoredIndexName2); + assertDocsPresentInIndex(client, restoredIndexName1, numDocsInIndex1); + assertDocsPresentInIndex(client, restoredIndexName2, numDocsInIndex2); + + // deleting data for restoredIndexName1 and restoring from remote store. + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(primaryNodeName(restoredIndexName1))); + ensureRed(restoredIndexName1); + assertAcked(client().admin().indices().prepareClose(restoredIndexName1)); + client().admin() + .cluster() + .restoreRemoteStore(new RestoreRemoteStoreRequest().indices(restoredIndexName1), PlainActionFuture.newFuture()); + ensureYellowAndNoInitializingShards(restoredIndexName1); + ensureGreen(restoredIndexName1); + assertDocsPresentInIndex(client(), restoredIndexName1, numDocsInIndex1); + // indexing some new docs and validating + indexDocuments(client, restoredIndexName1, numDocsInIndex1, numDocsInIndex1 + 2); + ensureGreen(restoredIndexName1); + assertDocsPresentInIndex(client, restoredIndexName1, numDocsInIndex1 + 2); + + // restore index as seg rep enabled with remote store and remote translog disabled + RestoreSnapshotResponse restoreSnapshotResponse3 = client.admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepoName, snapshotName1) + .setWaitForCompletion(false) + .setIgnoreIndexSettings(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_ENABLED) + .setIndices(indexName1) + .setRenamePattern(indexName1) + .setRenameReplacement(restoredIndexName1Seg) + .get(); + assertEquals(restoreSnapshotResponse3.status(), RestStatus.ACCEPTED); + ensureGreen(restoredIndexName1Seg); + + GetIndexResponse getIndexResponse = client().admin() + .indices() + .getIndex(new GetIndexRequest().indices(restoredIndexName1Seg).includeDefaults(true)) + .get(); + indexSettings = getIndexResponse.settings().get(restoredIndexName1Seg); + assertNull(indexSettings.get(SETTING_REMOTE_STORE_ENABLED)); + assertNull(indexSettings.get(SETTING_REMOTE_STORE_REPOSITORY, null)); + assertEquals(ReplicationType.SEGMENT.toString(), indexSettings.get(IndexMetadata.SETTING_REPLICATION_TYPE)); + assertDocsPresentInIndex(client, restoredIndexName1Seg, numDocsInIndex1); + // indexing some new docs and validating + indexDocuments(client, restoredIndexName1Seg, numDocsInIndex1, numDocsInIndex1 + 2); + ensureGreen(restoredIndexName1Seg); + assertDocsPresentInIndex(client, restoredIndexName1Seg, numDocsInIndex1 + 2); + + // restore index as doc rep based from shallow copy snapshot + RestoreSnapshotResponse restoreSnapshotResponse4 = client.admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepoName, snapshotName1) + .setWaitForCompletion(false) + .setIgnoreIndexSettings( + IndexMetadata.SETTING_REMOTE_STORE_ENABLED, + IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_ENABLED, + IndexMetadata.SETTING_REPLICATION_TYPE + ) + .setIndices(indexName1) + .setRenamePattern(indexName1) + .setRenameReplacement(restoredIndexName1Doc) + .get(); + assertEquals(restoreSnapshotResponse4.status(), RestStatus.ACCEPTED); + ensureGreen(restoredIndexName1Doc); + + getIndexResponse = client().admin() + .indices() + .getIndex(new GetIndexRequest().indices(restoredIndexName1Doc).includeDefaults(true)) + .get(); + indexSettings = getIndexResponse.settings().get(restoredIndexName1Doc); + assertNull(indexSettings.get(SETTING_REMOTE_STORE_ENABLED)); + assertNull(indexSettings.get(SETTING_REMOTE_STORE_REPOSITORY, null)); + assertNull(indexSettings.get(IndexMetadata.SETTING_REPLICATION_TYPE)); + assertDocsPresentInIndex(client, restoredIndexName1Doc, numDocsInIndex1); + // indexing some new docs and validating + indexDocuments(client, restoredIndexName1Doc, numDocsInIndex1, numDocsInIndex1 + 2); + ensureGreen(restoredIndexName1Doc); + assertDocsPresentInIndex(client, restoredIndexName1Doc, numDocsInIndex1 + 2); + } + + public void testRestoreInSameRemoteStoreEnabledIndex() throws IOException { + internalCluster().startClusterManagerOnlyNode(); + internalCluster().startNode(); + String indexName1 = "testindex1"; + String indexName2 = "testindex2"; + String snapshotRepoName = "test-restore-snapshot-repo"; + String remoteStoreRepoName = "test-rs-repo" + TEST_REMOTE_STORE_REPO_SUFFIX; + String snapshotName1 = "test-restore-snapshot1"; + String snapshotName2 = "test-restore-snapshot2"; + Path absolutePath1 = randomRepoPath().toAbsolutePath(); + Path absolutePath2 = randomRepoPath().toAbsolutePath(); + logger.info("Snapshot Path [{}]", absolutePath1); + logger.info("Remote Store Repo Path [{}]", absolutePath2); + String restoredIndexName2 = indexName2 + "-restored"; + + boolean enableShallowCopy = randomBoolean(); + createRepository(snapshotRepoName, "fs", getRepositorySettings(absolutePath1, enableShallowCopy)); + createRepository(remoteStoreRepoName, "fs", absolutePath2); + + Client client = client(); + Settings indexSettings = getIndexSettings(true, randomBoolean(), remoteStoreRepoName, 1, 0).build(); + createIndex(indexName1, indexSettings); + + Settings indexSettings2 = getIndexSettings(false, false, null, 1, 0).build(); + createIndex(indexName2, indexSettings2); + + final int numDocsInIndex1 = 5; + final int numDocsInIndex2 = 6; + indexDocuments(client, indexName1, numDocsInIndex1); + indexDocuments(client, indexName2, numDocsInIndex2); + ensureGreen(indexName1, indexName2); + + final String secondNode = internalCluster().startNode(); + logger.info("--> snapshot"); + CreateSnapshotResponse createSnapshotResponse = client.admin() + .cluster() + .prepareCreateSnapshot(snapshotRepoName, snapshotName1) + .setWaitForCompletion(true) + .setIndices(indexName1, indexName2) + .get(); + assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), greaterThan(0)); + assertThat( + createSnapshotResponse.getSnapshotInfo().successfulShards(), + equalTo(createSnapshotResponse.getSnapshotInfo().totalShards()) + ); + assertThat(createSnapshotResponse.getSnapshotInfo().state(), equalTo(SnapshotState.SUCCESS)); + + updateRepository(snapshotRepoName, "fs", getRepositorySettings(absolutePath1, false)); + CreateSnapshotResponse createSnapshotResponse2 = client.admin() + .cluster() + .prepareCreateSnapshot(snapshotRepoName, snapshotName2) + .setWaitForCompletion(true) + .setIndices(indexName1, indexName2) + .get(); + assertThat(createSnapshotResponse2.getSnapshotInfo().successfulShards(), greaterThan(0)); + assertThat( + createSnapshotResponse2.getSnapshotInfo().successfulShards(), + equalTo(createSnapshotResponse2.getSnapshotInfo().totalShards()) + ); + assertThat(createSnapshotResponse2.getSnapshotInfo().state(), equalTo(SnapshotState.SUCCESS)); + + DeleteResponse deleteResponse = client().prepareDelete(indexName1, "0").execute().actionGet(); + assertEquals(deleteResponse.getResult(), DocWriteResponse.Result.DELETED); + indexDocuments(client, indexName1, numDocsInIndex1, numDocsInIndex1 + randomIntBetween(2, 5)); + ensureGreen(indexName1); + + assertAcked(client().admin().indices().prepareClose(indexName1)); + + RestoreSnapshotResponse restoreSnapshotResponse1 = client.admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepoName, snapshotName1) + .setWaitForCompletion(false) + .setIndices(indexName1) + .get(); + RestoreSnapshotResponse restoreSnapshotResponse2 = client.admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepoName, snapshotName2) + .setWaitForCompletion(false) + .setIndices(indexName2) + .setRenamePattern(indexName2) + .setRenameReplacement(restoredIndexName2) + .get(); + assertEquals(restoreSnapshotResponse1.status(), RestStatus.ACCEPTED); + assertEquals(restoreSnapshotResponse2.status(), RestStatus.ACCEPTED); + ensureGreen(indexName1, restoredIndexName2); + assertDocsPresentInIndex(client, indexName1, numDocsInIndex1); + assertDocsPresentInIndex(client, restoredIndexName2, numDocsInIndex2); + + // deleting data for restoredIndexName1 and restoring from remote store. + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(primaryNodeName(indexName1))); + ensureRed(indexName1); + assertAcked(client().admin().indices().prepareClose(indexName1)); + client().admin().cluster().restoreRemoteStore(new RestoreRemoteStoreRequest().indices(indexName1), PlainActionFuture.newFuture()); + ensureYellowAndNoInitializingShards(indexName1); + ensureGreen(indexName1); + assertDocsPresentInIndex(client(), indexName1, numDocsInIndex1); + // indexing some new docs and validating + indexDocuments(client, indexName1, numDocsInIndex1, numDocsInIndex1 + 2); + ensureGreen(indexName1); + assertDocsPresentInIndex(client, indexName1, numDocsInIndex1 + 2); + } + + public void testRestoreShallowCopySnapshotWithDifferentRepo() throws IOException { + internalCluster().startClusterManagerOnlyNode(); + final String primaryNode = internalCluster().startNode(); + String indexName1 = "testindex1"; + String indexName2 = "testindex2"; + String snapshotRepoName = "test-restore-snapshot-repo"; + String remoteStoreRepoName = "test-rs-repo" + TEST_REMOTE_STORE_REPO_SUFFIX; + String remoteStoreRepo2Name = "test-rs-repo-2" + TEST_REMOTE_STORE_REPO_SUFFIX; + String snapshotName1 = "test-restore-snapshot1"; + Path absolutePath1 = randomRepoPath().toAbsolutePath(); + Path absolutePath2 = randomRepoPath().toAbsolutePath(); + Path absolutePath3 = randomRepoPath().toAbsolutePath(); + String restoredIndexName1 = indexName1 + "-restored"; + + createRepository(snapshotRepoName, "fs", getRepositorySettings(absolutePath1, false)); + createRepository(remoteStoreRepoName, "fs", absolutePath2); + createRepository(remoteStoreRepo2Name, "fs", absolutePath3); + + Client client = client(); + Settings indexSettings = getIndexSettings(true, true, remoteStoreRepoName, 1, 0).build(); + createIndex(indexName1, indexSettings); + + Settings indexSettings2 = getIndexSettings(false, false, null, 1, 0).build(); + createIndex(indexName2, indexSettings2); + + final int numDocsInIndex1 = 5; + final int numDocsInIndex2 = 6; + indexDocuments(client, indexName1, numDocsInIndex1); + indexDocuments(client, indexName2, numDocsInIndex2); + ensureGreen(indexName1, indexName2); + + final String secondNode = internalCluster().startNode(); + + logger.info("--> snapshot"); + CreateSnapshotResponse createSnapshotResponse = client.admin() + .cluster() + .prepareCreateSnapshot(snapshotRepoName, snapshotName1) + .setWaitForCompletion(true) + .setIndices(indexName1, indexName2) + .get(); + assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), greaterThan(0)); + assertThat( + createSnapshotResponse.getSnapshotInfo().successfulShards(), + equalTo(createSnapshotResponse.getSnapshotInfo().totalShards()) + ); + assertThat(createSnapshotResponse.getSnapshotInfo().state(), equalTo(SnapshotState.SUCCESS)); + + Settings remoteStoreIndexSettings = Settings.builder() + .put(IndexMetadata.SETTING_REMOTE_STORE_REPOSITORY, remoteStoreRepo2Name) + .build(); + // restore index as a remote store index with different remote store repo + RestoreSnapshotResponse restoreSnapshotResponse = client.admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepoName, snapshotName1) + .setWaitForCompletion(false) + .setIndexSettings(remoteStoreIndexSettings) + .setIndices(indexName1) + .setRenamePattern(indexName1) + .setRenameReplacement(restoredIndexName1) + .get(); + assertEquals(restoreSnapshotResponse.status(), RestStatus.ACCEPTED); + ensureGreen(restoredIndexName1); + assertDocsPresentInIndex(client(), restoredIndexName1, numDocsInIndex1); + + // deleting data for restoredIndexName1 and restoring from remote store. + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(primaryNodeName(restoredIndexName1))); + assertAcked(client().admin().indices().prepareClose(restoredIndexName1)); + client().admin() + .cluster() + .restoreRemoteStore(new RestoreRemoteStoreRequest().indices(restoredIndexName1), PlainActionFuture.newFuture()); + ensureYellowAndNoInitializingShards(restoredIndexName1); + ensureGreen(restoredIndexName1); + // indexing some new docs and validating + assertDocsPresentInIndex(client, restoredIndexName1, numDocsInIndex1); + indexDocuments(client, restoredIndexName1, numDocsInIndex1, numDocsInIndex1 + 2); + ensureGreen(restoredIndexName1); + assertDocsPresentInIndex(client, restoredIndexName1, numDocsInIndex1 + 2); + } + + private Settings.Builder getIndexSettings( + boolean enableRemoteStore, + boolean enableRemoteTranslog, + String remoteStoreRepo, + int numOfShards, + int numOfReplicas + ) { + Settings.Builder settingsBuilder = Settings.builder() + .put(super.indexSettings()) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, numOfShards) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, numOfReplicas); + if (enableRemoteStore) { + settingsBuilder.put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, true) + .put(IndexMetadata.SETTING_REMOTE_STORE_REPOSITORY, remoteStoreRepo) + .put(IndexSettings.INDEX_REFRESH_INTERVAL_SETTING.getKey(), "300s") + .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT); + } + if (enableRemoteTranslog) { + settingsBuilder.put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_ENABLED, true) + .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, remoteStoreRepo) + .build(); + } + return settingsBuilder; + } + + public void testRestoreShallowSnapshotRepositoryOverriden() throws ExecutionException, InterruptedException { + String indexName1 = "testindex1"; + String snapshotRepoName = "test-restore-snapshot-repo"; + String remoteStoreRepoName = "test-rs-repo" + TEST_REMOTE_STORE_REPO_SUFFIX; + String remoteStoreRepoNameUpdated = "test-rs-repo-updated" + TEST_REMOTE_STORE_REPO_SUFFIX; + String snapshotName1 = "test-restore-snapshot1"; + Path absolutePath1 = randomRepoPath().toAbsolutePath(); + Path absolutePath2 = randomRepoPath().toAbsolutePath(); + Path absolutePath3 = randomRepoPath().toAbsolutePath(); + String[] pathTokens = absolutePath1.toString().split("/"); + String basePath = pathTokens[pathTokens.length - 1]; + Arrays.copyOf(pathTokens, pathTokens.length - 1); + Path location = PathUtils.get(String.join("/", pathTokens)); + pathTokens = absolutePath2.toString().split("/"); + String basePath2 = pathTokens[pathTokens.length - 1]; + Arrays.copyOf(pathTokens, pathTokens.length - 1); + Path location2 = PathUtils.get(String.join("/", pathTokens)); + logger.info("Path 1 [{}]", absolutePath1); + logger.info("Path 2 [{}]", absolutePath2); + logger.info("Path 3 [{}]", absolutePath3); + String restoredIndexName1 = indexName1 + "-restored"; + + createRepository(snapshotRepoName, "fs", getRepositorySettings(location, basePath, true)); + createRepository(remoteStoreRepoName, "fs", absolutePath3); + + Client client = client(); + Settings indexSettings = Settings.builder() + .put(super.indexSettings()) + .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, true) + .put(IndexMetadata.SETTING_REMOTE_STORE_REPOSITORY, remoteStoreRepoName) + .put(IndexSettings.INDEX_REFRESH_INTERVAL_SETTING.getKey(), "300s") + .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .build(); + createIndex(indexName1, indexSettings); + + int numDocsInIndex1 = randomIntBetween(2, 5); + indexDocuments(client, indexName1, numDocsInIndex1); + + ensureGreen(indexName1); + + logger.info("--> snapshot"); + CreateSnapshotResponse createSnapshotResponse = client.admin() + .cluster() + .prepareCreateSnapshot(snapshotRepoName, snapshotName1) + .setWaitForCompletion(true) + .setIndices(indexName1) + .get(); + assertThat(createSnapshotResponse.getSnapshotInfo().successfulShards(), greaterThan(0)); + assertThat( + createSnapshotResponse.getSnapshotInfo().successfulShards(), + equalTo(createSnapshotResponse.getSnapshotInfo().totalShards()) + ); + assertThat(createSnapshotResponse.getSnapshotInfo().state(), equalTo(SnapshotState.SUCCESS)); + + createRepository(remoteStoreRepoName, "fs", absolutePath2); + + RestoreSnapshotResponse restoreSnapshotResponse = client.admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepoName, snapshotName1) + .setWaitForCompletion(true) + .setIndices(indexName1) + .setRenamePattern(indexName1) + .setRenameReplacement(restoredIndexName1) + .get(); + + assertTrue(restoreSnapshotResponse.getRestoreInfo().failedShards() > 0); + + ensureRed(restoredIndexName1); + + client().admin().indices().close(Requests.closeIndexRequest(restoredIndexName1)).get(); + createRepository(remoteStoreRepoNameUpdated, "fs", absolutePath3); + RestoreSnapshotResponse restoreSnapshotResponse2 = client.admin() + .cluster() + .prepareRestoreSnapshot(snapshotRepoName, snapshotName1) + .setWaitForCompletion(true) + .setIndices(indexName1) + .setRenamePattern(indexName1) + .setRenameReplacement(restoredIndexName1) + .setSourceRemoteStoreRepository(remoteStoreRepoNameUpdated) + .get(); + + assertTrue(restoreSnapshotResponse2.getRestoreInfo().failedShards() == 0); + ensureGreen(restoredIndexName1); + assertDocsPresentInIndex(client, restoredIndexName1, numDocsInIndex1); + + // indexing some new docs and validating + indexDocuments(client, restoredIndexName1, numDocsInIndex1, numDocsInIndex1 + 2); + ensureGreen(restoredIndexName1); + assertDocsPresentInIndex(client, restoredIndexName1, numDocsInIndex1 + 2); + } + + private void indexDocuments(Client client, String indexName, int numOfDocs) { + indexDocuments(client, indexName, 0, numOfDocs); + } + + private void indexDocuments(Client client, String indexName, int fromId, int toId) { + for (int i = fromId; i < toId; i++) { + String id = Integer.toString(i); + client.prepareIndex(indexName).setId(id).setSource("text", "sometext").get(); + } + client.admin().indices().prepareFlush(indexName).get(); + } + + private void assertDocsPresentInIndex(Client client, String indexName, int numOfDocs) { + for (int i = 0; i < numOfDocs; i++) { + String id = Integer.toString(i); + logger.info("checking for index " + indexName + " with docId" + id); + assertTrue("doc with id" + id + " is not present for index " + indexName, client.prepareGet(indexName, id).get().isExists()); + } + } + public void testParallelRestoreOperationsFromSingleSnapshot() throws Exception { String indexName1 = "testindex1"; String indexName2 = "testindex2"; 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 e127b44116b7e..7ff16fcf6fc03 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 @@ -113,6 +113,8 @@ private static StorageType fromString(String string) { private Settings indexSettings = EMPTY_SETTINGS; private String[] ignoreIndexSettings = Strings.EMPTY_ARRAY; private StorageType storageType = StorageType.LOCAL; + @Nullable + private String sourceRemoteStoreRepository = null; @Nullable // if any snapshot UUID will do private String snapshotUuid; @@ -148,6 +150,9 @@ public RestoreSnapshotRequest(StreamInput in) throws IOException { if (in.getVersion().onOrAfter(Version.V_2_7_0)) { storageType = in.readEnum(StorageType.class); } + if (in.getVersion().onOrAfter(Version.V_3_0_0)) { + sourceRemoteStoreRepository = in.readOptionalString(); + } } @Override @@ -169,6 +174,9 @@ public void writeTo(StreamOutput out) throws IOException { if (out.getVersion().onOrAfter(Version.V_2_7_0)) { out.writeEnum(storageType); } + if (out.getVersion().onOrAfter(Version.V_3_0_0)) { + out.writeOptionalString(sourceRemoteStoreRepository); + } } @Override @@ -521,6 +529,25 @@ public StorageType storageType() { return storageType; } + /** + * Sets Source Remote Store Repository for all the restored indices + * + * @param sourceRemoteStoreRepository name of the remote store repository that should be used for all restored indices. + */ + public RestoreSnapshotRequest setSourceRemoteStoreRepository(String sourceRemoteStoreRepository) { + this.sourceRemoteStoreRepository = sourceRemoteStoreRepository; + return this; + } + + /** + * Returns Source Remote Store Repository for all the restored indices + * + * @return source Remote Store Repository + */ + public String getSourceRemoteStoreRepository() { + return sourceRemoteStoreRepository; + } + /** * Parses restore definition * @@ -586,6 +613,12 @@ public RestoreSnapshotRequest source(Map source) { throw new IllegalArgumentException("malformed storage_type"); } + } else if (name.equals("source_remote_store_repository")) { + if (entry.getValue() instanceof String) { + setSourceRemoteStoreRepository((String) entry.getValue()); + } else { + throw new IllegalArgumentException("malformed source_remote_store_repository"); + } } else { if (IndicesOptions.isIndicesOptions(name) == false) { throw new IllegalArgumentException("Unknown parameter " + name); @@ -631,6 +664,9 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (storageType != null) { storageType.toXContent(builder); } + if (sourceRemoteStoreRepository != null) { + builder.field("source_remote_store_repository", sourceRemoteStoreRepository); + } builder.endObject(); return builder; } @@ -658,7 +694,8 @@ public boolean equals(Object o) { && Objects.equals(indexSettings, that.indexSettings) && Arrays.equals(ignoreIndexSettings, that.ignoreIndexSettings) && Objects.equals(snapshotUuid, that.snapshotUuid) - && Objects.equals(storageType, that.storageType); + && Objects.equals(storageType, that.storageType) + && Objects.equals(sourceRemoteStoreRepository, that.sourceRemoteStoreRepository); } @Override @@ -675,7 +712,8 @@ public int hashCode() { includeAliases, indexSettings, snapshotUuid, - storageType + storageType, + sourceRemoteStoreRepository ); result = 31 * result + Arrays.hashCode(indices); result = 31 * result + Arrays.hashCode(ignoreIndexSettings); diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/restore/RestoreSnapshotRequestBuilder.java b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/restore/RestoreSnapshotRequestBuilder.java index 0104637a00035..d9cca536d1c41 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/restore/RestoreSnapshotRequestBuilder.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/snapshots/restore/RestoreSnapshotRequestBuilder.java @@ -256,4 +256,12 @@ public RestoreSnapshotRequestBuilder setStorageType(RestoreSnapshotRequest.Stora request.storageType(storageType); return this; } + + /** + * Sets the source remote store repository name + */ + public RestoreSnapshotRequestBuilder setSourceRemoteStoreRepository(String repositoryName) { + request.setSourceRemoteStoreRepository(repositoryName); + return this; + } } diff --git a/server/src/main/java/org/opensearch/cluster/routing/RecoverySource.java b/server/src/main/java/org/opensearch/cluster/routing/RecoverySource.java index ef86eb31e2817..1af4fbe8ffb45 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/RecoverySource.java +++ b/server/src/main/java/org/opensearch/cluster/routing/RecoverySource.java @@ -34,6 +34,7 @@ import org.opensearch.Version; import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.Nullable; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.common.io.stream.StreamOutput; import org.opensearch.common.io.stream.Writeable; @@ -257,9 +258,11 @@ public static class SnapshotRecoverySource extends RecoverySource { private final IndexId index; private final Version version; private final boolean isSearchableSnapshot; + private final boolean remoteStoreIndexShallowCopy; + private final String sourceRemoteStoreRepository; public SnapshotRecoverySource(String restoreUUID, Snapshot snapshot, Version version, IndexId indexId) { - this(restoreUUID, snapshot, version, indexId, false); + this(restoreUUID, snapshot, version, indexId, false, false, null); } public SnapshotRecoverySource( @@ -267,13 +270,17 @@ public SnapshotRecoverySource( Snapshot snapshot, Version version, IndexId indexId, - boolean isSearchableSnapshot + boolean isSearchableSnapshot, + boolean remoteStoreIndexShallowCopy, + @Nullable String sourceRemoteStoreRepository ) { this.restoreUUID = restoreUUID; this.snapshot = Objects.requireNonNull(snapshot); this.version = Objects.requireNonNull(version); this.index = Objects.requireNonNull(indexId); this.isSearchableSnapshot = isSearchableSnapshot; + this.remoteStoreIndexShallowCopy = remoteStoreIndexShallowCopy; + this.sourceRemoteStoreRepository = sourceRemoteStoreRepository; } SnapshotRecoverySource(StreamInput in) throws IOException { @@ -286,6 +293,13 @@ public SnapshotRecoverySource( } else { isSearchableSnapshot = false; } + if (in.getVersion().onOrAfter(Version.V_3_0_0)) { + remoteStoreIndexShallowCopy = in.readBoolean(); + sourceRemoteStoreRepository = in.readOptionalString(); + } else { + remoteStoreIndexShallowCopy = false; + sourceRemoteStoreRepository = null; + } } public String restoreUUID() { @@ -314,6 +328,14 @@ public boolean isSearchableSnapshot() { return isSearchableSnapshot; } + public String sourceRemoteStoreRepository() { + return sourceRemoteStoreRepository; + } + + public boolean remoteStoreIndexShallowCopy() { + return remoteStoreIndexShallowCopy; + } + @Override protected void writeAdditionalFields(StreamOutput out) throws IOException { out.writeString(restoreUUID); @@ -323,6 +345,10 @@ protected void writeAdditionalFields(StreamOutput out) throws IOException { if (out.getVersion().onOrAfter(Version.V_2_7_0)) { out.writeBoolean(isSearchableSnapshot); } + if (out.getVersion().onOrAfter(Version.V_3_0_0)) { + out.writeBoolean(remoteStoreIndexShallowCopy); + out.writeOptionalString(sourceRemoteStoreRepository); + } } @Override @@ -337,7 +363,9 @@ public void addAdditionalFields(XContentBuilder builder, ToXContent.Params param .field("version", version.toString()) .field("index", index.getName()) .field("restoreUUID", restoreUUID) - .field("isSearchableSnapshot", isSearchableSnapshot); + .field("isSearchableSnapshot", isSearchableSnapshot) + .field("remoteStoreIndexShallowCopy", remoteStoreIndexShallowCopy) + .field("sourceRemoteStoreRepository", sourceRemoteStoreRepository); } @Override @@ -359,12 +387,24 @@ public boolean equals(Object o) { && snapshot.equals(that.snapshot) && index.equals(that.index) && version.equals(that.version) - && isSearchableSnapshot == that.isSearchableSnapshot; + && isSearchableSnapshot == that.isSearchableSnapshot + && remoteStoreIndexShallowCopy == that.remoteStoreIndexShallowCopy + && sourceRemoteStoreRepository != null + ? sourceRemoteStoreRepository.equals(that.sourceRemoteStoreRepository) + : that.sourceRemoteStoreRepository == null; } @Override public int hashCode() { - return Objects.hash(restoreUUID, snapshot, index, version, isSearchableSnapshot); + return Objects.hash( + restoreUUID, + snapshot, + index, + version, + isSearchableSnapshot, + remoteStoreIndexShallowCopy, + sourceRemoteStoreRepository + ); } } diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index 9938d11caca13..d89d51c713d70 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -44,6 +44,7 @@ import org.apache.lucene.index.LeafReader; import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.index.Term; +import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.search.Query; import org.apache.lucene.search.QueryCachingPolicy; import org.apache.lucene.search.ReferenceManager; @@ -1495,7 +1496,7 @@ public GatedCloseable acquireLastIndexCommitAndRefresh(boolean flus * @throws IOException if there is some failure in acquiring lock in remote store. */ public void acquireLockOnCommitData(String snapshotId, long primaryTerm, long generation) throws IOException { - RemoteSegmentStoreDirectory remoteSegmentStoreDirectory = getRemoteSegmentDirectoryForShard(); + RemoteSegmentStoreDirectory remoteSegmentStoreDirectory = getRemoteDirectory(); remoteSegmentStoreDirectory.acquireLock(primaryTerm, generation, snapshotId); } @@ -1507,20 +1508,10 @@ public void acquireLockOnCommitData(String snapshotId, long primaryTerm, long ge * @throws IOException if there is some failure in releasing lock in remote store. */ public void releaseLockOnCommitData(String snapshotId, long primaryTerm, long generation) throws IOException { - RemoteSegmentStoreDirectory remoteSegmentStoreDirectory = getRemoteSegmentDirectoryForShard(); + RemoteSegmentStoreDirectory remoteSegmentStoreDirectory = getRemoteDirectory(); remoteSegmentStoreDirectory.releaseLock(primaryTerm, generation, snapshotId); } - private RemoteSegmentStoreDirectory getRemoteSegmentDirectoryForShard() { - FilterDirectory remoteStoreDirectory = (FilterDirectory) remoteStore.directory(); - assert remoteStoreDirectory.getDelegate() instanceof FilterDirectory - : "Store.directory is not enclosing an instance of FilterDirectory"; - FilterDirectory byteSizeCachingStoreDirectory = (FilterDirectory) remoteStoreDirectory.getDelegate(); - final Directory remoteDirectory = byteSizeCachingStoreDirectory.getDelegate(); - assert remoteDirectory instanceof RemoteSegmentStoreDirectory : "remoteDirectory is not an instance of RemoteSegmentStoreDirectory"; - return ((RemoteSegmentStoreDirectory) remoteDirectory); - } - public Optional getReplicationEngine() { if (getEngine() instanceof NRTReplicationEngine) { return Optional.of((NRTReplicationEngine) getEngine()); @@ -2290,7 +2281,24 @@ public void openEngineAndSkipTranslogRecovery() throws IOException { getEngine().translogManager().skipTranslogRecovery(); } + public void openEngineAndSkipTranslogRecoveryFromSnapshot() throws IOException { + assert routingEntry().recoverySource().getType() == RecoverySource.Type.SNAPSHOT : "not a snapshot recovery [" + + routingEntry() + + "]"; + recoveryState.validateCurrentStage(RecoveryState.Stage.INDEX); + maybeCheckIndex(); + recoveryState.setStage(RecoveryState.Stage.TRANSLOG); + recoveryState.validateCurrentStage(RecoveryState.Stage.TRANSLOG); + loadGlobalCheckpointToReplicationTracker(); + innerOpenEngineAndTranslog(replicationTracker, false); + getEngine().translogManager().skipTranslogRecovery(); + } + private void innerOpenEngineAndTranslog(LongSupplier globalCheckpointSupplier) throws IOException { + innerOpenEngineAndTranslog(globalCheckpointSupplier, true); + } + + private void innerOpenEngineAndTranslog(LongSupplier globalCheckpointSupplier, boolean syncFromRemote) throws IOException { assert Thread.holdsLock(mutex) == false : "opening engine under mutex"; if (state != IndexShardState.RECOVERING) { throw new IndexShardNotRecoveringException(shardId, state); @@ -2309,11 +2317,20 @@ private void innerOpenEngineAndTranslog(LongSupplier globalCheckpointSupplier) t synchronized (engineMutex) { assert currentEngineReference.get() == null : "engine is running"; verifyNotClosed(); - if (indexSettings.isRemoteStoreEnabled()) { + if (indexSettings.isRemoteStoreEnabled() && syncFromRemote) { syncSegmentsFromRemoteSegmentStore(false, true, true); } if (indexSettings.isRemoteTranslogStoreEnabled() && shardRouting.primary()) { - syncRemoteTranslogAndUpdateGlobalCheckpoint(); + if (syncFromRemote) { + syncRemoteTranslogAndUpdateGlobalCheckpoint(); + } else { + // we will enter this block when we do not want to recover from remote translog. + // currently only during snapshot restore, we are coming into this block. + // here, as while initiliazing remote translog we cannot skip downloading translog files, + // so before that step, we are deleting the translog files present in remote store. + deleteTranslogFilesFromRemoteTranslog(); + + } } // we must create a new engine under mutex (see IndexShard#snapshotStoreMetadata). final Engine newEngine = engineFactory.newReadWriteEngine(config); @@ -2605,6 +2622,22 @@ public void restoreFromRemoteStore(ActionListener listener) { storeRecovery.recoverFromRemoteStore(this, listener); } + public void restoreFromSnapshotAndRemoteStore( + Repository repository, + RepositoriesService repositoriesService, + ActionListener listener + ) { + try { + assert shardRouting.primary() : "recover from store only makes sense if the shard is a primary shard"; + assert recoveryState.getRecoverySource().getType() == RecoverySource.Type.SNAPSHOT : "invalid recovery type: " + + recoveryState.getRecoverySource(); + StoreRecovery storeRecovery = new StoreRecovery(shardId, logger); + storeRecovery.recoverFromSnapshotAndRemoteStore(this, repository, repositoriesService, listener); + } catch (Exception e) { + listener.onFailure(e); + } + } + public void restoreFromRepository(Repository repository, ActionListener listener) { try { assert shardRouting.primary() : "recover from store only makes sense if the shard is a primary shard"; @@ -3418,6 +3451,15 @@ public void startRecovery( final SnapshotRecoverySource recoverySource = (SnapshotRecoverySource) recoveryState.getRecoverySource(); if (recoverySource.isSearchableSnapshot()) { executeRecovery("from snapshot (remote)", recoveryState, recoveryListener, this::recoverFromStore); + } else if (recoverySource.remoteStoreIndexShallowCopy()) { + final String repo = recoverySource.snapshot().getRepository(); + executeRecovery( + "from snapshot and remote store", + recoveryState, + recoveryListener, + l -> restoreFromSnapshotAndRemoteStore(repositoriesService.repository(repo), repositoriesService, l) + ); + // indicesService.indexService(shardRouting.shardId().getIndex()).addMetadataListener(); } else { final String repo = recoverySource.snapshot().getRepository(); executeRecovery( @@ -4536,6 +4578,13 @@ private void syncRemoteTranslogAndUpdateGlobalCheckpoint() throws IOException { loadGlobalCheckpointToReplicationTracker(); } + public void deleteTranslogFilesFromRemoteTranslog() throws IOException { + TranslogFactory translogFactory = translogFactorySupplier.apply(indexSettings, shardRouting); + assert translogFactory instanceof RemoteBlobStoreInternalTranslogFactory; + Repository repository = ((RemoteBlobStoreInternalTranslogFactory) translogFactory).getRepository(); + RemoteFsTranslog.cleanup(repository, shardId, getThreadPool()); + } + public void syncTranslogFilesFromRemoteTranslog() throws IOException { TranslogFactory translogFactory = translogFactorySupplier.apply(indexSettings, shardRouting); assert translogFactory instanceof RemoteBlobStoreInternalTranslogFactory; @@ -4558,12 +4607,11 @@ public void syncSegmentsFromRemoteSegmentStore(boolean overrideLocal, boolean re // We need to call RemoteSegmentStoreDirectory.init() in order to get latest metadata of the files that // are uploaded to the remote segment store. RemoteSegmentMetadata remoteSegmentMetadata = remoteDirectory.init(); - Map uploadedSegments = ((RemoteSegmentStoreDirectory) remoteDirectory) + + Map uploadedSegments = remoteDirectory .getSegmentsUploadedToRemoteStore(); store.incRef(); remoteStore.incRef(); - List downloadedSegments = new ArrayList<>(); - List skippedSegments = new ArrayList<>(); try { final Directory storeDirectory; if (recoveryState.getStage() == RecoveryState.Stage.INDEX) { @@ -4580,18 +4628,7 @@ public void syncSegmentsFromRemoteSegmentStore(boolean overrideLocal, boolean re storeDirectory = store.directory(); } Set localSegmentFiles = Sets.newHashSet(storeDirectory.listAll()); - for (String file : uploadedSegments.keySet()) { - long checksum = Long.parseLong(uploadedSegments.get(file).getChecksum()); - if (overrideLocal || localDirectoryContains(storeDirectory, file, checksum) == false) { - if (localSegmentFiles.contains(file)) { - storeDirectory.deleteFile(file); - } - storeDirectory.copyFrom(remoteDirectory, file, file, IOContext.DEFAULT); - downloadedSegments.add(file); - } else { - skippedSegments.add(file); - } - } + copySegmentFiles(storeDirectory, remoteDirectory, null, uploadedSegments, overrideLocal); if (refreshLevelSegmentSync && remoteSegmentMetadata != null) { try ( @@ -4637,13 +4674,113 @@ public void syncSegmentsFromRemoteSegmentStore(boolean overrideLocal, boolean re } catch (IOException e) { throw new IndexShardRecoveryException(shardId, "Exception while copying segment files from remote segment store", e); } finally { - logger.info("Downloaded segments: {}", downloadedSegments); - logger.info("Skipped download for segments: {}", skippedSegments); store.decRef(); remoteStore.decRef(); } } + /** + * Downloads segments from given remote segment store for a specific commit. + * @param overrideLocal flag to override local segment files with those in remote store + * @param sourceRemoteDirectory RemoteSegmentDirectory Instance from which we need to sync segments + * @param primaryTerm Primary Term for shard at the time of commit operation for which we are syncing segments + * @param commitGeneration commit generation at the time of commit operation for which we are syncing segments + * @throws IOException if exception occurs while reading segments from remote store + */ + public void syncSegmentsFromGivenRemoteSegmentStore( + boolean overrideLocal, + RemoteSegmentStoreDirectory sourceRemoteDirectory, + long primaryTerm, + long commitGeneration + ) throws IOException { + logger.info("Downloading segments from given remote segment store"); + RemoteSegmentStoreDirectory remoteDirectory = null; + if (remoteStore != null) { + remoteDirectory = getRemoteDirectory(); + remoteDirectory.init(); + remoteStore.incRef(); + } + Map uploadedSegments = sourceRemoteDirectory + .initializeToSpecificCommit(primaryTerm, commitGeneration) + .getMetadata(); + final Directory storeDirectory = store.directory(); + store.incRef(); + + try { + String segmentsNFile = copySegmentFiles( + storeDirectory, + sourceRemoteDirectory, + remoteDirectory, + uploadedSegments, + overrideLocal + ); + if (segmentsNFile != null) { + try ( + ChecksumIndexInput indexInput = new BufferedChecksumIndexInput( + storeDirectory.openInput(segmentsNFile, IOContext.DEFAULT) + ) + ) { + SegmentInfos infosSnapshot = SegmentInfos.readCommit(store.directory(), indexInput, commitGeneration); + long processedLocalCheckpoint = Long.parseLong(infosSnapshot.getUserData().get(LOCAL_CHECKPOINT_KEY)); + if (remoteStore != null) { + store.commitSegmentInfos(infosSnapshot, processedLocalCheckpoint, processedLocalCheckpoint); + } else { + store.directory().sync(infosSnapshot.files(true)); + store.directory().syncMetaData(); + } + } + } + } catch (IOException e) { + throw new IndexShardRecoveryException(shardId, "Exception while copying segment files from remote segment store", e); + } finally { + store.decRef(); + if (remoteStore != null) { + remoteStore.decRef(); + } + } + } + + private String copySegmentFiles( + Directory storeDirectory, + RemoteSegmentStoreDirectory sourceRemoteDirectory, + RemoteSegmentStoreDirectory targetRemoteDirectory, + Map uploadedSegments, + boolean overrideLocal + ) throws IOException { + List downloadedSegments = new ArrayList<>(); + List skippedSegments = new ArrayList<>(); + String segmentNFile = null; + try { + Set localSegmentFiles = Sets.newHashSet(storeDirectory.listAll()); + if (overrideLocal) { + for (String file : localSegmentFiles) { + storeDirectory.deleteFile(file); + } + } + for (String file : uploadedSegments.keySet()) { + long checksum = Long.parseLong(uploadedSegments.get(file).getChecksum()); + if (overrideLocal || localDirectoryContains(storeDirectory, file, checksum) == false) { + storeDirectory.copyFrom(sourceRemoteDirectory, file, file, IOContext.DEFAULT); + storeDirectory.sync(Collections.singleton(file)); + downloadedSegments.add(file); + } else { + skippedSegments.add(file); + } + if (targetRemoteDirectory != null) { + targetRemoteDirectory.copyFrom(storeDirectory, file, file, IOContext.DEFAULT); + } + if (file.startsWith(IndexFileNames.SEGMENTS)) { + assert segmentNFile == null : "There should be only one SegmentInfosSnapshot file"; + segmentNFile = file; + } + } + } finally { + logger.info("Downloaded segments here: {}", downloadedSegments); + logger.info("Skipped download for segments here: {}", skippedSegments); + } + return segmentNFile; + } + private boolean localDirectoryContains(Directory localDirectory, String file, long checksum) { try (IndexInput indexInput = localDirectory.openInput(file, IOContext.DEFAULT)) { if (checksum == CodecUtil.retrieveChecksum(indexInput)) { diff --git a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java index d7f7373e83bd0..7cfaaafcadd39 100644 --- a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java +++ b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java @@ -29,6 +29,7 @@ import org.opensearch.index.remote.RemoteRefreshSegmentTracker; import org.opensearch.index.seqno.SequenceNumbers; import org.opensearch.index.store.RemoteSegmentStoreDirectory; +import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadata; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.indices.replication.checkpoint.SegmentReplicationCheckpointPublisher; import org.opensearch.threadpool.Scheduler; @@ -71,6 +72,8 @@ public final class RemoteStoreRefreshListener implements ReferenceManager.Refres */ private static final int REMOTE_REFRESH_RETRY_MAX_INTERVAL_MILLIS = 10_000; + private static final int INVALID_PRIMARY_TERM = -1; + /** * Exponential back off policy with max retry interval. */ @@ -118,15 +121,18 @@ public RemoteStoreRefreshListener( this.storeDirectory = indexShard.store().directory(); this.remoteDirectory = (RemoteSegmentStoreDirectory) ((FilterDirectory) ((FilterDirectory) indexShard.remoteStore().directory()) .getDelegate()).getDelegate(); - this.primaryTerm = indexShard.getOperationPrimaryTerm(); localSegmentChecksumMap = new HashMap<>(); + RemoteSegmentMetadata remoteSegmentMetadata = null; if (indexShard.routingEntry().primary()) { try { - this.remoteDirectory.init(); + remoteSegmentMetadata = this.remoteDirectory.init(); } catch (IOException e) { logger.error("Exception while initialising RemoteSegmentStoreDirectory", e); } } + // initializing primary term with the primary term of latest metadata in remote store. + // if no metadata is present, this value will be initilized with -1. + this.primaryTerm = remoteSegmentMetadata != null ? remoteSegmentMetadata.getPrimaryTerm() : INVALID_PRIMARY_TERM; this.segmentTracker = segmentTracker; resetBackOffDelayIterator(); this.checkpointPublisher = checkpointPublisher; @@ -163,8 +169,9 @@ public void beforeRefresh() throws IOException {} */ @Override public void afterRefresh(boolean didRefresh) { - - if (didRefresh || remoteDirectory.getSegmentsUploadedToRemoteStore().isEmpty()) { + if (this.primaryTerm != indexShard.getOperationPrimaryTerm() + || didRefresh + || remoteDirectory.getSegmentsUploadedToRemoteStore().isEmpty()) { updateLocalRefreshTimeAndSeqNo(); try { indexShard.getThreadPool().executor(ThreadPool.Names.REMOTE_REFRESH).submit(() -> syncSegments(false)).get(); 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 a81dc96ff1145..119524e8caf8a 100644 --- a/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java +++ b/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java @@ -59,14 +59,19 @@ import org.opensearch.index.mapper.MapperService; import org.opensearch.index.seqno.SequenceNumbers; import org.opensearch.index.snapshots.IndexShardRestoreFailedException; +import org.opensearch.index.snapshots.blobstore.RemoteStoreShardShallowCopySnapshot; +import org.opensearch.index.store.RemoteSegmentStoreDirectory; +import org.opensearch.index.store.RemoteSegmentStoreDirectoryFactory; import org.opensearch.index.store.Store; import org.opensearch.index.translog.Translog; import org.opensearch.indices.recovery.RecoveryState; import org.opensearch.indices.replication.common.ReplicationLuceneIndex; import org.opensearch.repositories.IndexId; +import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.Repository; import java.io.IOException; +import java.nio.channels.FileChannel; import java.util.Arrays; import java.util.HashMap; import java.util.List; @@ -347,6 +352,72 @@ void recoverFromRepository(final IndexShard indexShard, Repository repository, A } } + void recoverFromSnapshotAndRemoteStore( + final IndexShard indexShard, + Repository repository, + RepositoriesService repositoriesService, + ActionListener listener + ) { + try { + if (canRecover(indexShard)) { + indexShard.preRecovery(); + RecoverySource.Type recoveryType = indexShard.recoveryState().getRecoverySource().getType(); + assert recoveryType == RecoverySource.Type.SNAPSHOT : "expected snapshot recovery type: " + recoveryType; + SnapshotRecoverySource recoverySource = (SnapshotRecoverySource) indexShard.recoveryState().getRecoverySource(); + final RecoveryState.Translog translogState = indexShard.recoveryState().getTranslog(); + translogState.totalOperations(0); + translogState.totalOperationsOnStart(0); + indexShard.prepareForIndexRecovery(); + + RemoteStoreShardShallowCopySnapshot shallowCopyShardMetadata = repository.getRemoteStoreShallowCopyShardMetadata( + recoverySource.snapshot().getSnapshotId(), + recoverySource.index(), + shardId + ); + + long primaryTerm = shallowCopyShardMetadata.getPrimaryTerm(); + long commitGeneration = shallowCopyShardMetadata.getCommitGeneration(); + String indexUUID = shallowCopyShardMetadata.getIndexUUID(); + String remoteStoreRepository = ((SnapshotRecoverySource) indexShard.recoveryState().getRecoverySource()) + .sourceRemoteStoreRepository(); + if (remoteStoreRepository == null) { + remoteStoreRepository = shallowCopyShardMetadata.getRemoteStoreRepository(); + } + + RemoteSegmentStoreDirectoryFactory directoryFactory = new RemoteSegmentStoreDirectoryFactory(() -> repositoriesService); + RemoteSegmentStoreDirectory sourceRemoteDirectory = (RemoteSegmentStoreDirectory) directoryFactory.newDirectory( + remoteStoreRepository, + indexUUID, + String.valueOf(shardId.id()) + ); + indexShard.syncSegmentsFromGivenRemoteSegmentStore(true, sourceRemoteDirectory, primaryTerm, commitGeneration); + final Store store = indexShard.store(); + if (indexShard.indexSettings.isRemoteTranslogStoreEnabled() == false) { + bootstrap(indexShard, store); + } else { + bootstrapForSnapshot(indexShard, store); + } + assert indexShard.shardRouting.primary() : "only primary shards can recover from store"; + writeEmptyRetentionLeasesFile(indexShard); + indexShard.recoveryState().getIndex().setFileDetailsComplete(); + if (indexShard.indexSettings.isRemoteStoreEnabled()) { + indexShard.openEngineAndSkipTranslogRecoveryFromSnapshot(); + } else { + indexShard.openEngineAndRecoverFromTranslog(); + } + indexShard.getEngine().fillSeqNoGaps(indexShard.getPendingPrimaryTerm()); + indexShard.finalizeRecovery(); + indexShard.postRecovery("restore done"); + + listener.onResponse(true); + } else { + listener.onResponse(false); + } + } catch (Exception e) { + listener.onFailure(e); + } + } + private boolean canRecover(IndexShard indexShard) { if (indexShard.state() == IndexShardState.CLOSED) { // got closed on us, just ignore this recovery @@ -597,10 +668,18 @@ private void restore( } final ActionListener restoreListener = ActionListener.wrap(v -> { final Store store = indexShard.store(); - bootstrap(indexShard, store); + if (indexShard.indexSettings.isRemoteTranslogStoreEnabled() == false) { + bootstrap(indexShard, store); + } else { + bootstrapForSnapshot(indexShard, store); + } assert indexShard.shardRouting.primary() : "only primary shards can recover from store"; writeEmptyRetentionLeasesFile(indexShard); - indexShard.openEngineAndRecoverFromTranslog(); + if (indexShard.indexSettings.isRemoteStoreEnabled()) { + indexShard.openEngineAndSkipTranslogRecoveryFromSnapshot(); + } else { + indexShard.openEngineAndRecoverFromTranslog(); + } indexShard.getEngine().fillSeqNoGaps(indexShard.getPendingPrimaryTerm()); indexShard.finalizeRecovery(); indexShard.postRecovery("restore done"); @@ -644,6 +723,21 @@ private void restore( } } + private void bootstrapForSnapshot(final IndexShard indexShard, final Store store) throws IOException { + store.bootstrapNewHistory(); + final SegmentInfos segmentInfos = store.readLastCommittedSegmentsInfo(); + final long localCheckpoint = Long.parseLong(segmentInfos.userData.get(SequenceNumbers.LOCAL_CHECKPOINT_KEY)); + String translogUUID = store.readLastCommittedSegmentsInfo().getUserData().get(Translog.TRANSLOG_UUID_KEY); + Translog.createEmptyTranslog( + indexShard.shardPath().resolveTranslog(), + shardId, + localCheckpoint, + indexShard.getPendingPrimaryTerm(), + translogUUID, + FileChannel::open + ); + } + private void bootstrap(final IndexShard indexShard, final Store store) throws IOException { store.bootstrapNewHistory(); final SegmentInfos segmentInfos = store.readLastCommittedSegmentsInfo(); diff --git a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java index 15c6fbea99148..addd8a24af9c5 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java @@ -126,6 +126,24 @@ public RemoteSegmentMetadata init() throws IOException { return remoteSegmentMetadata; } + /** + * Initializes the cache to a specific commit which keeps track of all the segment files uploaded to the + * remote segment store. + * this is currently used to restore snapshots, where we want to copy segment files from a given commit. + * TODO: check if we can return read only RemoteSegmentStoreDirectory object from here. + * @throws IOException if there were any failures in reading the metadata file + */ + public RemoteSegmentMetadata initializeToSpecificCommit(long primaryTerm, long commitGeneration) throws IOException { + String metadataFile = getMetadataFileForCommit(primaryTerm, commitGeneration); + RemoteSegmentMetadata remoteSegmentMetadata = readMetadataFile(metadataFile); + if (remoteSegmentMetadata != null) { + this.segmentsUploadedToRemoteStore = new ConcurrentHashMap<>(remoteSegmentMetadata.getMetadata()); + } else { + this.segmentsUploadedToRemoteStore = new ConcurrentHashMap<>(); + } + return remoteSegmentMetadata; + } + /** * Read the latest metadata file to get the list of segments uploaded to the remote segment store. * We upload a metadata file per refresh, but it is not unique per refresh. Refresh metadata file is unique for a given commit. @@ -485,6 +503,7 @@ public void uploadMetadata( new RemoteSegmentMetadata( RemoteSegmentMetadata.fromMapOfStrings(uploadedSegments), segmentInfoSnapshotByteArray, + primaryTerm, segmentInfosSnapshot.getGeneration() ) ); diff --git a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java index 388f80ea3e480..03995d5913fb3 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java @@ -43,6 +43,11 @@ public Directory newDirectory(IndexSettings indexSettings, ShardPath path) throw String repositoryName = indexSettings.getRemoteStoreRepository(); String indexUUID = indexSettings.getIndex().getUUID(); String shardId = String.valueOf(path.getShardId().getId()); + + return newDirectory(repositoryName, indexUUID, shardId); + } + + public Directory newDirectory(String repositoryName, String indexUUID, String shardId) throws IOException { try (Repository repository = repositoriesService.get().repository(repositoryName)) { assert repository instanceof BlobStoreRepository : "repository should be instance of BlobStoreRepository"; BlobPath commonBlobPath = ((BlobStoreRepository) repository).basePath(); diff --git a/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java b/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java index 2a84fbfb89c93..9a479346ff711 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java +++ b/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java @@ -38,16 +38,19 @@ public class RemoteSegmentMetadata { private final byte[] segmentInfosBytes; + private final long primaryTerm; private final long generation; public RemoteSegmentMetadata( Map metadata, byte[] segmentInfosBytes, + long primaryTerm, long generation ) { this.metadata = metadata; this.segmentInfosBytes = segmentInfosBytes; this.generation = generation; + this.primaryTerm = primaryTerm; } /** @@ -66,6 +69,10 @@ public long getGeneration() { return generation; } + public long getPrimaryTerm() { + return primaryTerm; + } + /** * Generate {@code Map} from {@link RemoteSegmentMetadata} * @return {@code Map} @@ -93,6 +100,7 @@ public static Map f public void write(IndexOutput out) throws IOException { out.writeMapOfStrings(toMapOfStrings()); out.writeLong(generation); + out.writeLong(primaryTerm); out.writeLong(segmentInfosBytes.length); out.writeBytes(segmentInfosBytes, segmentInfosBytes.length); } @@ -100,9 +108,10 @@ public void write(IndexOutput out) throws IOException { public static RemoteSegmentMetadata read(IndexInput indexInput) throws IOException { Map metadata = indexInput.readMapOfStrings(); long generation = indexInput.readLong(); + long primaryTerm = indexInput.readLong(); int byteArraySize = (int) indexInput.readLong(); byte[] segmentInfosBytes = new byte[byteArraySize]; indexInput.readBytes(segmentInfosBytes, 0, byteArraySize); - return new RemoteSegmentMetadata(RemoteSegmentMetadata.fromMapOfStrings(metadata), segmentInfosBytes, generation); + return new RemoteSegmentMetadata(RemoteSegmentMetadata.fromMapOfStrings(metadata), segmentInfosBytes, primaryTerm, generation); } } diff --git a/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java b/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java index 6ebb1bf7d2252..04057b581e8d9 100644 --- a/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java +++ b/server/src/main/java/org/opensearch/index/translog/RemoteFsTranslog.java @@ -423,6 +423,20 @@ private void deleteStaleRemotePrimaryTermsAndMetadataFiles() { } } + public static void cleanup(Repository repository, ShardId shardId, ThreadPool threadPool) throws IOException { + assert repository instanceof BlobStoreRepository : "repository should be instance of BlobStoreRepository"; + BlobStoreRepository blobStoreRepository = (BlobStoreRepository) repository; + FileTransferTracker fileTransferTracker = new FileTransferTracker(shardId); + TranslogTransferManager translogTransferManager = buildTranslogTransferManager( + blobStoreRepository, + threadPool, + shardId, + fileTransferTracker + ); + // clean up all remote translog files + translogTransferManager.deleteTranslogFiles(); + } + protected void onDelete() { if (primaryModeSupplier.getAsBoolean() == false) { logger.trace("skipped delete translog"); diff --git a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java index 58aca00d2e9d3..f6405bc9b5c82 100644 --- a/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java +++ b/server/src/main/java/org/opensearch/index/translog/transfer/TranslogTransferManager.java @@ -361,6 +361,11 @@ public void onFailure(Exception e) { }); } + public void deleteTranslogFiles() throws IOException { + transferService.delete(remoteMetadataTransferPath); + transferService.delete(remoteDataTransferPath); + } + /** * Deletes list of translog files asynchronously using the {@code REMOTE_PURGE} threadpool. * diff --git a/server/src/main/java/org/opensearch/indices/cluster/IndicesClusterStateService.java b/server/src/main/java/org/opensearch/indices/cluster/IndicesClusterStateService.java index 4a0fab82f9adc..e4b251914fa0b 100644 --- a/server/src/main/java/org/opensearch/indices/cluster/IndicesClusterStateService.java +++ b/server/src/main/java/org/opensearch/indices/cluster/IndicesClusterStateService.java @@ -107,6 +107,7 @@ import java.util.function.BiConsumer; import java.util.function.Consumer; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_STORE_ENABLED; import static org.opensearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason.CLOSED; import static org.opensearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason.DELETED; import static org.opensearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason.FAILURE; @@ -544,7 +545,19 @@ private void createIndices(final ClusterState state) { AllocatedIndex indexService = null; try { - indexService = indicesService.createIndex(indexMetadata, builtInIndexListener, true); + List updatedIndexEventListeners = new ArrayList<>(builtInIndexListener); + if (entry.getValue().size() > 0 + && entry.getValue().get(0).recoverySource().getType() == Type.SNAPSHOT + && indexMetadata.getSettings().getAsBoolean(SETTING_REMOTE_STORE_ENABLED, false)) { + final IndexEventListener refreshListenerAfterSnapshotRestore = new IndexEventListener() { + @Override + public void afterIndexShardStarted(IndexShard indexShard) { + indexShard.refresh("refresh to upload metadata to remote store"); + } + }; + updatedIndexEventListeners.add(refreshListenerAfterSnapshotRestore); + } + indexService = indicesService.createIndex(indexMetadata, updatedIndexEventListeners, true); if (indexService.updateMapping(null, indexMetadata) && sendRefreshMapping) { nodeMappingRefreshAction.nodeMappingRefresh( state.nodes().getClusterManagerNode(), diff --git a/server/src/main/java/org/opensearch/repositories/FilterRepository.java b/server/src/main/java/org/opensearch/repositories/FilterRepository.java index 88e14a4dff3a0..b108e2da1ab04 100644 --- a/server/src/main/java/org/opensearch/repositories/FilterRepository.java +++ b/server/src/main/java/org/opensearch/repositories/FilterRepository.java @@ -45,6 +45,7 @@ import org.opensearch.index.mapper.MapperService; import org.opensearch.index.shard.ShardId; import org.opensearch.index.snapshots.IndexShardSnapshotStatus; +import org.opensearch.index.snapshots.blobstore.RemoteStoreShardShallowCopySnapshot; import org.opensearch.index.store.Store; import org.opensearch.indices.recovery.RecoveryState; import org.opensearch.snapshots.SnapshotId; @@ -219,6 +220,15 @@ public void restoreShard( in.restoreShard(store, snapshotId, indexId, snapshotShardId, recoveryState, listener); } + @Override + public RemoteStoreShardShallowCopySnapshot getRemoteStoreShallowCopyShardMetadata( + SnapshotId snapshotId, + IndexId indexId, + ShardId snapshotShardId + ) { + return in.getRemoteStoreShallowCopyShardMetadata(snapshotId, indexId, snapshotShardId); + } + @Override public IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, IndexId indexId, ShardId shardId) { return in.getShardSnapshotStatus(snapshotId, indexId, shardId); diff --git a/server/src/main/java/org/opensearch/repositories/Repository.java b/server/src/main/java/org/opensearch/repositories/Repository.java index 045b7ad348a76..c08369b79452d 100644 --- a/server/src/main/java/org/opensearch/repositories/Repository.java +++ b/server/src/main/java/org/opensearch/repositories/Repository.java @@ -46,6 +46,7 @@ import org.opensearch.index.mapper.MapperService; import org.opensearch.index.shard.ShardId; import org.opensearch.index.snapshots.IndexShardSnapshotStatus; +import org.opensearch.index.snapshots.blobstore.RemoteStoreShardShallowCopySnapshot; import org.opensearch.index.store.Store; import org.opensearch.indices.recovery.RecoveryState; import org.opensearch.snapshots.SnapshotId; @@ -304,6 +305,22 @@ void restoreShard( ActionListener listener ); + /** + * Returns Snapshot Shard Metadata for remote store interop enabled snapshot. + *

+ * The index can be renamed on restore, hence different {@code shardId} and {@code snapshotShardId} are supplied. + * @param snapshotId snapshot id + * @param indexId id of the index in the repository from which the restore is occurring + * @param snapshotShardId shard id (in the snapshot) + */ + default RemoteStoreShardShallowCopySnapshot getRemoteStoreShallowCopyShardMetadata( + SnapshotId snapshotId, + IndexId indexId, + ShardId snapshotShardId + ) { + throw new UnsupportedOperationException(); + } + /** * Retrieve shard snapshot status for the stored snapshot * diff --git a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java index c0d6f49a5ce0d..f04bf83c2f1d1 100644 --- a/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/opensearch/repositories/blobstore/BlobStoreRepository.java @@ -2789,6 +2789,16 @@ public InputStream maybeRateLimitSnapshots(InputStream stream) { return maybeRateLimit(stream, () -> snapshotRateLimiter, snapshotRateLimitingTimeInNanos); } + @Override + public RemoteStoreShardShallowCopySnapshot getRemoteStoreShallowCopyShardMetadata( + SnapshotId snapshotId, + IndexId indexId, + ShardId snapshotShardId + ) { + final BlobContainer container = shardContainer(indexId, snapshotShardId); + return loadShallowCopyShardSnapshot(container, snapshotId); + } + @Override public IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, IndexId indexId, ShardId shardId) { BlobStoreIndexShardSnapshot snapshot = loadShardSnapshot(shardContainer(indexId, shardId), snapshotId); diff --git a/server/src/main/java/org/opensearch/snapshots/RestoreService.java b/server/src/main/java/org/opensearch/snapshots/RestoreService.java index 3d4b04889a5c9..bd162914e830e 100644 --- a/server/src/main/java/org/opensearch/snapshots/RestoreService.java +++ b/server/src/main/java/org/opensearch/snapshots/RestoreService.java @@ -450,12 +450,25 @@ public ClusterState execute(ClusterState currentState) { final boolean isSearchableSnapshot = IndexModule.Type.REMOTE_SNAPSHOT.match( snapshotIndexMetadata.getSettings().get(IndexModule.INDEX_STORE_TYPE_SETTING.getKey()) ); + final boolean isRemoteStoreShallowCopy = Boolean.TRUE.equals( + snapshotInfo.isRemoteStoreIndexShallowCopyEnabled() + ) && metadata.index(index).getSettings().getAsBoolean(SETTING_REMOTE_STORE_ENABLED, false); + if (isRemoteStoreShallowCopy && !currentState.getNodes().getMinNodeVersion().onOrAfter(Version.V_3_0_0)) { + throw new SnapshotRestoreException( + snapshot, + "cannot restore shallow copy snapshot for index [" + + index + + "] as some of the nodes in cluster have version less than 2.9" + ); + } final SnapshotRecoverySource recoverySource = new SnapshotRecoverySource( restoreUUID, snapshot, snapshotInfo.version(), repositoryData.resolveIndexId(index), - isSearchableSnapshot + isSearchableSnapshot, + isRemoteStoreShallowCopy, + request.getSourceRemoteStoreRepository() ); final Version minIndexCompatibilityVersion; if (isSearchableSnapshot && isSearchableSnapshotsExtendedCompatibilityEnabled()) { diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/snapshots/restore/RestoreSnapshotRequestTests.java b/server/src/test/java/org/opensearch/action/admin/cluster/snapshots/restore/RestoreSnapshotRequestTests.java index 737e7b2e4887b..bb55ac810ed09 100644 --- a/server/src/test/java/org/opensearch/action/admin/cluster/snapshots/restore/RestoreSnapshotRequestTests.java +++ b/server/src/test/java/org/opensearch/action/admin/cluster/snapshots/restore/RestoreSnapshotRequestTests.java @@ -112,6 +112,10 @@ private RestoreSnapshotRequest randomState(RestoreSnapshotRequest instance) { instance.snapshotUuid(randomBoolean() ? null : randomAlphaOfLength(10)); } + if (randomBoolean()) { + instance.setSourceRemoteStoreRepository(randomAlphaOfLengthBetween(5, 10)); + } + return instance; } diff --git a/server/src/test/java/org/opensearch/cluster/routing/RecoverySourceTests.java b/server/src/test/java/org/opensearch/cluster/routing/RecoverySourceTests.java index e4aae52f41e68..a5c006362a20c 100644 --- a/server/src/test/java/org/opensearch/cluster/routing/RecoverySourceTests.java +++ b/server/src/test/java/org/opensearch/cluster/routing/RecoverySourceTests.java @@ -36,6 +36,8 @@ import org.opensearch.common.UUIDs; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.repositories.IndexId; +import org.opensearch.snapshots.Snapshot; +import org.opensearch.snapshots.SnapshotId; import org.opensearch.test.OpenSearchTestCase; import java.io.IOException; @@ -54,6 +56,28 @@ public void testSerialization() throws IOException { assertEquals(recoverySource, serializedRecoverySource); } + public void testSerializationSnapshotRecoverySource() throws IOException { + boolean isSearchableSnapshot = randomBoolean(); + boolean isRemoteStoreShallowCopyEnabled = randomBoolean(); + String sourceRemoteStoreRepo = "test-remote-repo"; + RecoverySource.SnapshotRecoverySource recoverySource = new RecoverySource.SnapshotRecoverySource( + UUIDs.randomBase64UUID(), + new Snapshot("repo", new SnapshotId(randomAlphaOfLength(8), UUIDs.randomBase64UUID())), + Version.CURRENT, + new IndexId("some_index", UUIDs.randomBase64UUID(random())), + isSearchableSnapshot, + isRemoteStoreShallowCopyEnabled, + sourceRemoteStoreRepo + ); + BytesStreamOutput out = new BytesStreamOutput(); + recoverySource.writeTo(out); + RecoverySource serializedRecoverySource = RecoverySource.readFrom(out.bytes().streamInput()); + assertEquals(recoverySource.getType(), serializedRecoverySource.getType()); + assertEquals(recoverySource, serializedRecoverySource); + assertEquals(recoverySource.remoteStoreIndexShallowCopy(), isRemoteStoreShallowCopyEnabled); + assertEquals(recoverySource.isSearchableSnapshot(), isSearchableSnapshot); + } + public void testRecoverySourceTypeOrder() { assertEquals(RecoverySource.Type.EMPTY_STORE.ordinal(), 0); assertEquals(RecoverySource.Type.EXISTING_STORE.ordinal(), 1); diff --git a/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java index 1c0ebf17285f7..58527dbea5791 100644 --- a/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/IndexShardTests.java @@ -37,6 +37,7 @@ import org.apache.lucene.index.IndexCommit; import org.apache.lucene.index.IndexableField; import org.apache.lucene.index.Term; +import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.search.ReferenceManager; import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TopDocs; @@ -49,6 +50,7 @@ import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.Constants; import org.junit.Assert; +import org.opensearch.common.io.PathUtils; import org.opensearch.core.Assertions; import org.opensearch.OpenSearchException; import org.opensearch.Version; @@ -127,6 +129,7 @@ import org.opensearch.index.seqno.RetentionLeases; import org.opensearch.index.seqno.SeqNoStats; import org.opensearch.index.seqno.SequenceNumbers; +import org.opensearch.index.store.RemoteSegmentStoreDirectory; import org.opensearch.index.store.Store; import org.opensearch.index.store.StoreStats; import org.opensearch.index.store.StoreUtils; @@ -188,7 +191,7 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.Stream; - +import java.util.Collection; import static java.util.Collections.emptyMap; import static java.util.Collections.emptySet; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -2797,6 +2800,78 @@ public void restoreShard( closeShards(target); } + public void testSyncSegmentsFromGivenRemoteSegmentStore() throws IOException { + String remoteStorePath = createTempDir().toString(); + IndexShard source = newStartedShard( + true, + Settings.builder() + .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) + .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, true) + .put(IndexMetadata.SETTING_REMOTE_STORE_REPOSITORY, remoteStorePath + "__test") + .build(), + new InternalEngineFactory() + ); + indexDoc(source, "_doc", "1"); + indexDoc(source, "_doc", "2"); + source.refresh("test"); + assertDocs(source, "1", "2"); + indexDoc(source, "_doc", "3"); + source.refresh("test"); + flushShard(source); + + indexDoc(source, "_doc", "5"); + source.refresh("test"); + + indexDoc(source, "_doc", "4"); + source.refresh("test"); + + long primaryTerm; + long commitGeneration; + try (GatedCloseable segmentInfosGatedCloseable = source.getSegmentInfosSnapshot()) { + SegmentInfos segmentInfos = segmentInfosGatedCloseable.get(); + primaryTerm = source.getOperationPrimaryTerm(); + commitGeneration = segmentInfos.getGeneration(); + } + Collection lastCommitedSegmentsInSource = SegmentInfos.readLatestCommit(source.store().directory()).files(false); + + closeShards(source); + + RemoteSegmentStoreDirectory tempRemoteSegmentDirectory = createRemoteSegmentStoreDirectory( + source.shardId(), + PathUtils.get(remoteStorePath) + ); + + IndexShard target = newStartedShard( + true, + Settings.builder() + .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) + .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, true) + .build(), + new InternalEngineFactory() + ); + ShardRouting routing = ShardRoutingHelper.initWithSameId( + target.routingEntry(), + RecoverySource.ExistingStoreRecoverySource.INSTANCE + ); + routing = ShardRoutingHelper.newWithRestoreSource(routing, new RecoverySource.EmptyStoreRecoverySource()); + + target = reinitShard(target, routing); + + target.syncSegmentsFromGivenRemoteSegmentStore(false, tempRemoteSegmentDirectory, primaryTerm, commitGeneration); + RemoteSegmentStoreDirectory remoteStoreDirectory = ((RemoteSegmentStoreDirectory) ((FilterDirectory) ((FilterDirectory) target + .remoteStore() + .directory()).getDelegate()).getDelegate()); + Collection uploadFiles = remoteStoreDirectory.getSegmentsUploadedToRemoteStore().keySet(); + assertTrue(uploadFiles.containsAll(lastCommitedSegmentsInSource)); + assertTrue( + "Failed to sync all files to new shard", + List.of(target.store().directory().listAll()).containsAll(lastCommitedSegmentsInSource) + ); + Directory storeDirectory = ((FilterDirectory) ((FilterDirectory) target.store().directory()).getDelegate()).getDelegate(); + ((BaseDirectoryWrapper) storeDirectory).setCheckIndexOnClose(false); + closeShards(target); + } + public void testRefreshLevelRestoreShardFromRemoteStore() throws IOException { testRestoreShardFromRemoteStore(false); } diff --git a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java index fec9b04d6e371..3417e7b0aee04 100644 --- a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java @@ -228,7 +228,8 @@ private Map getDummyMetadata(String prefix, int commitGeneration * @return ByteArrayIndexInput: metadata file bytes with header and footer * @throws IOException IOException */ - private ByteArrayIndexInput createMetadataFileBytes(Map segmentFilesMap, long generation) throws IOException { + private ByteArrayIndexInput createMetadataFileBytes(Map segmentFilesMap, long generation, long primaryTerm) + throws IOException { ByteBuffersDataOutput byteBuffersIndexOutput = new ByteBuffersDataOutput(); segmentInfos.write(new ByteBuffersIndexOutput(byteBuffersIndexOutput, "", "")); byte[] byteArray = byteBuffersIndexOutput.toArrayCopy(); @@ -238,6 +239,7 @@ private ByteArrayIndexInput createMetadataFileBytes(Map segmentF CodecUtil.writeHeader(indexOutput, RemoteSegmentMetadata.METADATA_CODEC, RemoteSegmentMetadata.CURRENT_VERSION); indexOutput.writeMapOfStrings(segmentFilesMap); indexOutput.writeLong(generation); + indexOutput.writeLong(primaryTerm); indexOutput.writeLong(byteArray.length); indexOutput.writeBytes(byteArray, byteArray.length); CodecUtil.writeFooter(indexOutput); @@ -261,14 +263,14 @@ private Map> populateMetadata() throws IOException { ); when(remoteMetadataDirectory.openInput("metadata__1__5__abc", IOContext.DEFAULT)).thenReturn( - createMetadataFileBytes(metadataFilenameContentMapping.get("metadata__1__5__abc"), 1) + createMetadataFileBytes(metadataFilenameContentMapping.get("metadata__1__5__abc"), 1, 5) ); when(remoteMetadataDirectory.openInput("metadata__1__6__pqr", IOContext.DEFAULT)).thenReturn( - createMetadataFileBytes(metadataFilenameContentMapping.get("metadata__1__6__pqr"), 1) + createMetadataFileBytes(metadataFilenameContentMapping.get("metadata__1__6__pqr"), 1, 6) ); when(remoteMetadataDirectory.openInput("metadata__2__1__zxv", IOContext.DEFAULT)).thenReturn( - createMetadataFileBytes(metadataFilenameContentMapping.get("metadata__2__1__zxv"), 1), - createMetadataFileBytes(metadataFilenameContentMapping.get("metadata__2__1__zxv"), 1) + createMetadataFileBytes(metadataFilenameContentMapping.get("metadata__2__1__zxv"), 1, 2), + createMetadataFileBytes(metadataFilenameContentMapping.get("metadata__2__1__zxv"), 1, 2) ); return metadataFilenameContentMapping; @@ -503,7 +505,7 @@ public void testGetSegmentsUploadedToRemoteStore() throws IOException { ); when(remoteMetadataDirectory.openInput("metadata__1__5__abc", IOContext.DEFAULT)).thenReturn( - createMetadataFileBytes(metadataFilenameContentMapping.get("metadata__1__5__abc"), 1) + createMetadataFileBytes(metadataFilenameContentMapping.get("metadata__1__5__abc"), 1, 5) ); assert (remoteSegmentStoreDirectory.getSegmentsUploadedToRemoteStore(testPrimaryTerm, testGeneration).containsKey("segments_5")); @@ -577,7 +579,9 @@ public void testContainsFile() throws IOException { metadata.put("_0.cfe", "_0.cfe::_0.cfe__" + UUIDs.base64UUID() + "::1234::512"); metadata.put("_0.cfs", "_0.cfs::_0.cfs__" + UUIDs.base64UUID() + "::2345::1024"); - when(remoteMetadataDirectory.openInput("metadata__1__5__abc", IOContext.DEFAULT)).thenReturn(createMetadataFileBytes(metadata, 1)); + when(remoteMetadataDirectory.openInput("metadata__1__5__abc", IOContext.DEFAULT)).thenReturn( + createMetadataFileBytes(metadata, 1, 5) + ); remoteSegmentStoreDirectory.init(); diff --git a/server/src/test/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerTests.java b/server/src/test/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerTests.java index 3bf7781fb909f..cc0764a6700b1 100644 --- a/server/src/test/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerTests.java +++ b/server/src/test/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerTests.java @@ -61,6 +61,7 @@ public void testReadContentNoSegmentInfos() throws IOException { Map expectedOutput = getDummyData(); indexOutput.writeMapOfStrings(expectedOutput); indexOutput.writeLong(1234); + indexOutput.writeLong(1234); indexOutput.writeLong(0); indexOutput.writeBytes(new byte[0], 0); indexOutput.close(); @@ -77,6 +78,7 @@ public void testReadContentWithSegmentInfos() throws IOException { Map expectedOutput = getDummyData(); indexOutput.writeMapOfStrings(expectedOutput); indexOutput.writeLong(1234); + indexOutput.writeLong(1234); ByteBuffersIndexOutput segmentInfosOutput = new ByteBuffersIndexOutput(new ByteBuffersDataOutput(), "test", "resource"); segmentInfos.write(segmentInfosOutput); byte[] segmentInfosBytes = segmentInfosOutput.toArrayCopy(); @@ -103,6 +105,7 @@ public void testWriteContent() throws IOException { RemoteSegmentMetadata remoteSegmentMetadata = new RemoteSegmentMetadata( RemoteSegmentMetadata.fromMapOfStrings(expectedOutput), segmentInfosBytes, + 1234, 1234 ); remoteSegmentMetadataHandler.writeContent(indexOutput, remoteSegmentMetadata); @@ -113,6 +116,7 @@ public void testWriteContent() throws IOException { ); assertEquals(expectedOutput, metadata.toMapOfStrings()); assertEquals(1234, metadata.getGeneration()); + assertEquals(1234, metadata.getPrimaryTerm()); assertArrayEquals(segmentInfosBytes, metadata.getSegmentInfosBytes()); } diff --git a/server/src/test/java/org/opensearch/repositories/RepositoriesServiceTests.java b/server/src/test/java/org/opensearch/repositories/RepositoriesServiceTests.java index 43d371bf5a187..f5295bead19a4 100644 --- a/server/src/test/java/org/opensearch/repositories/RepositoriesServiceTests.java +++ b/server/src/test/java/org/opensearch/repositories/RepositoriesServiceTests.java @@ -58,6 +58,7 @@ import org.opensearch.index.mapper.MapperService; import org.opensearch.index.shard.ShardId; import org.opensearch.index.snapshots.IndexShardSnapshotStatus; +import org.opensearch.index.snapshots.blobstore.RemoteStoreShardShallowCopySnapshot; import org.opensearch.index.store.Store; import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.indices.recovery.RecoveryState; @@ -342,6 +343,15 @@ public void restoreShard( } + @Override + public RemoteStoreShardShallowCopySnapshot getRemoteStoreShallowCopyShardMetadata( + SnapshotId snapshotId, + IndexId indexId, + ShardId snapshotShardId + ) { + return null; + } + @Override public IndexShardSnapshotStatus getShardSnapshotStatus(SnapshotId snapshotId, IndexId indexId, ShardId shardId) { return null; diff --git a/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryTests.java b/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryTests.java index 8e0ee6b16ed48..105ccef500ce8 100644 --- a/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryTests.java +++ b/server/src/test/java/org/opensearch/repositories/blobstore/BlobStoreRepositoryTests.java @@ -50,7 +50,12 @@ import org.opensearch.core.xcontent.NamedXContentRegistry; import org.opensearch.env.Environment; import org.opensearch.index.IndexModule; +import org.opensearch.index.IndexService; +import org.opensearch.index.IndexSettings; +import org.opensearch.index.shard.ShardId; +import org.opensearch.index.snapshots.blobstore.RemoteStoreShardShallowCopySnapshot; import org.opensearch.index.store.RemoteBufferedOutputDirectory; +import org.opensearch.indices.IndicesService; import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.plugins.Plugin; @@ -117,11 +122,7 @@ protected void assertSnapshotOrGenericThread() { @Override protected Settings nodeSettings() { - return Settings.builder() - .put(super.nodeSettings()) - .put(FeatureFlags.SEGMENT_REPLICATION_EXPERIMENTAL, "true") - .put(FeatureFlags.REMOTE_STORE, "true") - .build(); + return Settings.builder().put(super.nodeSettings()).put(FeatureFlags.REMOTE_STORE, "true").build(); } public void testRetrieveSnapshots() throws Exception { @@ -326,12 +327,89 @@ public void testRetrieveShallowCopySnapshotCase1() throws IOException { final RepositoriesService repositoriesService = getInstanceFromNode(RepositoriesService.class); final BlobStoreRepository repository = (BlobStoreRepository) repositoriesService.repository(snapshotRepositoryName); - List snapshotIds = OpenSearchBlobStoreRepositoryIntegTestCase.getRepositoryData(repository) - .getSnapshotIds() + RepositoryData repositoryData = OpenSearchBlobStoreRepositoryIntegTestCase.getRepositoryData(repository); + IndexId indexId = repositoryData.resolveIndexId(remoteStoreIndexName); + + List snapshotIds = repositoryData.getSnapshotIds() .stream() .sorted((s1, s2) -> s1.getName().compareTo(s2.getName())) .collect(Collectors.toList()); assertThat(snapshotIds, equalTo(originalSnapshots)); + + // shallow copy shard metadata - getRemoteStoreShallowCopyShardMetadata + RemoteStoreShardShallowCopySnapshot shardShallowCopySnapshot = repository.getRemoteStoreShallowCopyShardMetadata( + snapshotId2, + indexId, + new ShardId(remoteStoreIndexName, indexId.getId(), 0) + ); + assertEquals(shardShallowCopySnapshot.getRemoteStoreRepository(), remoteStoreRepositoryName); + } + + public void testGetRemoteStoreShallowCopyShardMetadata() throws IOException { + FeatureFlagSetter.set(FeatureFlags.REMOTE_STORE); + final Client client = client(); + final String snapshotRepositoryName = "test-repo"; + final String remoteStoreRepositoryName = "test-rs-repo"; + + logger.info("--> creating snapshot repository"); + + Settings snapshotRepoSettings = Settings.builder() + .put(node().settings()) + .put("location", OpenSearchIntegTestCase.randomRepoPath(node().settings())) + .build(); + createRepository(client, snapshotRepositoryName, snapshotRepoSettings); + + logger.info("--> creating remote store repository"); + Settings remoteStoreRepoSettings = Settings.builder() + .put(node().settings()) + .put("location", OpenSearchIntegTestCase.randomRepoPath(node().settings())) + .build(); + createRepository(client, remoteStoreRepositoryName, remoteStoreRepoSettings); + + logger.info("--> creating a remote store enabled index and indexing documents"); + final String remoteStoreIndexName = "test-rs-idx"; + Settings indexSettings = getRemoteStoreBackedIndexSettings(remoteStoreRepositoryName); + createIndex(remoteStoreIndexName, indexSettings); + indexDocuments(client, remoteStoreIndexName); + + logger.info("--> create remote index shallow snapshot"); + Settings snapshotRepoSettingsForShallowCopy = Settings.builder() + .put(snapshotRepoSettings) + .put(BlobStoreRepository.REMOTE_STORE_INDEX_SHALLOW_COPY.getKey(), Boolean.TRUE) + .build(); + updateRepository(client, snapshotRepositoryName, snapshotRepoSettingsForShallowCopy); + + CreateSnapshotResponse createSnapshotResponse = client.admin() + .cluster() + .prepareCreateSnapshot(snapshotRepositoryName, "test-snap-2") + .setWaitForCompletion(true) + .setIndices(remoteStoreIndexName) + .get(); + final SnapshotId snapshotId = createSnapshotResponse.getSnapshotInfo().snapshotId(); + + String[] lockFiles = getLockFilesInRemoteStore(remoteStoreIndexName, remoteStoreRepositoryName); + assert (lockFiles.length == 1) : "there should be only one lock file, but found " + Arrays.toString(lockFiles); + assert lockFiles[0].endsWith(snapshotId.getUUID() + ".lock"); + + final RepositoriesService repositoriesService = getInstanceFromNode(RepositoriesService.class); + final BlobStoreRepository repository = (BlobStoreRepository) repositoriesService.repository(snapshotRepositoryName); + RepositoryData repositoryData = OpenSearchBlobStoreRepositoryIntegTestCase.getRepositoryData(repository); + IndexSettings indexSetting = getIndexSettings(remoteStoreIndexName); + IndexId indexId = repositoryData.resolveIndexId(remoteStoreIndexName); + RemoteStoreShardShallowCopySnapshot shardShallowCopySnapshot = repository.getRemoteStoreShallowCopyShardMetadata( + snapshotId, + indexId, + new ShardId(remoteStoreIndexName, indexSetting.getUUID(), 0) + ); + assertEquals(shardShallowCopySnapshot.getRemoteStoreRepository(), remoteStoreRepositoryName); + assertEquals(shardShallowCopySnapshot.getIndexUUID(), indexSetting.getUUID()); + assertEquals(shardShallowCopySnapshot.getRepositoryBasePath(), ""); + } + + private IndexSettings getIndexSettings(String indexName) { + final IndicesService indicesService = getInstanceFromNode(IndicesService.class); + final IndexService indexService = indicesService.indexService(resolveIndex(indexName)); + return indexService.getIndexSettings(); } // Validate Scenario remoteStoreShallowCopy Snapshot -> remoteStoreShallowCopy Snapshot diff --git a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java index 659f473403ec8..ea9e9342673db 100644 --- a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java @@ -67,6 +67,7 @@ import org.opensearch.common.blobstore.fs.FsBlobStore; import org.opensearch.common.bytes.BytesArray; import org.opensearch.common.concurrent.GatedCloseable; +import org.opensearch.common.io.PathUtils; import org.opensearch.common.lucene.uid.Versions; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; @@ -101,6 +102,9 @@ import org.opensearch.index.store.RemoteSegmentStoreDirectory; import org.opensearch.index.store.Store; import org.opensearch.index.store.StoreFileMetadata; +import org.opensearch.index.store.lockmanager.RemoteStoreLockManager; +import org.opensearch.index.store.lockmanager.RemoteStoreMetadataLockManager; +import org.opensearch.index.store.RemoteBufferedOutputDirectory; import org.opensearch.index.translog.InternalTranslogFactory; import org.opensearch.index.translog.RemoteBlobStoreInternalTranslogFactory; import org.opensearch.index.translog.Translog; @@ -574,7 +578,14 @@ protected IndexShard newShard( RemoteRefreshSegmentPressureService remoteRefreshSegmentPressureService = null; if (indexSettings.isRemoteStoreEnabled()) { if (remoteStore == null) { - remoteStore = createRemoteStore(createTempDir(), routing, indexMetadata); + Path remoteStorePath; + String remoteStoreRepository = indexSettings.getRemoteStoreRepository(); + if (remoteStoreRepository != null && remoteStoreRepository.endsWith("__test")) { + remoteStorePath = PathUtils.get(remoteStoreRepository.replace("__test", "")); + } else { + remoteStorePath = createTempDir(); + } + remoteStore = createRemoteStore(remoteStorePath, routing, indexMetadata); } remoteRefreshSegmentPressureService = new RemoteRefreshSegmentPressureService(clusterService, indexSettings.getSettings()); } @@ -642,21 +653,30 @@ protected RepositoriesService createRepositoriesService() { protected Store createRemoteStore(Path path, ShardRouting shardRouting, IndexMetadata metadata) throws IOException { Settings nodeSettings = Settings.builder().put("node.name", shardRouting.currentNodeId()).build(); + ShardId shardId = shardRouting.shardId(); + RemoteSegmentStoreDirectory remoteSegmentStoreDirectory = createRemoteSegmentStoreDirectory(shardId, path); + return createStore(shardId, new IndexSettings(metadata, nodeSettings), remoteSegmentStoreDirectory); + } - ShardId shardId = new ShardId("index", "_na_", 0); + protected RemoteSegmentStoreDirectory createRemoteSegmentStoreDirectory(ShardId shardId, Path path) throws IOException { NodeEnvironment.NodePath remoteNodePath = new NodeEnvironment.NodePath(path); ShardPath remoteShardPath = new ShardPath(false, remoteNodePath.resolve(shardId), remoteNodePath.resolve(shardId), shardId); RemoteDirectory dataDirectory = newRemoteDirectory(remoteShardPath.resolveIndex()); RemoteDirectory metadataDirectory = newRemoteDirectory(remoteShardPath.resolveIndex()); - RemoteSegmentStoreDirectory remoteSegmentStoreDirectory = new RemoteSegmentStoreDirectory(dataDirectory, metadataDirectory, null); - return createStore(shardId, new IndexSettings(metadata, nodeSettings), remoteSegmentStoreDirectory); + RemoteStoreLockManager remoteStoreLockManager = new RemoteStoreMetadataLockManager( + new RemoteBufferedOutputDirectory(getBlobContainer(remoteShardPath.resolveIndex())) + ); + return new RemoteSegmentStoreDirectory(dataDirectory, metadataDirectory, remoteStoreLockManager); } private RemoteDirectory newRemoteDirectory(Path f) throws IOException { + return new RemoteDirectory(getBlobContainer(f)); + } + + protected BlobContainer getBlobContainer(Path f) throws IOException { FsBlobStore fsBlobStore = new FsBlobStore(1024, f, false); BlobPath blobPath = new BlobPath(); - BlobContainer fsBlobContainer = new FsBlobContainer(fsBlobStore, blobPath, f); - return new RemoteDirectory(fsBlobContainer); + return new FsBlobContainer(fsBlobStore, blobPath, f); } /** 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 28660ba834a65..ad515f2405f1d 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 @@ -240,15 +240,18 @@ private static void assertSnapshotUUIDs(BlobStoreRepository repository, Reposito final BlobContainer repoRoot = repository.blobContainer(); final Collection snapshotIds = repositoryData.getSnapshotIds(); final List expectedSnapshotUUIDs = snapshotIds.stream().map(SnapshotId::getUUID).collect(Collectors.toList()); - for (String prefix : new String[] { BlobStoreRepository.SNAPSHOT_PREFIX, BlobStoreRepository.METADATA_PREFIX }) { - final Collection foundSnapshotUUIDs = repoRoot.listBlobs() - .keySet() - .stream() - .filter(p -> p.startsWith(prefix)) - .map(p -> p.replace(prefix, "").replace(".dat", "")) - .collect(Collectors.toSet()); - assertThat(foundSnapshotUUIDs, containsInAnyOrder(expectedSnapshotUUIDs.toArray(Strings.EMPTY_ARRAY))); + Collection foundSnapshotUUIDs = new HashSet<>(); + for (String prefix : new String[] { BlobStoreRepository.SNAPSHOT_PREFIX, BlobStoreRepository.SHALLOW_SNAPSHOT_PREFIX }) { + foundSnapshotUUIDs.addAll( + repoRoot.listBlobs() + .keySet() + .stream() + .filter(p -> p.startsWith(prefix)) + .map(p -> p.replace(prefix, "").replace(".dat", "")) + .collect(Collectors.toSet()) + ); } + assertThat(foundSnapshotUUIDs, containsInAnyOrder(expectedSnapshotUUIDs.toArray(Strings.EMPTY_ARRAY))); final BlobContainer indicesContainer = repository.getBlobContainer().children().get("indices"); final Map indices; @@ -303,10 +306,16 @@ private static void assertSnapshotUUIDs(BlobStoreRepository repository, Reposito .stream() .noneMatch(shardFailure -> shardFailure.index().equals(index) && shardFailure.shardId() == shardId)) { final Map shardPathContents = shardContainer.listBlobs(); - assertThat( - shardPathContents, - hasKey(String.format(Locale.ROOT, BlobStoreRepository.SNAPSHOT_NAME_FORMAT, snapshotId.getUUID())) + + assertTrue( + shardPathContents.containsKey( + String.format(Locale.ROOT, BlobStoreRepository.SHALLOW_SNAPSHOT_NAME_FORMAT, snapshotId.getUUID()) + ) + || shardPathContents.containsKey( + String.format(Locale.ROOT, BlobStoreRepository.SNAPSHOT_NAME_FORMAT, snapshotId.getUUID()) + ) ); + assertThat( shardPathContents.keySet() .stream() diff --git a/test/framework/src/main/java/org/opensearch/snapshots/AbstractSnapshotIntegTestCase.java b/test/framework/src/main/java/org/opensearch/snapshots/AbstractSnapshotIntegTestCase.java index 9933297aa1c96..ddf9f3e96b9b4 100644 --- a/test/framework/src/main/java/org/opensearch/snapshots/AbstractSnapshotIntegTestCase.java +++ b/test/framework/src/main/java/org/opensearch/snapshots/AbstractSnapshotIntegTestCase.java @@ -105,6 +105,7 @@ public abstract class AbstractSnapshotIntegTestCase extends OpenSearchIntegTestCase { + protected final static String TEST_REMOTE_STORE_REPO_SUFFIX = "__rs"; private static final String OLD_VERSION_SNAPSHOT_PREFIX = "old-version-snapshot-"; // Large snapshot pool settings to set up nodes for tests involving multiple repositories that need to have enough @@ -148,14 +149,19 @@ public void verifyNoLeakedListeners() throws Exception { @After public void assertRepoConsistency() { if (skipRepoConsistencyCheckReason == null) { - clusterAdmin().prepareGetRepositories().get().repositories().forEach(repositoryMetadata -> { - final String name = repositoryMetadata.name(); - if (repositoryMetadata.settings().getAsBoolean("readonly", false) == false) { - clusterAdmin().prepareDeleteSnapshot(name, OLD_VERSION_SNAPSHOT_PREFIX + "*").get(); - clusterAdmin().prepareCleanupRepository(name).get(); - } - BlobStoreTestUtil.assertRepoConsistency(internalCluster(), name); - }); + clusterAdmin().prepareGetRepositories() + .get() + .repositories() + .stream() + .filter(repositoryMetadata -> !repositoryMetadata.name().endsWith(TEST_REMOTE_STORE_REPO_SUFFIX)) + .forEach(repositoryMetadata -> { + final String name = repositoryMetadata.name(); + if (repositoryMetadata.settings().getAsBoolean("readonly", false) == false) { + clusterAdmin().prepareDeleteSnapshot(name, OLD_VERSION_SNAPSHOT_PREFIX + "*").get(); + clusterAdmin().prepareCleanupRepository(name).get(); + } + BlobStoreTestUtil.assertRepoConsistency(internalCluster(), name); + }); } else { logger.info("--> skipped repo consistency checks because [{}]", skipRepoConsistencyCheckReason); } @@ -367,10 +373,36 @@ protected void createRepository(String repoName, String type, Settings.Builder s assertAcked(clusterAdmin().preparePutRepository(repoName).setType(type).setSettings(settings)); } + protected void updateRepository(String repoName, String type, Settings.Builder settings) { + logger.info("--> updating repository [{}] [{}]", repoName, type); + assertAcked(clusterAdmin().preparePutRepository(repoName).setType(type).setSettings(settings)); + } + protected void createRepository(String repoName, String type, Path location) { createRepository(repoName, type, Settings.builder().put("location", location)); } + protected Settings.Builder getRepositorySettings(Path location, boolean shallowCopyEnabled) { + Settings.Builder settingsBuilder = randomRepositorySettings(); + settingsBuilder.put("location", location); + if (shallowCopyEnabled) { + settingsBuilder.put(BlobStoreRepository.REMOTE_STORE_INDEX_SHALLOW_COPY.getKey(), true); + } + return settingsBuilder; + } + + protected Settings.Builder getRepositorySettings(Path location, String basePath, boolean shallowCopyEnabled) { + Settings.Builder settingsBuilder = randomRepositorySettings(); + settingsBuilder.put("location", location); + if (shallowCopyEnabled) { + settingsBuilder.put(BlobStoreRepository.REMOTE_STORE_INDEX_SHALLOW_COPY.getKey(), true); + } + if (basePath != null) { + settingsBuilder.put("base_path", basePath); + } + return settingsBuilder; + } + protected void createRepository(String repoName, String type) { createRepository(repoName, type, randomRepositorySettings()); } From 3947cb48faf1522fb0e90d234bc228161be11714 Mon Sep 17 00:00:00 2001 From: Kunal Kotwani Date: Tue, 27 Jun 2023 16:28:14 -0700 Subject: [PATCH 076/109] Remove minimum file cache size restriction (#8294) Signed-off-by: Kunal Kotwani --- CHANGELOG.md | 1 + .../remote/filecache/FileCacheFactory.java | 18 +----- ...TransportClearIndicesCacheActionTests.java | 2 +- .../filecache/FileCacheCleanerTests.java | 2 +- .../remote/filecache/FileCacheTests.java | 55 ++++++++----------- 5 files changed, 29 insertions(+), 49 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 50e1fe78daf5d..0ffb14d6750c5 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -133,6 +133,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), ### Removed - Remove `COMPRESSOR` variable from `CompressorFactory` and use `DEFLATE_COMPRESSOR` instead ([7907](https://github.com/opensearch-project/OpenSearch/pull/7907)) +- Remove concurrency based minimum file cache size restriction ([#8294](https://github.com/opensearch-project/OpenSearch/pull/8294)) ### Fixed - Fixing error: adding a new/forgotten parameter to the configuration for checking the config on startup in plugins/repository-s3 #7924 diff --git a/server/src/main/java/org/opensearch/index/store/remote/filecache/FileCacheFactory.java b/server/src/main/java/org/opensearch/index/store/remote/filecache/FileCacheFactory.java index 5ed1f5e0ee21b..f23e057196096 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/filecache/FileCacheFactory.java +++ b/server/src/main/java/org/opensearch/index/store/remote/filecache/FileCacheFactory.java @@ -11,7 +11,6 @@ import org.opensearch.common.breaker.CircuitBreaker; import org.opensearch.common.cache.RemovalReason; import org.opensearch.index.store.remote.utils.cache.SegmentedCache; -import org.opensearch.index.store.remote.file.OnDemandBlockSnapshotIndexInput; import java.nio.file.Files; import java.nio.file.Path; @@ -39,24 +38,11 @@ public class FileCacheFactory { public static FileCache createConcurrentLRUFileCache(long capacity, CircuitBreaker circuitBreaker) { - return createFileCache(createDefaultBuilder().capacity(capacity).build(), circuitBreaker); + return new FileCache(createDefaultBuilder().capacity(capacity).build(), circuitBreaker); } public static FileCache createConcurrentLRUFileCache(long capacity, int concurrencyLevel, CircuitBreaker circuitBreaker) { - return createFileCache(createDefaultBuilder().capacity(capacity).concurrencyLevel(concurrencyLevel).build(), circuitBreaker); - } - - private static FileCache createFileCache(SegmentedCache segmentedCache, CircuitBreaker circuitBreaker) { - /* - * Since OnDemandBlockSnapshotIndexInput.Builder.DEFAULT_BLOCK_SIZE is not overridden then it will be upper bound for max IndexInput - * size on disk. A single IndexInput size should always be more than a single segment in segmented cache. A FileCache capacity might - * be defined with large capacity (> IndexInput block size) but due to segmentation and concurrency factor, that capacity is - * distributed equally across segments. - */ - if (segmentedCache.getPerSegmentCapacity() <= OnDemandBlockSnapshotIndexInput.Builder.DEFAULT_BLOCK_SIZE) { - throw new IllegalStateException("FileSystem Cache per segment capacity is less than single IndexInput default block size"); - } - return new FileCache(segmentedCache, circuitBreaker); + return new FileCache(createDefaultBuilder().capacity(capacity).concurrencyLevel(concurrencyLevel).build(), circuitBreaker); } private static SegmentedCache.Builder createDefaultBuilder() { diff --git a/server/src/test/java/org/opensearch/action/admin/indices/cache/clear/TransportClearIndicesCacheActionTests.java b/server/src/test/java/org/opensearch/action/admin/indices/cache/clear/TransportClearIndicesCacheActionTests.java index 85aa60d6f308b..57a41f08f73f2 100644 --- a/server/src/test/java/org/opensearch/action/admin/indices/cache/clear/TransportClearIndicesCacheActionTests.java +++ b/server/src/test/java/org/opensearch/action/admin/indices/cache/clear/TransportClearIndicesCacheActionTests.java @@ -85,7 +85,7 @@ public void testOnShardOperation() throws IOException { when(shardRouting.shardId()).thenReturn(shardId); final ShardPath shardPath = ShardPath.loadFileCachePath(nodeEnvironment, shardId); final Path cacheEntryPath = shardPath.getDataPath(); - final FileCache fileCache = FileCacheFactory.createConcurrentLRUFileCache(1024 * 1024 * 1024, 16, new NoopCircuitBreaker("")); + final FileCache fileCache = FileCacheFactory.createConcurrentLRUFileCache(1024 * 1024, 16, new NoopCircuitBreaker("")); when(testNode.fileCache()).thenReturn(fileCache); when(testNode.getNodeEnvironment()).thenReturn(nodeEnvironment); diff --git a/server/src/test/java/org/opensearch/index/store/remote/filecache/FileCacheCleanerTests.java b/server/src/test/java/org/opensearch/index/store/remote/filecache/FileCacheCleanerTests.java index 3924cd4ed1913..5bdba0513af23 100644 --- a/server/src/test/java/org/opensearch/index/store/remote/filecache/FileCacheCleanerTests.java +++ b/server/src/test/java/org/opensearch/index/store/remote/filecache/FileCacheCleanerTests.java @@ -48,7 +48,7 @@ public class FileCacheCleanerTests extends OpenSearchTestCase { ); private final FileCache fileCache = FileCacheFactory.createConcurrentLRUFileCache( - 1024 * 1024 * 1024, + 1024 * 1024, 1, new NoopCircuitBreaker(CircuitBreaker.REQUEST) ); diff --git a/server/src/test/java/org/opensearch/index/store/remote/filecache/FileCacheTests.java b/server/src/test/java/org/opensearch/index/store/remote/filecache/FileCacheTests.java index 43a5c04b59f83..02b6a48b6f48e 100644 --- a/server/src/test/java/org/opensearch/index/store/remote/filecache/FileCacheTests.java +++ b/server/src/test/java/org/opensearch/index/store/remote/filecache/FileCacheTests.java @@ -31,7 +31,6 @@ public class FileCacheTests extends OpenSearchTestCase { // but fatal to these tests private final static int CONCURRENCY_LEVEL = 16; private final static int MEGA_BYTES = 1024 * 1024; - private final static int GIGA_BYTES = 1024 * 1024 * 1024; private final static String FAKE_PATH_SUFFIX = "Suffix"; private Path path; @@ -66,15 +65,9 @@ private void createFile(String indexName, String shardId, String fileName) throw Files.write(filePath, "test-data".getBytes()); } - public void testCreateCacheWithSmallSegments() { - assertThrows(IllegalStateException.class, () -> { - FileCacheFactory.createConcurrentLRUFileCache(1000, CONCURRENCY_LEVEL, new NoopCircuitBreaker(CircuitBreaker.REQUEST)); - }); - } - // test get method public void testGet() { - FileCache fileCache = createFileCache(GIGA_BYTES); + FileCache fileCache = createFileCache(8 * MEGA_BYTES); for (int i = 0; i < 4; i++) { fileCache.put(createPath(Integer.toString(i)), new StubCachedIndexInput(8 * MEGA_BYTES)); } @@ -86,27 +79,27 @@ public void testGet() { public void testGetThrowException() { assertThrows(NullPointerException.class, () -> { - FileCache fileCache = createFileCache(GIGA_BYTES); + FileCache fileCache = createFileCache(MEGA_BYTES); fileCache.get(null); }); } public void testPutThrowException() { assertThrows(NullPointerException.class, () -> { - FileCache fileCache = createFileCache(GIGA_BYTES); + FileCache fileCache = createFileCache(MEGA_BYTES); fileCache.put(null, null); }); } public void testPutThrowCircuitBreakingException() { - FileCache fileCache = createCircuitBreakingFileCache(GIGA_BYTES); + FileCache fileCache = createCircuitBreakingFileCache(MEGA_BYTES); Path path = createPath("0"); assertThrows(CircuitBreakingException.class, () -> fileCache.put(path, new StubCachedIndexInput(8 * MEGA_BYTES))); assertNull(fileCache.get(path)); } public void testCompute() { - FileCache fileCache = createFileCache(GIGA_BYTES); + FileCache fileCache = createFileCache(MEGA_BYTES); Path path = createPath("0"); fileCache.put(path, new StubCachedIndexInput(8 * MEGA_BYTES)); fileCache.incRef(path); @@ -117,20 +110,20 @@ public void testCompute() { public void testComputeThrowException() { assertThrows(NullPointerException.class, () -> { - FileCache fileCache = createFileCache(GIGA_BYTES); + FileCache fileCache = createFileCache(MEGA_BYTES); fileCache.compute(null, null); }); } public void testComputeThrowCircuitBreakingException() { - FileCache fileCache = createCircuitBreakingFileCache(GIGA_BYTES); + FileCache fileCache = createCircuitBreakingFileCache(MEGA_BYTES); Path path = createPath("0"); assertThrows(CircuitBreakingException.class, () -> fileCache.compute(path, (p, i) -> new StubCachedIndexInput(8 * MEGA_BYTES))); assertNull(fileCache.get(path)); } public void testRemove() { - FileCache fileCache = createFileCache(GIGA_BYTES); + FileCache fileCache = createFileCache(MEGA_BYTES); for (int i = 0; i < 4; i++) { fileCache.put(createPath(Integer.toString(i)), new StubCachedIndexInput(8 * MEGA_BYTES)); } @@ -145,13 +138,13 @@ public void testRemove() { public void testRemoveThrowException() { assertThrows(NullPointerException.class, () -> { - FileCache fileCache = createFileCache(GIGA_BYTES); + FileCache fileCache = createFileCache(MEGA_BYTES); fileCache.remove(null); }); } public void testIncDecRef() { - FileCache fileCache = createFileCache(GIGA_BYTES); + FileCache fileCache = createFileCache(MEGA_BYTES); for (int i = 0; i < 4; i++) { fileCache.put(createPath(Integer.toString(i)), new StubCachedIndexInput(8 * MEGA_BYTES)); } @@ -184,7 +177,7 @@ public void testIncDecRef() { public void testIncRefThrowException() { assertThrows(NullPointerException.class, () -> { - FileCache fileCache = createFileCache(GIGA_BYTES); + FileCache fileCache = createFileCache(MEGA_BYTES); fileCache.incRef(null); }); @@ -192,19 +185,19 @@ public void testIncRefThrowException() { public void testDecRefThrowException() { assertThrows(NullPointerException.class, () -> { - FileCache fileCache = createFileCache(GIGA_BYTES); + FileCache fileCache = createFileCache(MEGA_BYTES); fileCache.decRef(null); }); } public void testCapacity() { - FileCache fileCache = createFileCache(GIGA_BYTES); - assertEquals(fileCache.capacity(), GIGA_BYTES); + FileCache fileCache = createFileCache(MEGA_BYTES); + assertEquals(fileCache.capacity(), MEGA_BYTES); } public void testSize() { - FileCache fileCache = createFileCache(GIGA_BYTES); + FileCache fileCache = createFileCache(MEGA_BYTES); for (int i = 0; i < 4; i++) { fileCache.put(createPath(Integer.toString(i)), new StubCachedIndexInput(8 * MEGA_BYTES)); } @@ -213,34 +206,34 @@ public void testSize() { } public void testPrune() { - FileCache fileCache = createFileCache(GIGA_BYTES); + FileCache fileCache = createFileCache(MEGA_BYTES); for (int i = 0; i < 4; i++) { putAndDecRef(fileCache, i, 8 * MEGA_BYTES); } // before prune - assertEquals(fileCache.size(), 4); + assertTrue(fileCache.size() >= 1); fileCache.prune(); // after prune - assertEquals(fileCache.size(), 0); + assertEquals(0, fileCache.size()); } public void testPruneWithPredicate() { - FileCache fileCache = createFileCache(GIGA_BYTES); + FileCache fileCache = createFileCache(MEGA_BYTES); for (int i = 0; i < 4; i++) { putAndDecRef(fileCache, i, 8 * MEGA_BYTES); } // before prune - assertEquals(fileCache.size(), 4); + assertTrue(fileCache.size() >= 1); // after prune with false predicate fileCache.prune(path -> false); - assertEquals(fileCache.size(), 4); + assertTrue(fileCache.size() >= 1); // after prune with true predicate fileCache.prune(path -> true); - assertEquals(fileCache.size(), 0); + assertEquals(0, fileCache.size()); } public void testUsage() { @@ -258,7 +251,7 @@ public void testUsage() { } public void testStats() { - FileCache fileCache = createFileCache(GIGA_BYTES); + FileCache fileCache = createFileCache(MEGA_BYTES); for (int i = 0; i < 4; i++) { fileCache.put(createPath(Integer.toString(i)), new StubCachedIndexInput(8 * MEGA_BYTES)); } @@ -284,7 +277,7 @@ public void testCacheRestore() throws IOException { String indexName = "test-index"; String shardId = "0"; createFile(indexName, shardId, "test.0"); - FileCache fileCache = createFileCache(GIGA_BYTES); + FileCache fileCache = createFileCache(MEGA_BYTES); assertEquals(0, fileCache.usage().usage()); Path fileCachePath = path.resolve(NodeEnvironment.CACHE_FOLDER).resolve(indexName).resolve(shardId); fileCache.restoreFromDirectory(List.of(fileCachePath)); From 4c02dd185c3033c6fabf0c52b8b39a019d324ca4 Mon Sep 17 00:00:00 2001 From: Owais Kazi Date: Tue, 27 Jun 2023 16:33:39 -0700 Subject: [PATCH 077/109] Removed redundant ubuntu precommit check (#8207) * Run precommit before gradle check Signed-off-by: Owais Kazi * Removed redundant precommit for ubuntu Signed-off-by: Owais Kazi * Addressed PR comment Signed-off-by: Owais Kazi --------- Signed-off-by: Owais Kazi --- .github/workflows/gradle-check.yml | 88 +++++++++++++++--------------- .github/workflows/precommit.yml | 4 +- 2 files changed, 46 insertions(+), 46 deletions(-) diff --git a/.github/workflows/gradle-check.yml b/.github/workflows/gradle-check.yml index ba1dc3dc3b5e6..f895dfc2c1f4d 100644 --- a/.github/workflows/gradle-check.yml +++ b/.github/workflows/gradle-check.yml @@ -22,34 +22,34 @@ jobs: timeout-minutes: 130 steps: - name: Checkout OpenSearch repo - uses: actions/checkout@v2 + uses: actions/checkout@v3 with: ref: ${{ github.event.pull_request.head.sha }} - name: Setup environment variables (PR) if: github.event_name == 'pull_request_target' run: | - echo "pr_from_sha=$(jq --raw-output .pull_request.head.sha $GITHUB_EVENT_PATH)" >> $GITHUB_ENV - echo "pr_from_clone_url=$(jq --raw-output .pull_request.head.repo.clone_url $GITHUB_EVENT_PATH)" >> $GITHUB_ENV - echo "pr_to_clone_url=$(jq --raw-output .pull_request.base.repo.clone_url $GITHUB_EVENT_PATH)" >> $GITHUB_ENV - echo "pr_title=$(jq --raw-output .pull_request.title $GITHUB_EVENT_PATH)" >> $GITHUB_ENV - echo "pr_number=$(jq --raw-output .pull_request.number $GITHUB_EVENT_PATH)" >> $GITHUB_ENV + echo "pr_from_sha=$(jq --raw-output .pull_request.head.sha $GITHUB_EVENT_PATH)" >> $GITHUB_ENV + echo "pr_from_clone_url=$(jq --raw-output .pull_request.head.repo.clone_url $GITHUB_EVENT_PATH)" >> $GITHUB_ENV + echo "pr_to_clone_url=$(jq --raw-output .pull_request.base.repo.clone_url $GITHUB_EVENT_PATH)" >> $GITHUB_ENV + echo "pr_title=$(jq --raw-output .pull_request.title $GITHUB_EVENT_PATH)" >> $GITHUB_ENV + echo "pr_number=$(jq --raw-output .pull_request.number $GITHUB_EVENT_PATH)" >> $GITHUB_ENV - name: Setup environment variables (Push) if: github.event_name == 'push' run: | - repo_url="https://github.com/opensearch-project/OpenSearch" - ref_id=$(git rev-parse HEAD) - branch_name=$(git rev-parse --abbrev-ref HEAD) - echo "branch_name=$branch_name" >> $GITHUB_ENV - echo "pr_from_sha=$ref_id" >> $GITHUB_ENV - echo "pr_from_clone_url=$repo_url" >> $GITHUB_ENV - echo "pr_to_clone_url=$repo_url" >> $GITHUB_ENV - echo "pr_title=Push trigger $branch_name $ref_id $repo_url" >> $GITHUB_ENV - echo "pr_number=Null" >> $GITHUB_ENV + repo_url="https://github.com/opensearch-project/OpenSearch" + ref_id=$(git rev-parse HEAD) + branch_name=$(git rev-parse --abbrev-ref HEAD) + echo "branch_name=$branch_name" >> $GITHUB_ENV + echo "pr_from_sha=$ref_id" >> $GITHUB_ENV + echo "pr_from_clone_url=$repo_url" >> $GITHUB_ENV + echo "pr_to_clone_url=$repo_url" >> $GITHUB_ENV + echo "pr_title=Push trigger $branch_name $ref_id $repo_url" >> $GITHUB_ENV + echo "pr_number=Null" >> $GITHUB_ENV - name: Checkout opensearch-build repo - uses: actions/checkout@v2 + uses: actions/checkout@v3 with: repository: opensearch-project/opensearch-build ref: main @@ -57,17 +57,17 @@ jobs: - name: Trigger jenkins workflow to run gradle check run: | - set -e - set -o pipefail - bash opensearch-build/scripts/gradle/gradle-check.sh ${{ secrets.JENKINS_GRADLE_CHECK_GENERIC_WEBHOOK_TOKEN }} | tee -a gradle-check.log + set -e + set -o pipefail + bash opensearch-build/scripts/gradle/gradle-check.sh ${{ secrets.JENKINS_GRADLE_CHECK_GENERIC_WEBHOOK_TOKEN }} | tee -a gradle-check.log - name: Setup Result Status if: always() run: | - WORKFLOW_URL=`cat gradle-check.log | grep 'WORKFLOW_URL' | awk '{print $2}'` - RESULT=`cat gradle-check.log | grep 'Result:' | awk '{print $2}'` - echo "workflow_url=$WORKFLOW_URL" >> $GITHUB_ENV - echo "result=$RESULT" >> $GITHUB_ENV + WORKFLOW_URL=`cat gradle-check.log | grep 'WORKFLOW_URL' | awk '{print $2}'` + RESULT=`cat gradle-check.log | grep 'Result:' | awk '{print $2}'` + echo "workflow_url=$WORKFLOW_URL" >> $GITHUB_ENV + echo "result=$RESULT" >> $GITHUB_ENV - name: Upload Coverage Report if: success() @@ -81,25 +81,25 @@ jobs: with: issue-number: ${{ env.pr_number }} body: | - ### Gradle Check (Jenkins) Run Completed with: - * **RESULT:** ${{ env.result }} :white_check_mark: - * **URL:** ${{ env.workflow_url }} - * **CommitID:** ${{ env.pr_from_sha }} + ### Gradle Check (Jenkins) Run Completed with: + * **RESULT:** ${{ env.result }} :white_check_mark: + * **URL:** ${{ env.workflow_url }} + * **CommitID:** ${{ env.pr_from_sha }} - name: Extract Test Failure if: ${{ github.event_name == 'pull_request_target' && env.result != 'SUCCESS' }} run: | - TEST_FAILURES=`curl -s "${{ env.workflow_url }}/testReport/api/json?tree=suites\[cases\[status,className,name\]\]" | jq -r '.. | objects | select(.status=="FAILED",.status=="REGRESSION") | (.className + "." + .name)' | uniq -c | sort -n -r | head -n 10` - if [[ "$TEST_FAILURES" != "" ]] - then - echo "test_failures<> $GITHUB_ENV - echo "" >> $GITHUB_ENV - echo "* **TEST FAILURES:**" >> $GITHUB_ENV - echo '```' >> $GITHUB_ENV - echo "$TEST_FAILURES" >> $GITHUB_ENV - echo '```' >> $GITHUB_ENV - echo "EOF" >> $GITHUB_ENV - fi + TEST_FAILURES=`curl -s "${{ env.workflow_url }}/testReport/api/json?tree=suites\[cases\[status,className,name\]\]" | jq -r '.. | objects | select(.status=="FAILED",.status=="REGRESSION") | (.className + "." + .name)' | uniq -c | sort -n -r | head -n 10` + if [[ "$TEST_FAILURES" != "" ]] + then + echo "test_failures<> $GITHUB_ENV + echo "" >> $GITHUB_ENV + echo "* **TEST FAILURES:**" >> $GITHUB_ENV + echo '```' >> $GITHUB_ENV + echo "$TEST_FAILURES" >> $GITHUB_ENV + echo '```' >> $GITHUB_ENV + echo "EOF" >> $GITHUB_ENV + fi - name: Create Comment Flaky if: ${{ github.event_name == 'pull_request_target' && success() && env.result != 'SUCCESS' }} @@ -119,12 +119,12 @@ jobs: with: issue-number: ${{ env.pr_number }} body: | - ### Gradle Check (Jenkins) Run Completed with: - * **RESULT:** ${{ env.result }} :x: ${{ env.test_failures }} - * **URL:** ${{ env.workflow_url }} - * **CommitID:** ${{ env.pr_from_sha }} - Please examine the workflow log, locate, and copy-paste the failure(s) below, then iterate to green. - Is the failure [a flaky test](https://github.com/opensearch-project/OpenSearch/blob/main/DEVELOPER_GUIDE.md#flaky-tests) unrelated to your change? + ### Gradle Check (Jenkins) Run Completed with: + * **RESULT:** ${{ env.result }} :x: ${{ env.test_failures }} + * **URL:** ${{ env.workflow_url }} + * **CommitID:** ${{ env.pr_from_sha }} + Please examine the workflow log, locate, and copy-paste the failure(s) below, then iterate to green. + Is the failure [a flaky test](https://github.com/opensearch-project/OpenSearch/blob/main/DEVELOPER_GUIDE.md#flaky-tests) unrelated to your change? - name: Create Issue On Push Failure if: ${{ github.event_name == 'push' && failure() }} diff --git a/.github/workflows/precommit.yml b/.github/workflows/precommit.yml index e264d65cdf191..8bbba657737c8 100644 --- a/.github/workflows/precommit.yml +++ b/.github/workflows/precommit.yml @@ -1,12 +1,12 @@ name: Gradle Precommit on: [pull_request] - + jobs: precommit: runs-on: ${{ matrix.os }} strategy: matrix: - os: [ubuntu-latest, windows-latest, macos-latest] + os: [windows-latest, macos-latest] # precommit on ubuntu-latest is run as a part of the gradle-check workflow steps: - uses: actions/checkout@v2 - name: Set up JDK 11 From 03bc19220a711853660652a4761c05285f597884 Mon Sep 17 00:00:00 2001 From: suraj kumar Date: Wed, 28 Jun 2023 12:13:49 +0530 Subject: [PATCH 078/109] Add request tracing framework (#7648) * Add request tracing framework Signed-off-by: suranjay * Introduce ThreadContextStatePropagator to propagate request and transient headers within ThreadContext Signed-off-by: Andriy Redko --------- Signed-off-by: suranjay Signed-off-by: Andriy Redko Co-authored-by: Andriy Redko --- CHANGELOG.md | 2 + buildSrc/version.properties | 4 + libs/telemetry/build.gradle | 23 ++ .../org/opensearch/telemetry/Telemetry.java | 31 +++ .../telemetry/metrics/MetricsTelemetry.java | 16 ++ .../telemetry/metrics/package-info.java | 12 ++ .../opensearch/telemetry/package-info.java | 12 ++ .../telemetry/tracing/AbstractSpan.java | 45 ++++ .../telemetry/tracing/DefaultTracer.java | 109 ++++++++++ .../opensearch/telemetry/tracing/Scope.java | 26 +++ .../telemetry/tracing/ScopeImpl.java | 33 +++ .../opensearch/telemetry/tracing/Span.java | 84 ++++++++ .../telemetry/tracing/SpanReference.java | 41 ++++ .../opensearch/telemetry/tracing/Tracer.java | 67 ++++++ .../tracing/TracerContextStorage.java | 35 +++ .../tracing/TracingContextPropagator.java | 34 +++ .../telemetry/tracing/TracingTelemetry.java | 37 ++++ .../telemetry/tracing/noop/NoopTracer.java | 76 +++++++ .../telemetry/tracing/noop/package-info.java | 12 ++ .../telemetry/tracing/package-info.java | 12 ++ .../telemetry/tracing/DefaultTracerTests.java | 122 +++++++++++ plugins/telemetry-otel/build.gradle | 56 +++++ .../opentelemetry-api-1.26.0.jar.sha1 | 1 + .../licenses/opentelemetry-api-LICENSE.txt | 202 ++++++++++++++++++ .../licenses/opentelemetry-api-NOTICE.txt | 0 ...entelemetry-api-logs-1.26.0-alpha.jar.sha1 | 1 + .../opentelemetry-api-logs-LICENSE.txt | 202 ++++++++++++++++++ .../opentelemetry-api-logs-NOTICE.txt | 0 .../opentelemetry-context-1.26.0.jar.sha1 | 1 + .../opentelemetry-context-LICENSE.txt | 202 ++++++++++++++++++ .../licenses/opentelemetry-context-NOTICE.txt | 0 ...telemetry-exporter-logging-1.26.0.jar.sha1 | 1 + ...opentelemetry-exporter-logging-LICENSE.txt | 202 ++++++++++++++++++ .../opentelemetry-exporter-logging-NOTICE.txt | 0 .../opentelemetry-sdk-1.26.0.jar.sha1 | 1 + .../licenses/opentelemetry-sdk-LICENSE.txt | 202 ++++++++++++++++++ .../licenses/opentelemetry-sdk-NOTICE.txt | 0 .../opentelemetry-sdk-common-1.26.0.jar.sha1 | 1 + .../opentelemetry-sdk-common-LICENSE.txt | 202 ++++++++++++++++++ .../opentelemetry-sdk-common-NOTICE.txt | 0 ...entelemetry-sdk-logs-1.26.0-alpha.jar.sha1 | 1 + .../opentelemetry-sdk-logs-LICENSE.txt | 202 ++++++++++++++++++ .../opentelemetry-sdk-logs-NOTICE.txt | 0 .../opentelemetry-sdk-metrics-1.26.0.jar.sha1 | 1 + .../opentelemetry-sdk-metrics-LICENSE.txt | 202 ++++++++++++++++++ .../opentelemetry-sdk-metrics-NOTICE.txt | 0 .../opentelemetry-sdk-trace-1.26.0.jar.sha1 | 1 + .../opentelemetry-sdk-trace-LICENSE.txt | 202 ++++++++++++++++++ .../opentelemetry-sdk-trace-NOTICE.txt | 0 ...pentelemetry-semconv-1.26.0-alpha.jar.sha1 | 1 + .../opentelemetry-semconv-LICENSE.txt | 202 ++++++++++++++++++ .../licenses/opentelemetry-semconv-NOTICE.txt | 0 .../telemetry/OTelTelemetryPlugin.java | 92 ++++++++ .../opensearch/telemetry/package-info.java | 12 ++ .../telemetry/tracing/OTelPropagatedSpan.java | 23 ++ .../tracing/OTelResourceProvider.java | 78 +++++++ .../telemetry/tracing/OTelSpan.java | 70 ++++++ .../telemetry/tracing/OTelTelemetry.java | 41 ++++ .../tracing/OTelTracingContextPropagator.java | 75 +++++++ .../tracing/OTelTracingTelemetry.java | 68 ++++++ .../telemetry/tracing/package-info.java | 12 ++ .../plugin-metadata/plugin-security.policy | 12 ++ .../telemetry/OTelTelemetryPluginTests.java | 50 +++++ .../telemetry/tracing/OTelSpanTests.java | 92 ++++++++ .../OTelTracingContextPropagatorTests.java | 54 +++++ .../tracing/OTelTracingTelemetryTests.java | 68 ++++++ server/build.gradle | 2 + .../common/settings/ClusterSettings.java | 5 +- .../common/settings/FeatureFlagSettings.java | 3 +- .../opensearch/common/util/FeatureFlags.java | 7 + .../common/util/concurrent/ThreadContext.java | 74 +++++-- .../ThreadContextStatePropagator.java | 31 +++ .../main/java/org/opensearch/node/Node.java | 21 ++ .../opensearch/plugins/TelemetryPlugin.java | 25 +++ .../TaskThreadContextStatePropagator.java | 38 ++++ .../opensearch/telemetry/TelemetryModule.java | 47 ++++ .../telemetry/TelemetrySettings.java | 42 ++++ .../opensearch/telemetry/package-info.java | 12 ++ .../telemetry/tracing/NoopTracerFactory.java | 33 +++ ...hreadContextBasedTracerContextStorage.java | 93 ++++++++ .../telemetry/tracing/TracerFactory.java | 70 ++++++ .../telemetry/tracing/package-info.java | 12 ++ .../telemetry/TelemetryModuleTests.java | 65 ++++++ .../telemetry/tracing/TracerFactoryTests.java | 68 ++++++ 84 files changed, 4299 insertions(+), 15 deletions(-) create mode 100644 libs/telemetry/build.gradle create mode 100644 libs/telemetry/src/main/java/org/opensearch/telemetry/Telemetry.java create mode 100644 libs/telemetry/src/main/java/org/opensearch/telemetry/metrics/MetricsTelemetry.java create mode 100644 libs/telemetry/src/main/java/org/opensearch/telemetry/metrics/package-info.java create mode 100644 libs/telemetry/src/main/java/org/opensearch/telemetry/package-info.java create mode 100644 libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/AbstractSpan.java create mode 100644 libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/DefaultTracer.java create mode 100644 libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/Scope.java create mode 100644 libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/ScopeImpl.java create mode 100644 libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/Span.java create mode 100644 libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/SpanReference.java create mode 100644 libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/Tracer.java create mode 100644 libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/TracerContextStorage.java create mode 100644 libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/TracingContextPropagator.java create mode 100644 libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/TracingTelemetry.java create mode 100644 libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/noop/NoopTracer.java create mode 100644 libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/noop/package-info.java create mode 100644 libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/package-info.java create mode 100644 libs/telemetry/src/test/java/org/opensearch/telemetry/tracing/DefaultTracerTests.java create mode 100644 plugins/telemetry-otel/build.gradle create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-api-1.26.0.jar.sha1 create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-api-LICENSE.txt create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-api-NOTICE.txt create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-api-logs-1.26.0-alpha.jar.sha1 create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-api-logs-LICENSE.txt create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-api-logs-NOTICE.txt create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-context-1.26.0.jar.sha1 create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-context-LICENSE.txt create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-context-NOTICE.txt create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-exporter-logging-1.26.0.jar.sha1 create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-exporter-logging-LICENSE.txt create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-exporter-logging-NOTICE.txt create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-sdk-1.26.0.jar.sha1 create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-sdk-LICENSE.txt create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-sdk-NOTICE.txt create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-sdk-common-1.26.0.jar.sha1 create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-sdk-common-LICENSE.txt create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-sdk-common-NOTICE.txt create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-sdk-logs-1.26.0-alpha.jar.sha1 create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-sdk-logs-LICENSE.txt create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-sdk-logs-NOTICE.txt create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-sdk-metrics-1.26.0.jar.sha1 create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-sdk-metrics-LICENSE.txt create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-sdk-metrics-NOTICE.txt create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-sdk-trace-1.26.0.jar.sha1 create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-sdk-trace-LICENSE.txt create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-sdk-trace-NOTICE.txt create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-semconv-1.26.0-alpha.jar.sha1 create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-semconv-LICENSE.txt create mode 100644 plugins/telemetry-otel/licenses/opentelemetry-semconv-NOTICE.txt create mode 100644 plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/OTelTelemetryPlugin.java create mode 100644 plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/package-info.java create mode 100644 plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelPropagatedSpan.java create mode 100644 plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelResourceProvider.java create mode 100644 plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelSpan.java create mode 100644 plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelTelemetry.java create mode 100644 plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelTracingContextPropagator.java create mode 100644 plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelTracingTelemetry.java create mode 100644 plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/package-info.java create mode 100644 plugins/telemetry-otel/src/main/plugin-metadata/plugin-security.policy create mode 100644 plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/OTelTelemetryPluginTests.java create mode 100644 plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/tracing/OTelSpanTests.java create mode 100644 plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/tracing/OTelTracingContextPropagatorTests.java create mode 100644 plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/tracing/OTelTracingTelemetryTests.java create mode 100644 server/src/main/java/org/opensearch/common/util/concurrent/ThreadContextStatePropagator.java create mode 100644 server/src/main/java/org/opensearch/plugins/TelemetryPlugin.java create mode 100644 server/src/main/java/org/opensearch/tasks/TaskThreadContextStatePropagator.java create mode 100644 server/src/main/java/org/opensearch/telemetry/TelemetryModule.java create mode 100644 server/src/main/java/org/opensearch/telemetry/TelemetrySettings.java create mode 100644 server/src/main/java/org/opensearch/telemetry/package-info.java create mode 100644 server/src/main/java/org/opensearch/telemetry/tracing/NoopTracerFactory.java create mode 100644 server/src/main/java/org/opensearch/telemetry/tracing/ThreadContextBasedTracerContextStorage.java create mode 100644 server/src/main/java/org/opensearch/telemetry/tracing/TracerFactory.java create mode 100644 server/src/main/java/org/opensearch/telemetry/tracing/package-info.java create mode 100644 server/src/test/java/org/opensearch/telemetry/TelemetryModuleTests.java create mode 100644 server/src/test/java/org/opensearch/telemetry/tracing/TracerFactoryTests.java diff --git a/CHANGELOG.md b/CHANGELOG.md index 0ffb14d6750c5..8c370dbe7b0d9 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -35,6 +35,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `io.opencensus:opencensus-api` from 0.18.0 to 0.31.1 ([#7291](https://github.com/opensearch-project/OpenSearch/pull/7291)) - OpenJDK Update (April 2023 Patch releases) ([#7344](https://github.com/opensearch-project/OpenSearch/pull/7344) - Bump `com.google.http-client:google-http-client:1.43.2` from 1.42.0 to 1.43.2 ([7928](https://github.com/opensearch-project/OpenSearch/pull/7928))) +- Add Opentelemetry dependencies ([#7543](https://github.com/opensearch-project/OpenSearch/issues/7543)) ### Changed - [CCR] Add getHistoryOperationsFromTranslog method to fetch the history snapshot from translogs ([#3948](https://github.com/opensearch-project/OpenSearch/pull/3948)) @@ -83,6 +84,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Update components of segrep backpressure to support remote store. ([#8020](https://github.com/opensearch-project/OpenSearch/pull/8020)) - Make remote cluster connection setup in async ([#8038](https://github.com/opensearch-project/OpenSearch/pull/8038)) - Add API to initialize extensions ([#8029]()https://github.com/opensearch-project/OpenSearch/pull/8029) +- Add distributed tracing framework ([#7543](https://github.com/opensearch-project/OpenSearch/issues/7543)) ### Dependencies - Bump `com.azure:azure-storage-common` from 12.21.0 to 12.21.1 (#7566, #7814) diff --git a/buildSrc/version.properties b/buildSrc/version.properties index dd64569259c2d..735a9fe4f8255 100644 --- a/buildSrc/version.properties +++ b/buildSrc/version.properties @@ -66,3 +66,7 @@ zstd = 1.5.5-3 jzlib = 1.1.3 resteasy = 6.2.4.Final + +# opentelemetry dependencies +opentelemetry = 1.26.0 + diff --git a/libs/telemetry/build.gradle b/libs/telemetry/build.gradle new file mode 100644 index 0000000000000..ce94698836b4f --- /dev/null +++ b/libs/telemetry/build.gradle @@ -0,0 +1,23 @@ +/* + * 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. + * + * Modifications Copyright OpenSearch Contributors. See + * GitHub history for details. + */ + +dependencies { + testImplementation "com.carrotsearch.randomizedtesting:randomizedtesting-runner:${versions.randomizedrunner}" + testImplementation "junit:junit:${versions.junit}" + testImplementation "org.hamcrest:hamcrest:${versions.hamcrest}" + testImplementation(project(":test:framework")) { + exclude group: 'org.opensearch', module: 'opensearch-telemetry' + } +} + +tasks.named('forbiddenApisMain').configure { + replaceSignatureFiles 'jdk-signatures' +} diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/Telemetry.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/Telemetry.java new file mode 100644 index 0000000000000..6f50699528b6b --- /dev/null +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/Telemetry.java @@ -0,0 +1,31 @@ +/* + * 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.telemetry; + +import org.opensearch.telemetry.metrics.MetricsTelemetry; +import org.opensearch.telemetry.tracing.TracingTelemetry; + +/** + * Interface defining telemetry + */ +public interface Telemetry { + + /** + * Provides tracing telemetry + * @return tracing telemetry instance + */ + TracingTelemetry getTracingTelemetry(); + + /** + * Provides metrics telemetry + * @return metrics telemetry instance + */ + MetricsTelemetry getMetricsTelemetry(); + +} diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/metrics/MetricsTelemetry.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/metrics/MetricsTelemetry.java new file mode 100644 index 0000000000000..fa3b7fd192f1a --- /dev/null +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/metrics/MetricsTelemetry.java @@ -0,0 +1,16 @@ +/* + * 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.telemetry.metrics; + +/** + * Interface for metrics telemetry providers + */ +public interface MetricsTelemetry { + +} diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/metrics/package-info.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/metrics/package-info.java new file mode 100644 index 0000000000000..dfe17cc1c11ed --- /dev/null +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/metrics/package-info.java @@ -0,0 +1,12 @@ +/* + * 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. + */ + +/** + * Contains metrics related classes + */ +package org.opensearch.telemetry.metrics; diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/package-info.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/package-info.java new file mode 100644 index 0000000000000..ad76f5e308bea --- /dev/null +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/package-info.java @@ -0,0 +1,12 @@ +/* + * 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. + */ + +/** + * Contains telemetry related classes + */ +package org.opensearch.telemetry; diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/AbstractSpan.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/AbstractSpan.java new file mode 100644 index 0000000000000..316edc971913e --- /dev/null +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/AbstractSpan.java @@ -0,0 +1,45 @@ +/* + * 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.telemetry.tracing; + +/** + * Base span + */ +public abstract class AbstractSpan implements Span { + + /** + * name of the span + */ + private final String spanName; + /** + * span's parent span + */ + private final Span parentSpan; + + /** + * Base constructor + * @param spanName name of the span + * @param parentSpan span's parent span + */ + protected AbstractSpan(String spanName, Span parentSpan) { + this.spanName = spanName; + this.parentSpan = parentSpan; + } + + @Override + public Span getParentSpan() { + return parentSpan; + } + + @Override + public String getSpanName() { + return spanName; + } + +} diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/DefaultTracer.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/DefaultTracer.java new file mode 100644 index 0000000000000..ab9110af7c3ab --- /dev/null +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/DefaultTracer.java @@ -0,0 +1,109 @@ +/* + * 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.telemetry.tracing; + +import java.io.Closeable; +import java.io.IOException; + +/** + * + * The default tracer implementation. This class implements the basic logic for span lifecycle and its state management. + * It also handles tracing context propagation between spans. + * + * + */ +public class DefaultTracer implements Tracer { + static final String THREAD_NAME = "th_name"; + + private final TracingTelemetry tracingTelemetry; + private final TracerContextStorage tracerContextStorage; + + /** + * Creates DefaultTracer instance + * + * @param tracingTelemetry tracing telemetry instance + * @param tracerContextStorage storage used for storing current span context + */ + public DefaultTracer(TracingTelemetry tracingTelemetry, TracerContextStorage tracerContextStorage) { + this.tracingTelemetry = tracingTelemetry; + this.tracerContextStorage = tracerContextStorage; + } + + @Override + public Scope startSpan(String spanName) { + Span span = createSpan(spanName, getCurrentSpan()); + setCurrentSpanInContext(span); + addDefaultAttributes(span); + return new ScopeImpl(() -> endSpan(span)); + } + + @Override + public void addSpanAttribute(String key, String value) { + Span currentSpan = getCurrentSpan(); + currentSpan.addAttribute(key, value); + } + + @Override + public void addSpanAttribute(String key, long value) { + Span currentSpan = getCurrentSpan(); + currentSpan.addAttribute(key, value); + } + + @Override + public void addSpanAttribute(String key, double value) { + Span currentSpan = getCurrentSpan(); + currentSpan.addAttribute(key, value); + } + + @Override + public void addSpanAttribute(String key, boolean value) { + Span currentSpan = getCurrentSpan(); + currentSpan.addAttribute(key, value); + } + + @Override + public void addSpanEvent(String event) { + Span currentSpan = getCurrentSpan(); + currentSpan.addEvent(event); + } + + @Override + public void close() throws IOException { + ((Closeable) tracingTelemetry).close(); + } + + // Visible for testing + Span getCurrentSpan() { + return tracerContextStorage.get(TracerContextStorage.CURRENT_SPAN); + } + + private void endSpan(Span span) { + if (span != null) { + span.endSpan(); + setCurrentSpanInContext(span.getParentSpan()); + } + } + + private Span createSpan(String spanName, Span parentSpan) { + return tracingTelemetry.createSpan(spanName, parentSpan); + } + + private void setCurrentSpanInContext(Span span) { + tracerContextStorage.put(TracerContextStorage.CURRENT_SPAN, span); + } + + /** + * Adds default attributes in the span + * @param span the current active span + */ + protected void addDefaultAttributes(Span span) { + span.addAttribute(THREAD_NAME, Thread.currentThread().getName()); + } + +} diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/Scope.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/Scope.java new file mode 100644 index 0000000000000..52f4eaf648eea --- /dev/null +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/Scope.java @@ -0,0 +1,26 @@ +/* + * 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.telemetry.tracing; + +/** + * An auto-closeable that represents scope of the span. + * It is recommended that you use this class with a try-with-resources block: + */ +public interface Scope extends AutoCloseable { + /** + * No-op Scope implementation + */ + Scope NO_OP = () -> {}; + + /** + * closes the scope + */ + @Override + void close(); +} diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/ScopeImpl.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/ScopeImpl.java new file mode 100644 index 0000000000000..30a7ac7fa90e7 --- /dev/null +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/ScopeImpl.java @@ -0,0 +1,33 @@ +/* + * 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.telemetry.tracing; + +/** + * Executes the runnable on close + */ +public class ScopeImpl implements Scope { + + private Runnable runnableOnClose; + + /** + * Creates Scope instance + * @param runnableOnClose runnable to execute on scope close + */ + public ScopeImpl(Runnable runnableOnClose) { + this.runnableOnClose = runnableOnClose; + } + + /** + * Executes the runnable to end the scope + */ + @Override + public void close() { + runnableOnClose.run(); + } +} diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/Span.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/Span.java new file mode 100644 index 0000000000000..0710b8a22a37f --- /dev/null +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/Span.java @@ -0,0 +1,84 @@ +/* + * 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.telemetry.tracing; + +/** + * An interface that represents a tracing span. + * Spans are created by the Tracer.startSpan method. + * Span must be ended by calling Tracer.endSpan which internally calls Span's endSpan. + */ +public interface Span { + + /** + * Ends the span + */ + void endSpan(); + + /** + * Returns span's parent span + */ + Span getParentSpan(); + + /** + * Returns the name of the {@link Span} + */ + String getSpanName(); + + /** + * Adds string type attribute in the span + * + * @param key of the attribute + * @param value value of the attribute + */ + void addAttribute(String key, String value); + + /** + * Adds long type attribute in the span + * + * @param key of the attribute + * @param value value of the attribute + */ + void addAttribute(String key, Long value); + + /** + * Adds double type attribute in the span + * + * @param key of the attribute + * @param value value of the attribute + */ + void addAttribute(String key, Double value); + + /** + * Adds boolean type attribute in the span + * + * @param key of the attribute + * @param value value of the attribute + */ + void addAttribute(String key, Boolean value); + + /** + * Adds an event in the span + * + * @param event name of the event + */ + void addEvent(String event); + + /** + * Returns traceId of the span + * @return span's traceId + */ + String getTraceId(); + + /** + * Returns spanId of the span + * @return span's spanId + */ + String getSpanId(); + +} diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/SpanReference.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/SpanReference.java new file mode 100644 index 0000000000000..99d1bd3c93c84 --- /dev/null +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/SpanReference.java @@ -0,0 +1,41 @@ +/* + * 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.telemetry.tracing; + +/** + * Wrapper class to hold reference of Span + */ +public class SpanReference { + + private Span span; + + /** + * Creates the wrapper with given span + * @param span the span object to wrap + */ + public SpanReference(Span span) { + this.span = span; + } + + /** + * Returns the span object + * @return underlying span + */ + public Span getSpan() { + return span; + } + + /** + * Updates the underlying span + * @param span underlying span + */ + public void setSpan(Span span) { + this.span = span; + } +} diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/Tracer.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/Tracer.java new file mode 100644 index 0000000000000..fcc091eb39c48 --- /dev/null +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/Tracer.java @@ -0,0 +1,67 @@ +/* + * 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.telemetry.tracing; + +import java.io.Closeable; + +/** + * Tracer is the interface used to create a {@link Span} and interact with current active {@link Span}. + * It automatically handles the context propagation between threads, tasks, nodes etc. + * + * All methods on the Tracer object are multi-thread safe. + */ +public interface Tracer extends Closeable { + + /** + * Starts the {@link Span} with given name + * + * @param spanName span name + * @return scope of the span, must be closed with explicit close or with try-with-resource + */ + Scope startSpan(String spanName); + + /** + * Adds string attribute to the current active {@link Span}. + * + * @param key attribute key + * @param value attribute value + */ + void addSpanAttribute(String key, String value); + + /** + * Adds long attribute to the current active {@link Span}. + * + * @param key attribute key + * @param value attribute value + */ + void addSpanAttribute(String key, long value); + + /** + * Adds double attribute to the current active {@link Span}. + * + * @param key attribute key + * @param value attribute value + */ + void addSpanAttribute(String key, double value); + + /** + * Adds boolean attribute to the current active {@link Span}. + * + * @param key attribute key + * @param value attribute value + */ + void addSpanAttribute(String key, boolean value); + + /** + * Adds an event to the current active {@link Span}. + * + * @param event event name + */ + void addSpanEvent(String event); +} diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/TracerContextStorage.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/TracerContextStorage.java new file mode 100644 index 0000000000000..eb93006835332 --- /dev/null +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/TracerContextStorage.java @@ -0,0 +1,35 @@ +/* + * 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.telemetry.tracing; + +/** + * Storage interface used for storing tracing context + * @param key type + * @param value type + */ +public interface TracerContextStorage { + /** + * Key for storing current span + */ + String CURRENT_SPAN = "current_span"; + + /** + * Fetches value corresponding to key + * @param key of the tracing context + * @return value for key + */ + V get(K key); + + /** + * Puts tracing context value with key + * @param key of the tracing context + * @param value of the tracing context + */ + void put(K key, V value); +} diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/TracingContextPropagator.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/TracingContextPropagator.java new file mode 100644 index 0000000000000..1152e3aedfa88 --- /dev/null +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/TracingContextPropagator.java @@ -0,0 +1,34 @@ +/* + * 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.telemetry.tracing; + +import java.util.Map; +import java.util.function.BiConsumer; + +/** + * Interface defining the tracing related context propagation + */ +public interface TracingContextPropagator { + + /** + * Extracts current span from context + * @param props properties + * @return current span + */ + Span extract(Map props); + + /** + * Injects tracing context + * + * @param currentSpan the current active span + * @param setter to add tracing context in map + */ + void inject(Span currentSpan, BiConsumer setter); + +} diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/TracingTelemetry.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/TracingTelemetry.java new file mode 100644 index 0000000000000..16c76bd0cc141 --- /dev/null +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/TracingTelemetry.java @@ -0,0 +1,37 @@ +/* + * 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.telemetry.tracing; + +import java.io.Closeable; + +/** + * Interface for tracing telemetry providers + */ +public interface TracingTelemetry extends Closeable { + + /** + * Creates span with provided arguments + * @param spanName name of the span + * @param parentSpan span's parent span + * @return span instance + */ + Span createSpan(String spanName, Span parentSpan); + + /** + * provides tracing context propagator + * @return tracing context propagator instance + */ + TracingContextPropagator getContextPropagator(); + + /** + * closes the resource + */ + void close(); + +} diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/noop/NoopTracer.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/noop/NoopTracer.java new file mode 100644 index 0000000000000..18fc60e41e54d --- /dev/null +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/noop/NoopTracer.java @@ -0,0 +1,76 @@ +/* + * 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.telemetry.tracing.noop; + +import org.opensearch.telemetry.tracing.Scope; +import org.opensearch.telemetry.tracing.Tracer; + +/** + * No-op implementation of Tracer + */ +public class NoopTracer implements Tracer { + + /** + * No-op Tracer instance + */ + public static final Tracer INSTANCE = new NoopTracer(); + + private NoopTracer() {} + + @Override + public Scope startSpan(String spanName) { + return Scope.NO_OP; + } + + /** + * @param key attribute key + * @param value attribute value + */ + @Override + public void addSpanAttribute(String key, String value) { + + } + + /** + * @param key attribute key + * @param value attribute value + */ + @Override + public void addSpanAttribute(String key, long value) { + + } + + /** + * @param key attribute key + * @param value attribute value + */ + @Override + public void addSpanAttribute(String key, double value) { + + } + + /** + * @param key attribute key + * @param value attribute value + */ + @Override + public void addSpanAttribute(String key, boolean value) { + + } + + @Override + public void addSpanEvent(String event) { + + } + + @Override + public void close() { + + } +} diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/noop/package-info.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/noop/package-info.java new file mode 100644 index 0000000000000..b9d83e7bc7275 --- /dev/null +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/noop/package-info.java @@ -0,0 +1,12 @@ +/* + * 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. + */ + +/** + * Contains No-op implementations + */ +package org.opensearch.telemetry.tracing.noop; diff --git a/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/package-info.java b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/package-info.java new file mode 100644 index 0000000000000..66898bd58b753 --- /dev/null +++ b/libs/telemetry/src/main/java/org/opensearch/telemetry/tracing/package-info.java @@ -0,0 +1,12 @@ +/* + * 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. + */ + +/** + * Contains tracing related classes + */ +package org.opensearch.telemetry.tracing; diff --git a/libs/telemetry/src/test/java/org/opensearch/telemetry/tracing/DefaultTracerTests.java b/libs/telemetry/src/test/java/org/opensearch/telemetry/tracing/DefaultTracerTests.java new file mode 100644 index 0000000000000..f0e8f3c2e2344 --- /dev/null +++ b/libs/telemetry/src/test/java/org/opensearch/telemetry/tracing/DefaultTracerTests.java @@ -0,0 +1,122 @@ +/* + * 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.telemetry.tracing; + +import org.junit.Assert; +import org.opensearch.test.OpenSearchTestCase; + +import java.io.IOException; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.verify; + +public class DefaultTracerTests extends OpenSearchTestCase { + + private TracingTelemetry mockTracingTelemetry; + private TracerContextStorage mockTracerContextStorage; + private Span mockSpan; + private Span mockParentSpan; + + @Override + public void setUp() throws Exception { + super.setUp(); + setupMocks(); + } + + @Override + public void tearDown() throws Exception { + super.tearDown(); + } + + public void testCreateSpan() { + DefaultTracer defaultTracer = new DefaultTracer(mockTracingTelemetry, mockTracerContextStorage); + + defaultTracer.startSpan("span_name"); + + Assert.assertEquals("span_name", defaultTracer.getCurrentSpan().getSpanName()); + } + + public void testEndSpanByClosingScope() { + DefaultTracer defaultTracer = new DefaultTracer(mockTracingTelemetry, mockTracerContextStorage); + try (Scope scope = defaultTracer.startSpan("span_name")) { + verify(mockTracerContextStorage).put(TracerContextStorage.CURRENT_SPAN, mockSpan); + } + verify(mockTracerContextStorage).put(TracerContextStorage.CURRENT_SPAN, mockParentSpan); + } + + public void testAddSpanAttributeString() { + Tracer defaultTracer = new DefaultTracer(mockTracingTelemetry, mockTracerContextStorage); + defaultTracer.startSpan("span_name"); + + defaultTracer.addSpanAttribute("key", "value"); + + verify(mockSpan).addAttribute("key", "value"); + } + + public void testAddSpanAttributeLong() { + Tracer defaultTracer = new DefaultTracer(mockTracingTelemetry, mockTracerContextStorage); + defaultTracer.startSpan("span_name"); + + defaultTracer.addSpanAttribute("key", 1L); + + verify(mockSpan).addAttribute("key", 1L); + } + + public void testAddSpanAttributeDouble() { + Tracer defaultTracer = new DefaultTracer(mockTracingTelemetry, mockTracerContextStorage); + defaultTracer.startSpan("span_name"); + + defaultTracer.addSpanAttribute("key", 1.0); + + verify(mockSpan).addAttribute("key", 1.0); + } + + public void testAddSpanAttributeBoolean() { + Tracer defaultTracer = new DefaultTracer(mockTracingTelemetry, mockTracerContextStorage); + defaultTracer.startSpan("span_name"); + + defaultTracer.addSpanAttribute("key", true); + + verify(mockSpan).addAttribute("key", true); + } + + public void testAddEvent() { + Tracer defaultTracer = new DefaultTracer(mockTracingTelemetry, mockTracerContextStorage); + defaultTracer.startSpan("span_name"); + + defaultTracer.addSpanEvent("eventName"); + + verify(mockSpan).addEvent("eventName"); + } + + public void testClose() throws IOException { + Tracer defaultTracer = new DefaultTracer(mockTracingTelemetry, mockTracerContextStorage); + + defaultTracer.close(); + + verify(mockTracingTelemetry).close(); + } + + @SuppressWarnings("unchecked") + private void setupMocks() { + mockTracingTelemetry = mock(TracingTelemetry.class); + mockSpan = mock(Span.class); + mockParentSpan = mock(Span.class); + mockTracerContextStorage = mock(TracerContextStorage.class); + when(mockSpan.getSpanName()).thenReturn("span_name"); + when(mockSpan.getSpanId()).thenReturn("span_id"); + when(mockSpan.getTraceId()).thenReturn("trace_id"); + when(mockSpan.getParentSpan()).thenReturn(mockParentSpan); + when(mockParentSpan.getSpanId()).thenReturn("parent_span_id"); + when(mockParentSpan.getTraceId()).thenReturn("trace_id"); + when(mockTracerContextStorage.get(TracerContextStorage.CURRENT_SPAN)).thenReturn(mockParentSpan, mockSpan); + when(mockTracingTelemetry.createSpan("span_name", mockParentSpan)).thenReturn(mockSpan); + } +} diff --git a/plugins/telemetry-otel/build.gradle b/plugins/telemetry-otel/build.gradle new file mode 100644 index 0000000000000..7a56621be5f1e --- /dev/null +++ b/plugins/telemetry-otel/build.gradle @@ -0,0 +1,56 @@ +/* + * 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. + */ + +import org.apache.tools.ant.taskdefs.condition.Os +import org.opensearch.gradle.Architecture +import org.opensearch.gradle.OS +import org.opensearch.gradle.info.BuildParams + +opensearchplugin { + description 'Opentelemetry based telemetry implementation.' + classname 'org.opensearch.telemetry.OTelTelemetryPlugin' + hasClientJar = true +} + +dependencies { + api project(":libs:opensearch-telemetry") + api "io.opentelemetry:opentelemetry-api:${versions.opentelemetry}" + api "io.opentelemetry:opentelemetry-context:${versions.opentelemetry}" + api "io.opentelemetry:opentelemetry-sdk:${versions.opentelemetry}" + api "io.opentelemetry:opentelemetry-sdk-common:${versions.opentelemetry}" + api "io.opentelemetry:opentelemetry-sdk-trace:${versions.opentelemetry}" + api "io.opentelemetry:opentelemetry-sdk-metrics:${versions.opentelemetry}" + api "io.opentelemetry:opentelemetry-exporter-logging:${versions.opentelemetry}" + api "io.opentelemetry:opentelemetry-semconv:${versions.opentelemetry}-alpha" + api "io.opentelemetry:opentelemetry-sdk-logs:${versions.opentelemetry}-alpha" + api "io.opentelemetry:opentelemetry-api-logs:${versions.opentelemetry}-alpha" +} + + +thirdPartyAudit { + ignoreViolations( + 'io.opentelemetry.internal.shaded.jctools.queues.MpscArrayQueueConsumerIndexField', + 'io.opentelemetry.internal.shaded.jctools.queues.MpscArrayQueueProducerIndexField', + 'io.opentelemetry.internal.shaded.jctools.queues.MpscArrayQueueProducerLimitField', + 'io.opentelemetry.internal.shaded.jctools.util.UnsafeAccess', + 'io.opentelemetry.internal.shaded.jctools.util.UnsafeRefArrayAccess' + ) + + ignoreMissingClasses( + 'io.opentelemetry.api.events.EventEmitter', + 'io.opentelemetry.api.events.EventEmitterBuilder', + 'io.opentelemetry.api.events.EventEmitterProvider', + 'io.opentelemetry.extension.incubator.metrics.ExtendedDoubleHistogramBuilder', + 'io.opentelemetry.extension.incubator.metrics.ExtendedLongHistogramBuilder', + 'io.opentelemetry.extension.incubator.metrics.HistogramAdviceConfigurer', + 'io.opentelemetry.sdk.autoconfigure.spi.ConfigProperties', + 'io.opentelemetry.sdk.autoconfigure.spi.logs.ConfigurableLogRecordExporterProvider', + 'io.opentelemetry.sdk.autoconfigure.spi.metrics.ConfigurableMetricExporterProvider', + 'io.opentelemetry.sdk.autoconfigure.spi.traces.ConfigurableSpanExporterProvider' + ) +} diff --git a/plugins/telemetry-otel/licenses/opentelemetry-api-1.26.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-api-1.26.0.jar.sha1 new file mode 100644 index 0000000000000..da3abcc8f70d2 --- /dev/null +++ b/plugins/telemetry-otel/licenses/opentelemetry-api-1.26.0.jar.sha1 @@ -0,0 +1 @@ +7ee1ccca95155e4640094ba8dfbd0bb8c1709c83 \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-api-LICENSE.txt b/plugins/telemetry-otel/licenses/opentelemetry-api-LICENSE.txt new file mode 100644 index 0000000000000..d645695673349 --- /dev/null +++ b/plugins/telemetry-otel/licenses/opentelemetry-api-LICENSE.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. diff --git a/plugins/telemetry-otel/licenses/opentelemetry-api-NOTICE.txt b/plugins/telemetry-otel/licenses/opentelemetry-api-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/plugins/telemetry-otel/licenses/opentelemetry-api-logs-1.26.0-alpha.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-api-logs-1.26.0-alpha.jar.sha1 new file mode 100644 index 0000000000000..2c233d785dcb2 --- /dev/null +++ b/plugins/telemetry-otel/licenses/opentelemetry-api-logs-1.26.0-alpha.jar.sha1 @@ -0,0 +1 @@ +1b0b6c1a20da0f841634d4f736e331aa4871a4db \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-api-logs-LICENSE.txt b/plugins/telemetry-otel/licenses/opentelemetry-api-logs-LICENSE.txt new file mode 100644 index 0000000000000..d645695673349 --- /dev/null +++ b/plugins/telemetry-otel/licenses/opentelemetry-api-logs-LICENSE.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. diff --git a/plugins/telemetry-otel/licenses/opentelemetry-api-logs-NOTICE.txt b/plugins/telemetry-otel/licenses/opentelemetry-api-logs-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/plugins/telemetry-otel/licenses/opentelemetry-context-1.26.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-context-1.26.0.jar.sha1 new file mode 100644 index 0000000000000..01d9fd732249b --- /dev/null +++ b/plugins/telemetry-otel/licenses/opentelemetry-context-1.26.0.jar.sha1 @@ -0,0 +1 @@ +42991f523a7a10761213e2f11633c67c8beaed88 \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-context-LICENSE.txt b/plugins/telemetry-otel/licenses/opentelemetry-context-LICENSE.txt new file mode 100644 index 0000000000000..d645695673349 --- /dev/null +++ b/plugins/telemetry-otel/licenses/opentelemetry-context-LICENSE.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. diff --git a/plugins/telemetry-otel/licenses/opentelemetry-context-NOTICE.txt b/plugins/telemetry-otel/licenses/opentelemetry-context-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/plugins/telemetry-otel/licenses/opentelemetry-exporter-logging-1.26.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-exporter-logging-1.26.0.jar.sha1 new file mode 100644 index 0000000000000..ef07e4cb81e34 --- /dev/null +++ b/plugins/telemetry-otel/licenses/opentelemetry-exporter-logging-1.26.0.jar.sha1 @@ -0,0 +1 @@ +1b932170774da5e766440fa058d879f68fe2c5dd \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-exporter-logging-LICENSE.txt b/plugins/telemetry-otel/licenses/opentelemetry-exporter-logging-LICENSE.txt new file mode 100644 index 0000000000000..d645695673349 --- /dev/null +++ b/plugins/telemetry-otel/licenses/opentelemetry-exporter-logging-LICENSE.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. diff --git a/plugins/telemetry-otel/licenses/opentelemetry-exporter-logging-NOTICE.txt b/plugins/telemetry-otel/licenses/opentelemetry-exporter-logging-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-1.26.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-sdk-1.26.0.jar.sha1 new file mode 100644 index 0000000000000..dc9946de3b160 --- /dev/null +++ b/plugins/telemetry-otel/licenses/opentelemetry-sdk-1.26.0.jar.sha1 @@ -0,0 +1 @@ +79a86f258ede8625627e8fbdff07d1149c88a8e6 \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-LICENSE.txt b/plugins/telemetry-otel/licenses/opentelemetry-sdk-LICENSE.txt new file mode 100644 index 0000000000000..d645695673349 --- /dev/null +++ b/plugins/telemetry-otel/licenses/opentelemetry-sdk-LICENSE.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-NOTICE.txt b/plugins/telemetry-otel/licenses/opentelemetry-sdk-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-common-1.26.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-sdk-common-1.26.0.jar.sha1 new file mode 100644 index 0000000000000..2bd3e60a1faf6 --- /dev/null +++ b/plugins/telemetry-otel/licenses/opentelemetry-sdk-common-1.26.0.jar.sha1 @@ -0,0 +1 @@ +b42359d2232f8d802d55153be5330b1d9e21ee15 \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-common-LICENSE.txt b/plugins/telemetry-otel/licenses/opentelemetry-sdk-common-LICENSE.txt new file mode 100644 index 0000000000000..d645695673349 --- /dev/null +++ b/plugins/telemetry-otel/licenses/opentelemetry-sdk-common-LICENSE.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-common-NOTICE.txt b/plugins/telemetry-otel/licenses/opentelemetry-sdk-common-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-logs-1.26.0-alpha.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-sdk-logs-1.26.0-alpha.jar.sha1 new file mode 100644 index 0000000000000..90bb8202c4c9d --- /dev/null +++ b/plugins/telemetry-otel/licenses/opentelemetry-sdk-logs-1.26.0-alpha.jar.sha1 @@ -0,0 +1 @@ +a8abeaee240291cce9067f07569f151d11a6275a \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-logs-LICENSE.txt b/plugins/telemetry-otel/licenses/opentelemetry-sdk-logs-LICENSE.txt new file mode 100644 index 0000000000000..d645695673349 --- /dev/null +++ b/plugins/telemetry-otel/licenses/opentelemetry-sdk-logs-LICENSE.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-logs-NOTICE.txt b/plugins/telemetry-otel/licenses/opentelemetry-sdk-logs-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-metrics-1.26.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-sdk-metrics-1.26.0.jar.sha1 new file mode 100644 index 0000000000000..62396a603423f --- /dev/null +++ b/plugins/telemetry-otel/licenses/opentelemetry-sdk-metrics-1.26.0.jar.sha1 @@ -0,0 +1 @@ +8c4af22d7d92a3a79714be3f79724b0ab774ba9e \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-metrics-LICENSE.txt b/plugins/telemetry-otel/licenses/opentelemetry-sdk-metrics-LICENSE.txt new file mode 100644 index 0000000000000..d645695673349 --- /dev/null +++ b/plugins/telemetry-otel/licenses/opentelemetry-sdk-metrics-LICENSE.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-metrics-NOTICE.txt b/plugins/telemetry-otel/licenses/opentelemetry-sdk-metrics-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-trace-1.26.0.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-sdk-trace-1.26.0.jar.sha1 new file mode 100644 index 0000000000000..0fcebee353105 --- /dev/null +++ b/plugins/telemetry-otel/licenses/opentelemetry-sdk-trace-1.26.0.jar.sha1 @@ -0,0 +1 @@ +fcc5785b2cf2be897f31b927e24b53e46e377388 \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-trace-LICENSE.txt b/plugins/telemetry-otel/licenses/opentelemetry-sdk-trace-LICENSE.txt new file mode 100644 index 0000000000000..d645695673349 --- /dev/null +++ b/plugins/telemetry-otel/licenses/opentelemetry-sdk-trace-LICENSE.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. diff --git a/plugins/telemetry-otel/licenses/opentelemetry-sdk-trace-NOTICE.txt b/plugins/telemetry-otel/licenses/opentelemetry-sdk-trace-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/plugins/telemetry-otel/licenses/opentelemetry-semconv-1.26.0-alpha.jar.sha1 b/plugins/telemetry-otel/licenses/opentelemetry-semconv-1.26.0-alpha.jar.sha1 new file mode 100644 index 0000000000000..47c7ece8c9f6c --- /dev/null +++ b/plugins/telemetry-otel/licenses/opentelemetry-semconv-1.26.0-alpha.jar.sha1 @@ -0,0 +1 @@ +1f4f963673f8209208f868666cd43e79b9a2dd15 \ No newline at end of file diff --git a/plugins/telemetry-otel/licenses/opentelemetry-semconv-LICENSE.txt b/plugins/telemetry-otel/licenses/opentelemetry-semconv-LICENSE.txt new file mode 100644 index 0000000000000..d645695673349 --- /dev/null +++ b/plugins/telemetry-otel/licenses/opentelemetry-semconv-LICENSE.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed 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. diff --git a/plugins/telemetry-otel/licenses/opentelemetry-semconv-NOTICE.txt b/plugins/telemetry-otel/licenses/opentelemetry-semconv-NOTICE.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/OTelTelemetryPlugin.java b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/OTelTelemetryPlugin.java new file mode 100644 index 0000000000000..1c38c9dc8d6be --- /dev/null +++ b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/OTelTelemetryPlugin.java @@ -0,0 +1,92 @@ +/* + * 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.telemetry; + +import org.opensearch.common.settings.Setting; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.plugins.Plugin; +import org.opensearch.plugins.TelemetryPlugin; +import org.opensearch.telemetry.metrics.MetricsTelemetry; +import org.opensearch.telemetry.tracing.OTelResourceProvider; +import org.opensearch.telemetry.tracing.OTelTelemetry; +import org.opensearch.telemetry.tracing.OTelTracingTelemetry; + +import java.util.Arrays; +import java.util.List; +import java.util.Optional; + +/** + * Telemetry plugin based on Otel + */ +public class OTelTelemetryPlugin extends Plugin implements TelemetryPlugin { + + static final String OTEL_TRACER_NAME = "otel"; + + /** + * span exporter batch size + */ + public static final Setting TRACER_EXPORTER_BATCH_SIZE_SETTING = Setting.intSetting( + "telemetry.otel.tracer.exporter.batch_size", + 512, + 1, + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); + /** + * span exporter max queue size + */ + public static final Setting TRACER_EXPORTER_MAX_QUEUE_SIZE_SETTING = Setting.intSetting( + "telemetry.otel.tracer.exporter.max_queue_size", + 2048, + 1, + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); + /** + * span exporter delay in seconds + */ + public static final Setting TRACER_EXPORTER_DELAY_SETTING = Setting.timeSetting( + "telemetry.otel.tracer.exporter.delay", + TimeValue.timeValueSeconds(2), + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); + + private final Settings settings; + + /** + * Creates Otel plugin + * @param settings cluster settings + */ + public OTelTelemetryPlugin(Settings settings) { + this.settings = settings; + } + + @Override + public List> getSettings() { + return Arrays.asList(TRACER_EXPORTER_BATCH_SIZE_SETTING, TRACER_EXPORTER_DELAY_SETTING, TRACER_EXPORTER_MAX_QUEUE_SIZE_SETTING); + } + + @Override + public Optional getTelemetry(TelemetrySettings settings) { + return Optional.of(telemetry()); + } + + @Override + public String getName() { + return OTEL_TRACER_NAME; + } + + private Telemetry telemetry() { + return new OTelTelemetry(new OTelTracingTelemetry(OTelResourceProvider.get(settings)), new MetricsTelemetry() { + }); + } + +} diff --git a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/package-info.java b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/package-info.java new file mode 100644 index 0000000000000..4545f0ef5990e --- /dev/null +++ b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/package-info.java @@ -0,0 +1,12 @@ +/* + * 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. + */ + +/** + * This package contains classes needed for telemetry. + */ +package org.opensearch.telemetry; diff --git a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelPropagatedSpan.java b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelPropagatedSpan.java new file mode 100644 index 0000000000000..5aa1069e60367 --- /dev/null +++ b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelPropagatedSpan.java @@ -0,0 +1,23 @@ +/* + * 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.telemetry.tracing; + +/** + * Propagated span through context propagation + */ +public class OTelPropagatedSpan extends OTelSpan { + + /** + * Creates OTelPropagatedSpan + * @param span otel propagated span + */ + public OTelPropagatedSpan(io.opentelemetry.api.trace.Span span) { + super(null, span, null); + } +} diff --git a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelResourceProvider.java b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelResourceProvider.java new file mode 100644 index 0000000000000..04bade9ec942a --- /dev/null +++ b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelResourceProvider.java @@ -0,0 +1,78 @@ +/* + * 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.telemetry.tracing; + +import io.opentelemetry.api.OpenTelemetry; +import io.opentelemetry.api.common.Attributes; +import io.opentelemetry.api.trace.propagation.W3CTraceContextPropagator; +import io.opentelemetry.context.propagation.ContextPropagators; +import io.opentelemetry.exporter.logging.LoggingSpanExporter; +import io.opentelemetry.sdk.OpenTelemetrySdk; +import io.opentelemetry.sdk.resources.Resource; +import io.opentelemetry.sdk.trace.SdkTracerProvider; +import io.opentelemetry.sdk.trace.export.BatchSpanProcessor; +import io.opentelemetry.sdk.trace.export.SpanExporter; +import io.opentelemetry.sdk.trace.samplers.Sampler; +import io.opentelemetry.semconv.resource.attributes.ResourceAttributes; +import org.opensearch.common.settings.Settings; + +import java.util.concurrent.TimeUnit; + +import static org.opensearch.telemetry.OTelTelemetryPlugin.TRACER_EXPORTER_BATCH_SIZE_SETTING; +import static org.opensearch.telemetry.OTelTelemetryPlugin.TRACER_EXPORTER_DELAY_SETTING; +import static org.opensearch.telemetry.OTelTelemetryPlugin.TRACER_EXPORTER_MAX_QUEUE_SIZE_SETTING; + +/** + * This class encapsulates all OpenTelemetry related resources + */ +public final class OTelResourceProvider { + private OTelResourceProvider() {} + + /** + * Creates OpenTelemetry instance with default configuration + * @param settings cluster settings + * @return OpenTelemetry instance + */ + public static OpenTelemetry get(Settings settings) { + return get( + settings, + new LoggingSpanExporter(), + ContextPropagators.create(W3CTraceContextPropagator.getInstance()), + Sampler.alwaysOn() + ); + } + + /** + * Creates OpenTelemetry instance with provided configuration + * @param settings cluster settings + * @param spanExporter span exporter instance + * @param contextPropagators context propagator instance + * @param sampler sampler instance + * @return Opentelemetry instance + */ + public static OpenTelemetry get(Settings settings, SpanExporter spanExporter, ContextPropagators contextPropagators, Sampler sampler) { + Resource resource = Resource.create(Attributes.of(ResourceAttributes.SERVICE_NAME, "OpenSearch")); + SdkTracerProvider sdkTracerProvider = SdkTracerProvider.builder() + .addSpanProcessor(spanProcessor(settings, spanExporter)) + .setResource(resource) + .setSampler(sampler) + .build(); + + return OpenTelemetrySdk.builder().setTracerProvider(sdkTracerProvider).setPropagators(contextPropagators).buildAndRegisterGlobal(); + } + + private static BatchSpanProcessor spanProcessor(Settings settings, SpanExporter spanExporter) { + return BatchSpanProcessor.builder(spanExporter) + .setScheduleDelay(TRACER_EXPORTER_DELAY_SETTING.get(settings).getSeconds(), TimeUnit.SECONDS) + .setMaxExportBatchSize(TRACER_EXPORTER_BATCH_SIZE_SETTING.get(settings)) + .setMaxQueueSize(TRACER_EXPORTER_MAX_QUEUE_SIZE_SETTING.get(settings)) + .build(); + } + +} diff --git a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelSpan.java b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelSpan.java new file mode 100644 index 0000000000000..23a2d9baa3e6e --- /dev/null +++ b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelSpan.java @@ -0,0 +1,70 @@ +/* + * 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.telemetry.tracing; + +import io.opentelemetry.api.trace.Span; + +/** + * Default implementation of {@link Span} using Otel span. It keeps a reference of OpenTelemetry Span and handles span + * lifecycle management by delegating calls to it. + */ +class OTelSpan extends AbstractSpan { + + private final Span delegateSpan; + + public OTelSpan(String spanName, Span span, org.opensearch.telemetry.tracing.Span parentSpan) { + super(spanName, parentSpan); + this.delegateSpan = span; + } + + @Override + public void endSpan() { + delegateSpan.end(); + } + + @Override + public void addAttribute(String key, String value) { + delegateSpan.setAttribute(key, value); + } + + @Override + public void addAttribute(String key, Long value) { + delegateSpan.setAttribute(key, value); + } + + @Override + public void addAttribute(String key, Double value) { + delegateSpan.setAttribute(key, value); + } + + @Override + public void addAttribute(String key, Boolean value) { + delegateSpan.setAttribute(key, value); + } + + @Override + public void addEvent(String event) { + delegateSpan.addEvent(event); + } + + @Override + public String getTraceId() { + return delegateSpan.getSpanContext().getTraceId(); + } + + @Override + public String getSpanId() { + return delegateSpan.getSpanContext().getSpanId(); + } + + io.opentelemetry.api.trace.Span getDelegateSpan() { + return delegateSpan; + } + +} diff --git a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelTelemetry.java b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelTelemetry.java new file mode 100644 index 0000000000000..282fabd43346b --- /dev/null +++ b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelTelemetry.java @@ -0,0 +1,41 @@ +/* + * 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.telemetry.tracing; + +import org.opensearch.telemetry.Telemetry; +import org.opensearch.telemetry.metrics.MetricsTelemetry; + +/** + * Otel implementation of Telemetry + */ +public class OTelTelemetry implements Telemetry { + + private final TracingTelemetry tracingTelemetry; + private final MetricsTelemetry metricsTelemetry; + + /** + * Creates Telemetry instance + * @param tracingTelemetry tracing telemetry + * @param metricsTelemetry metrics telemetry + */ + public OTelTelemetry(TracingTelemetry tracingTelemetry, MetricsTelemetry metricsTelemetry) { + this.tracingTelemetry = tracingTelemetry; + this.metricsTelemetry = metricsTelemetry; + } + + @Override + public TracingTelemetry getTracingTelemetry() { + return tracingTelemetry; + } + + @Override + public MetricsTelemetry getMetricsTelemetry() { + return metricsTelemetry; + } +} diff --git a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelTracingContextPropagator.java b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelTracingContextPropagator.java new file mode 100644 index 0000000000000..15609b39b6b94 --- /dev/null +++ b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelTracingContextPropagator.java @@ -0,0 +1,75 @@ +/* + * 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.telemetry.tracing; + +import io.opentelemetry.api.OpenTelemetry; +import io.opentelemetry.context.Context; +import io.opentelemetry.context.propagation.TextMapGetter; +import io.opentelemetry.context.propagation.TextMapSetter; + +import java.util.Map; +import java.util.function.BiConsumer; + +/** + * Otel implementation of TracingContextPropagator + */ +public class OTelTracingContextPropagator implements TracingContextPropagator { + + private final OpenTelemetry openTelemetry; + + /** + * Creates OTelTracingContextPropagator instance + * @param openTelemetry Otel OpenTelemetry instance + */ + public OTelTracingContextPropagator(OpenTelemetry openTelemetry) { + this.openTelemetry = openTelemetry; + } + + @Override + public Span extract(Map props) { + Context context = openTelemetry.getPropagators().getTextMapPropagator().extract(Context.current(), props, TEXT_MAP_GETTER); + if (context != null) { + io.opentelemetry.api.trace.Span span = io.opentelemetry.api.trace.Span.fromContext(context); + return new OTelPropagatedSpan(span); + } + return null; + } + + @Override + public void inject(Span currentSpan, BiConsumer setter) { + openTelemetry.getPropagators().getTextMapPropagator().inject(context((OTelSpan) currentSpan), setter, TEXT_MAP_SETTER); + + } + + private static Context context(OTelSpan oTelSpan) { + return Context.current().with(io.opentelemetry.api.trace.Span.wrap(oTelSpan.getDelegateSpan().getSpanContext())); + } + + private static final TextMapSetter> TEXT_MAP_SETTER = (carrier, key, value) -> { + if (carrier != null) { + carrier.accept(key, value); + } + }; + + private static final TextMapGetter> TEXT_MAP_GETTER = new TextMapGetter<>() { + @Override + public Iterable keys(Map headers) { + return headers.keySet(); + } + + @Override + public String get(Map headers, String key) { + if (headers != null && headers.containsKey(key)) { + return headers.get(key); + } + return null; + } + }; + +} diff --git a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelTracingTelemetry.java b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelTracingTelemetry.java new file mode 100644 index 0000000000000..8a0034e098461 --- /dev/null +++ b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelTracingTelemetry.java @@ -0,0 +1,68 @@ +/* + * 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.telemetry.tracing; + +import io.opentelemetry.api.OpenTelemetry; +import io.opentelemetry.context.Context; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.io.Closeable; +import java.io.IOException; + +/** + * OTel based Telemetry provider + */ +public class OTelTracingTelemetry implements TracingTelemetry { + + private static final Logger logger = LogManager.getLogger(OTelTracingTelemetry.class); + + private final OpenTelemetry openTelemetry; + private final io.opentelemetry.api.trace.Tracer otelTracer; + + /** + * Creates OTel based Telemetry + * @param openTelemetry OpenTelemetry instance + */ + public OTelTracingTelemetry(OpenTelemetry openTelemetry) { + this.openTelemetry = openTelemetry; + this.otelTracer = openTelemetry.getTracer("os-tracer"); + + } + + @Override + public void close() { + try { + ((Closeable) openTelemetry).close(); + } catch (IOException e) { + logger.warn("Error while closing Opentelemetry", e); + } + } + + @Override + public Span createSpan(String spanName, Span parentSpan) { + return createOtelSpan(spanName, parentSpan); + } + + @Override + public TracingContextPropagator getContextPropagator() { + return new OTelTracingContextPropagator(openTelemetry); + } + + private Span createOtelSpan(String spanName, Span parentSpan) { + io.opentelemetry.api.trace.Span otelSpan = otelSpan(spanName, parentSpan); + return new OTelSpan(spanName, otelSpan, parentSpan); + } + + io.opentelemetry.api.trace.Span otelSpan(String spanName, Span parentOTelSpan) { + return parentOTelSpan == null || !(parentOTelSpan instanceof OTelSpan) + ? otelTracer.spanBuilder(spanName).startSpan() + : otelTracer.spanBuilder(spanName).setParent(Context.current().with(((OTelSpan) parentOTelSpan).getDelegateSpan())).startSpan(); + } +} diff --git a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/package-info.java b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/package-info.java new file mode 100644 index 0000000000000..4ac1e4c212c81 --- /dev/null +++ b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/package-info.java @@ -0,0 +1,12 @@ +/* + * 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. + */ + +/** + * This package contains classes needed for tracing requests. + */ +package org.opensearch.telemetry.tracing; diff --git a/plugins/telemetry-otel/src/main/plugin-metadata/plugin-security.policy b/plugins/telemetry-otel/src/main/plugin-metadata/plugin-security.policy new file mode 100644 index 0000000000000..0f556121915bb --- /dev/null +++ b/plugins/telemetry-otel/src/main/plugin-metadata/plugin-security.policy @@ -0,0 +1,12 @@ +/* + * 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. + */ + +grant { +}; + + diff --git a/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/OTelTelemetryPluginTests.java b/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/OTelTelemetryPluginTests.java new file mode 100644 index 0000000000000..c6ffba04ac285 --- /dev/null +++ b/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/OTelTelemetryPluginTests.java @@ -0,0 +1,50 @@ +/* + * 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.telemetry; + +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Setting; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.FeatureFlags; +import org.opensearch.telemetry.tracing.OTelTracingTelemetry; +import org.opensearch.telemetry.tracing.TracingTelemetry; +import org.opensearch.test.OpenSearchTestCase; + +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.Optional; + +import static org.opensearch.telemetry.OTelTelemetryPlugin.OTEL_TRACER_NAME; +import static org.opensearch.telemetry.OTelTelemetryPlugin.TRACER_EXPORTER_BATCH_SIZE_SETTING; +import static org.opensearch.telemetry.OTelTelemetryPlugin.TRACER_EXPORTER_DELAY_SETTING; +import static org.opensearch.telemetry.OTelTelemetryPlugin.TRACER_EXPORTER_MAX_QUEUE_SIZE_SETTING; + +public class OTelTelemetryPluginTests extends OpenSearchTestCase { + + public void testGetTelemetry() { + Set> allTracerSettings = new HashSet<>(); + ClusterSettings.FEATURE_FLAGGED_CLUSTER_SETTINGS.get(List.of(FeatureFlags.TELEMETRY)).stream().forEach((allTracerSettings::add)); + Settings settings = Settings.builder().build(); + OTelTelemetryPlugin oTelTracerModulePlugin = new OTelTelemetryPlugin(settings); + Optional tracer = oTelTracerModulePlugin.getTelemetry(null); + + assertEquals(OTEL_TRACER_NAME, oTelTracerModulePlugin.getName()); + TracingTelemetry tracingTelemetry = tracer.get().getTracingTelemetry(); + assertTrue(tracingTelemetry instanceof OTelTracingTelemetry); + assertEquals( + Arrays.asList(TRACER_EXPORTER_BATCH_SIZE_SETTING, TRACER_EXPORTER_DELAY_SETTING, TRACER_EXPORTER_MAX_QUEUE_SIZE_SETTING), + oTelTracerModulePlugin.getSettings() + ); + tracingTelemetry.close(); + + } + +} diff --git a/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/tracing/OTelSpanTests.java b/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/tracing/OTelSpanTests.java new file mode 100644 index 0000000000000..ac849e620673a --- /dev/null +++ b/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/tracing/OTelSpanTests.java @@ -0,0 +1,92 @@ +/* + * 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.telemetry.tracing; + +import io.opentelemetry.api.trace.Span; +import io.opentelemetry.api.trace.SpanContext; +import io.opentelemetry.api.trace.TraceFlags; +import io.opentelemetry.api.trace.TraceState; +import org.opensearch.test.OpenSearchTestCase; + +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class OTelSpanTests extends OpenSearchTestCase { + + private static final String TRACE_ID = "4aa59968f31dcbff7807741afa9d7d62"; + private static final String SPAN_ID = "bea205cd25756b5e"; + + public void testEndSpanTest() { + Span mockSpan = getMockSpan(); + OTelSpan oTelSpan = new OTelSpan("spanName", mockSpan, null); + oTelSpan.endSpan(); + verify(mockSpan).end(); + } + + public void testAddAttributeString() { + Span mockSpan = getMockSpan(); + OTelSpan oTelSpan = new OTelSpan("spanName", mockSpan, null); + oTelSpan.addAttribute("key", "value"); + + verify(mockSpan).setAttribute("key", "value"); + } + + public void testAddAttributeLong() { + Span mockSpan = getMockSpan(); + OTelSpan oTelSpan = new OTelSpan("spanName", mockSpan, null); + oTelSpan.addAttribute("key", 1L); + + verify(mockSpan).setAttribute("key", 1L); + } + + public void testAddAttributeDouble() { + Span mockSpan = getMockSpan(); + OTelSpan oTelSpan = new OTelSpan("spanName", mockSpan, null); + oTelSpan.addAttribute("key", 1.0); + + verify(mockSpan).setAttribute("key", 1.0); + } + + public void testAddAttributeBoolean() { + Span mockSpan = getMockSpan(); + OTelSpan oTelSpan = new OTelSpan("spanName", mockSpan, null); + oTelSpan.addAttribute("key", true); + + verify(mockSpan).setAttribute("key", true); + } + + public void testAddEvent() { + Span mockSpan = getMockSpan(); + OTelSpan oTelSpan = new OTelSpan("spanName", mockSpan, null); + oTelSpan.addEvent("eventName"); + + verify(mockSpan).addEvent("eventName"); + } + + public void testGetTraceId() { + Span mockSpan = getMockSpan(); + OTelSpan oTelSpan = new OTelSpan("spanName", mockSpan, null); + + assertEquals(TRACE_ID, oTelSpan.getTraceId()); + } + + public void testGetSpanId() { + Span mockSpan = getMockSpan(); + OTelSpan oTelSpan = new OTelSpan("spanName", mockSpan, null); + + assertEquals(SPAN_ID, oTelSpan.getSpanId()); + } + + private Span getMockSpan() { + Span mockSpan = mock(Span.class); + when(mockSpan.getSpanContext()).thenReturn(SpanContext.create(TRACE_ID, SPAN_ID, TraceFlags.getDefault(), TraceState.getDefault())); + return mockSpan; + } +} diff --git a/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/tracing/OTelTracingContextPropagatorTests.java b/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/tracing/OTelTracingContextPropagatorTests.java new file mode 100644 index 0000000000000..1f76b0b9def18 --- /dev/null +++ b/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/tracing/OTelTracingContextPropagatorTests.java @@ -0,0 +1,54 @@ +/* + * 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.telemetry.tracing; + +import io.opentelemetry.api.OpenTelemetry; +import io.opentelemetry.api.trace.Span; +import io.opentelemetry.api.trace.SpanContext; +import io.opentelemetry.api.trace.TraceFlags; +import io.opentelemetry.api.trace.TraceState; +import io.opentelemetry.api.trace.propagation.W3CTraceContextPropagator; +import io.opentelemetry.context.propagation.ContextPropagators; +import org.opensearch.test.OpenSearchTestCase; + +import java.util.HashMap; +import java.util.Map; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class OTelTracingContextPropagatorTests extends OpenSearchTestCase { + + private static final String TRACE_ID = "4aa59968f31dcbff7807741afa9d7d62"; + private static final String SPAN_ID = "bea205cd25756b5e"; + + public void testAddTracerContextToHeader() { + Span mockSpan = mock(Span.class); + when(mockSpan.getSpanContext()).thenReturn(SpanContext.create(TRACE_ID, SPAN_ID, TraceFlags.getDefault(), TraceState.getDefault())); + OTelSpan span = new OTelSpan("spanName", mockSpan, null); + Map requestHeaders = new HashMap<>(); + OpenTelemetry mockOpenTelemetry = mock(OpenTelemetry.class); + when(mockOpenTelemetry.getPropagators()).thenReturn(ContextPropagators.create(W3CTraceContextPropagator.getInstance())); + TracingContextPropagator tracingContextPropagator = new OTelTracingContextPropagator(mockOpenTelemetry); + + tracingContextPropagator.inject(span, (key, value) -> requestHeaders.put(key, value)); + assertEquals("00-" + TRACE_ID + "-" + SPAN_ID + "-00", requestHeaders.get("traceparent")); + } + + public void testExtractTracerContextFromHeader() { + Map requestHeaders = new HashMap<>(); + requestHeaders.put("traceparent", "00-" + TRACE_ID + "-" + SPAN_ID + "-00"); + OpenTelemetry mockOpenTelemetry = mock(OpenTelemetry.class); + when(mockOpenTelemetry.getPropagators()).thenReturn(ContextPropagators.create(W3CTraceContextPropagator.getInstance())); + TracingContextPropagator tracingContextPropagator = new OTelTracingContextPropagator(mockOpenTelemetry); + org.opensearch.telemetry.tracing.Span span = tracingContextPropagator.extract(requestHeaders); + assertEquals(TRACE_ID, span.getTraceId()); + assertEquals(SPAN_ID, span.getSpanId()); + } +} diff --git a/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/tracing/OTelTracingTelemetryTests.java b/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/tracing/OTelTracingTelemetryTests.java new file mode 100644 index 0000000000000..7dec7824b9790 --- /dev/null +++ b/plugins/telemetry-otel/src/test/java/org/opensearch/telemetry/tracing/OTelTracingTelemetryTests.java @@ -0,0 +1,68 @@ +/* + * 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.telemetry.tracing; + +import io.opentelemetry.api.OpenTelemetry; +import io.opentelemetry.api.trace.SpanBuilder; +import io.opentelemetry.api.trace.Tracer; +import org.opensearch.test.OpenSearchTestCase; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class OTelTracingTelemetryTests extends OpenSearchTestCase { + + public void testCreateSpanWithoutParent() { + OpenTelemetry mockOpenTelemetry = mock(OpenTelemetry.class); + Tracer mockTracer = mock(Tracer.class); + when(mockOpenTelemetry.getTracer("os-tracer")).thenReturn(mockTracer); + SpanBuilder mockSpanBuilder = mock(SpanBuilder.class); + when(mockTracer.spanBuilder("span_name")).thenReturn(mockSpanBuilder); + when(mockSpanBuilder.startSpan()).thenReturn(mock(io.opentelemetry.api.trace.Span.class)); + + TracingTelemetry tracingTelemetry = new OTelTracingTelemetry(mockOpenTelemetry); + Span span = tracingTelemetry.createSpan("span_name", null); + + verify(mockSpanBuilder, never()).setParent(any()); + assertNull(span.getParentSpan()); + } + + public void testCreateSpanWithParent() { + OpenTelemetry mockOpenTelemetry = mock(OpenTelemetry.class); + Tracer mockTracer = mock(Tracer.class); + when(mockOpenTelemetry.getTracer("os-tracer")).thenReturn(mockTracer); + SpanBuilder mockSpanBuilder = mock(SpanBuilder.class); + when(mockTracer.spanBuilder("span_name")).thenReturn(mockSpanBuilder); + when(mockSpanBuilder.setParent(any())).thenReturn(mockSpanBuilder); + when(mockSpanBuilder.startSpan()).thenReturn(mock(io.opentelemetry.api.trace.Span.class)); + + Span parentSpan = new OTelSpan("parent_span", mock(io.opentelemetry.api.trace.Span.class), null); + + TracingTelemetry tracingTelemetry = new OTelTracingTelemetry(mockOpenTelemetry); + Span span = tracingTelemetry.createSpan("span_name", parentSpan); + + verify(mockSpanBuilder).setParent(any()); + assertNotNull(span.getParentSpan()); + assertEquals("parent_span", span.getParentSpan().getSpanName()); + } + + public void testGetContextPropagator() { + OpenTelemetry mockOpenTelemetry = mock(OpenTelemetry.class); + Tracer mockTracer = mock(Tracer.class); + when(mockOpenTelemetry.getTracer("os-tracer")).thenReturn(mockTracer); + + TracingTelemetry tracingTelemetry = new OTelTracingTelemetry(mockOpenTelemetry); + + assertTrue(tracingTelemetry.getContextPropagator() instanceof OTelTracingContextPropagator); + } + +} diff --git a/server/build.gradle b/server/build.gradle index 2738e4351a109..ab67eabe76d0c 100644 --- a/server/build.gradle +++ b/server/build.gradle @@ -102,6 +102,8 @@ dependencies { api project(':libs:opensearch-secure-sm') api project(':libs:opensearch-x-content') api project(":libs:opensearch-geo") + api project(":libs:opensearch-telemetry") + compileOnly project(':libs:opensearch-plugin-classloader') testRuntimeOnly project(':libs:opensearch-plugin-classloader') diff --git a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java index fe1d292dbd8f6..d70ea16cf5fdd 100644 --- a/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/ClusterSettings.java @@ -52,6 +52,7 @@ import org.opensearch.tasks.TaskManager; import org.opensearch.tasks.TaskResourceTrackingService; import org.opensearch.tasks.consumer.TopNSearchTasksLogger; +import org.opensearch.telemetry.TelemetrySettings; import org.opensearch.watcher.ResourceWatcherService; import org.opensearch.action.admin.cluster.configuration.TransportAddVotingConfigExclusionsAction; import org.opensearch.action.admin.indices.close.TransportCloseIndexAction; @@ -675,6 +676,8 @@ public void apply(Settings value, Settings current, Settings previous) { IndicesService.CLUSTER_REMOTE_TRANSLOG_REPOSITORY_SETTING ), List.of(FeatureFlags.CONCURRENT_SEGMENT_SEARCH), - List.of(SearchService.CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING) + List.of(SearchService.CLUSTER_CONCURRENT_SEGMENT_SEARCH_SETTING), + List.of(FeatureFlags.TELEMETRY), + List.of(TelemetrySettings.TRACER_ENABLED_SETTING) ); } diff --git a/server/src/main/java/org/opensearch/common/settings/FeatureFlagSettings.java b/server/src/main/java/org/opensearch/common/settings/FeatureFlagSettings.java index 19a5808579d50..dae66c79c63ec 100644 --- a/server/src/main/java/org/opensearch/common/settings/FeatureFlagSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/FeatureFlagSettings.java @@ -39,7 +39,8 @@ protected FeatureFlagSettings( FeatureFlags.EXTENSIONS_SETTING, FeatureFlags.IDENTITY_SETTING, FeatureFlags.SEARCH_PIPELINE_SETTING, - FeatureFlags.CONCURRENT_SEGMENT_SEARCH_SETTING + FeatureFlags.CONCURRENT_SEGMENT_SEARCH_SETTING, + FeatureFlags.TELEMETRY_SETTING ) ) ); diff --git a/server/src/main/java/org/opensearch/common/util/FeatureFlags.java b/server/src/main/java/org/opensearch/common/util/FeatureFlags.java index a26b4006bb31e..2b4dab616d00f 100644 --- a/server/src/main/java/org/opensearch/common/util/FeatureFlags.java +++ b/server/src/main/java/org/opensearch/common/util/FeatureFlags.java @@ -63,6 +63,11 @@ public class FeatureFlags { */ public static final String CONCURRENT_SEGMENT_SEARCH = "opensearch.experimental.feature.concurrent_segment_search.enabled"; + /** + * Gates the functionality of telemetry framework. + */ + public static final String TELEMETRY = "opensearch.experimental.feature.telemetry.enabled"; + /** * Should store the settings from opensearch.yml. */ @@ -105,6 +110,8 @@ public static boolean isEnabled(String featureFlagName) { public static final Setting IDENTITY_SETTING = Setting.boolSetting(IDENTITY, false, Property.NodeScope); + public static final Setting TELEMETRY_SETTING = Setting.boolSetting(TELEMETRY, false, Property.NodeScope); + public static final Setting CONCURRENT_SEGMENT_SEARCH_SETTING = Setting.boolSetting( CONCURRENT_SEGMENT_SEARCH, false, diff --git a/server/src/main/java/org/opensearch/common/util/concurrent/ThreadContext.java b/server/src/main/java/org/opensearch/common/util/concurrent/ThreadContext.java index 5b9a77c75dddb..025fb7a36b684 100644 --- a/server/src/main/java/org/opensearch/common/util/concurrent/ThreadContext.java +++ b/server/src/main/java/org/opensearch/common/util/concurrent/ThreadContext.java @@ -45,6 +45,7 @@ import org.opensearch.common.settings.Settings; import org.opensearch.http.HttpTransportSettings; import org.opensearch.tasks.Task; +import org.opensearch.tasks.TaskThreadContextStatePropagator; import java.io.IOException; import java.nio.charset.StandardCharsets; @@ -56,7 +57,9 @@ import java.util.LinkedHashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; +import java.util.concurrent.CopyOnWriteArrayList; import java.util.function.BiConsumer; import java.util.function.BinaryOperator; import java.util.function.Function; @@ -66,7 +69,6 @@ import static org.opensearch.http.HttpTransportSettings.SETTING_HTTP_MAX_WARNING_HEADER_COUNT; import static org.opensearch.http.HttpTransportSettings.SETTING_HTTP_MAX_WARNING_HEADER_SIZE; -import static org.opensearch.tasks.TaskResourceTrackingService.TASK_ID; /** * A ThreadContext is a map of string headers and a transient map of keyed objects that are associated with @@ -113,6 +115,7 @@ public final class ThreadContext implements Writeable { private final ThreadLocal threadLocal; private final int maxWarningHeaderCount; private final long maxWarningHeaderSize; + private final List propagators; /** * Creates a new ThreadContext instance @@ -123,6 +126,15 @@ public ThreadContext(Settings settings) { this.threadLocal = ThreadLocal.withInitial(() -> DEFAULT_CONTEXT); this.maxWarningHeaderCount = SETTING_HTTP_MAX_WARNING_HEADER_COUNT.get(settings); this.maxWarningHeaderSize = SETTING_HTTP_MAX_WARNING_HEADER_SIZE.get(settings).getBytes(); + this.propagators = new CopyOnWriteArrayList<>(List.of(new TaskThreadContextStatePropagator())); + } + + public void registerThreadContextStatePropagator(final ThreadContextStatePropagator propagator) { + propagators.add(Objects.requireNonNull(propagator)); + } + + public void unregisterThreadContextStatePropagator(final ThreadContextStatePropagator propagator) { + propagators.remove(Objects.requireNonNull(propagator)); } /** @@ -147,8 +159,9 @@ public StoredContext stashContext() { ); } - if (context.transientHeaders.containsKey(TASK_ID)) { - threadContextStruct = threadContextStruct.putTransient(TASK_ID, context.transientHeaders.get(TASK_ID)); + final Map transientHeaders = propagateTransients(context.transientHeaders); + if (!transientHeaders.isEmpty()) { + threadContextStruct = threadContextStruct.putTransient(transientHeaders); } threadLocal.set(threadContextStruct); @@ -166,7 +179,10 @@ public StoredContext stashContext() { */ public Writeable captureAsWriteable() { final ThreadContextStruct context = threadLocal.get(); - return out -> context.writeTo(out, defaultHeader); + return out -> { + final Map propagatedHeaders = propagateHeaders(context.transientHeaders); + context.writeTo(out, defaultHeader, propagatedHeaders); + }; } /** @@ -224,17 +240,24 @@ public StoredContext newStoredContext(boolean preserveResponseHeaders) { */ public StoredContext newStoredContext(boolean preserveResponseHeaders, Collection transientHeadersToClear) { final ThreadContextStruct originalContext = threadLocal.get(); + final Map newTransientHeaders = new HashMap<>(originalContext.transientHeaders); + + boolean transientHeadersModified = false; + final Map transientHeaders = propagateTransients(originalContext.transientHeaders); + if (!transientHeaders.isEmpty()) { + newTransientHeaders.putAll(transientHeaders); + transientHeadersModified = true; + } + // clear specific transient headers from the current context - Map newTransientHeaders = null; for (String transientHeaderToClear : transientHeadersToClear) { - if (originalContext.transientHeaders.containsKey(transientHeaderToClear)) { - if (newTransientHeaders == null) { - newTransientHeaders = new HashMap<>(originalContext.transientHeaders); - } + if (newTransientHeaders.containsKey(transientHeaderToClear)) { newTransientHeaders.remove(transientHeaderToClear); + transientHeadersModified = true; } } - if (newTransientHeaders != null) { + + if (transientHeadersModified == true) { ThreadContextStruct threadContextStruct = new ThreadContextStruct( originalContext.requestHeaders, originalContext.responseHeaders, @@ -246,6 +269,7 @@ public StoredContext newStoredContext(boolean preserveResponseHeaders, Collectio } // this is the context when this method returns final ThreadContextStruct newContext = threadLocal.get(); + return () -> { if (preserveResponseHeaders && threadLocal.get() != newContext) { threadLocal.set(originalContext.putResponseHeaders(threadLocal.get().responseHeaders)); @@ -294,7 +318,9 @@ public Supplier wrapRestorable(StoredContext storedContext) { @Override public void writeTo(StreamOutput out) throws IOException { - threadLocal.get().writeTo(out, defaultHeader); + final ThreadContextStruct context = threadLocal.get(); + final Map propagatedHeaders = propagateHeaders(context.transientHeaders); + context.writeTo(out, defaultHeader, propagatedHeaders); } /** @@ -522,6 +548,18 @@ public static Map buildDefaultHeaders(Settings settings) { } } + private Map propagateTransients(Map source) { + final Map transients = new HashMap<>(); + propagators.forEach(p -> transients.putAll(p.transients(source))); + return transients; + } + + private Map propagateHeaders(Map source) { + final Map headers = new HashMap<>(); + propagators.forEach(p -> headers.putAll(p.headers(source))); + return headers; + } + private static final class ThreadContextStruct { private static final ThreadContextStruct EMPTY = new ThreadContextStruct( @@ -695,6 +733,14 @@ private ThreadContextStruct putResponse( return new ThreadContextStruct(requestHeaders, newResponseHeaders, transientHeaders, isSystemContext, newWarningHeaderSize); } + private ThreadContextStruct putTransient(Map values) { + Map newTransient = new HashMap<>(this.transientHeaders); + for (Map.Entry entry : values.entrySet()) { + putSingleHeader(entry.getKey(), entry.getValue(), newTransient); + } + return new ThreadContextStruct(requestHeaders, responseHeaders, newTransient, isSystemContext); + } + private ThreadContextStruct putTransient(String key, Object value) { Map newTransient = new HashMap<>(this.transientHeaders); putSingleHeader(key, value, newTransient); @@ -709,13 +755,15 @@ private ThreadContextStruct copyHeaders(Iterable> head return putHeaders(newHeaders); } - private void writeTo(StreamOutput out, Map defaultHeaders) throws IOException { + private void writeTo(StreamOutput out, Map defaultHeaders, Map propagatedHeaders) + throws IOException { final Map requestHeaders; - if (defaultHeaders.isEmpty()) { + if (defaultHeaders.isEmpty() && propagatedHeaders.isEmpty()) { requestHeaders = this.requestHeaders; } else { requestHeaders = new HashMap<>(defaultHeaders); requestHeaders.putAll(this.requestHeaders); + requestHeaders.putAll(propagatedHeaders); } out.writeVInt(requestHeaders.size()); diff --git a/server/src/main/java/org/opensearch/common/util/concurrent/ThreadContextStatePropagator.java b/server/src/main/java/org/opensearch/common/util/concurrent/ThreadContextStatePropagator.java new file mode 100644 index 0000000000000..b3fc79c5446db --- /dev/null +++ b/server/src/main/java/org/opensearch/common/util/concurrent/ThreadContextStatePropagator.java @@ -0,0 +1,31 @@ +/* + * 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.common.util.concurrent; + +import java.util.Map; + +/** + * The propagator for {@link ThreadContext} that helps to carry-over the state from one + * thread to another (tasks, tracing context, etc). + */ +public interface ThreadContextStatePropagator { + /** + * Returns the list of transient headers that needs to be propagated from current context to new thread context. + * @param source current context transient headers + * @return the list of transient headers that needs to be propagated from current context to new thread context + */ + Map transients(Map source); + + /** + * Returns the list of request headers that needs to be propagated from current context to request. + * @param source current context headers + * @return the list of request headers that needs to be propagated from current context to request + */ + Map headers(Map source); +} diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index 688f2d05b203b..3742c817118da 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -56,6 +56,8 @@ import org.opensearch.monitor.fs.FsProbe; import org.opensearch.plugins.ExtensionAwarePlugin; import org.opensearch.plugins.SearchPipelinePlugin; +import org.opensearch.telemetry.tracing.NoopTracerFactory; +import org.opensearch.telemetry.tracing.TracerFactory; import org.opensearch.search.backpressure.SearchBackpressureService; import org.opensearch.search.backpressure.settings.SearchBackpressureSettings; import org.opensearch.search.pipeline.SearchPipelineService; @@ -65,6 +67,8 @@ import org.opensearch.tasks.consumer.TopNSearchTasksLogger; import org.opensearch.threadpool.RunnableTaskExecutionListener; import org.opensearch.index.store.RemoteSegmentStoreDirectoryFactory; +import org.opensearch.telemetry.TelemetryModule; +import org.opensearch.telemetry.TelemetrySettings; import org.opensearch.watcher.ResourceWatcherService; import org.opensearch.core.Assertions; import org.opensearch.Build; @@ -194,6 +198,7 @@ import org.opensearch.plugins.ScriptPlugin; import org.opensearch.plugins.SearchPlugin; import org.opensearch.plugins.SystemIndexPlugin; +import org.opensearch.plugins.TelemetryPlugin; import org.opensearch.repositories.RepositoriesModule; import org.opensearch.repositories.RepositoriesService; import org.opensearch.rest.RestController; @@ -254,6 +259,7 @@ import static java.util.stream.Collectors.toList; import static org.opensearch.common.util.FeatureFlags.SEARCH_PIPELINE; +import static org.opensearch.common.util.FeatureFlags.TELEMETRY; import static org.opensearch.env.NodeEnvironment.collectFileCacheDataPath; import static org.opensearch.index.ShardIndexingPressureSettings.SHARD_INDEXING_PRESSURE_ENABLED_ATTRIBUTE_KEY; @@ -373,6 +379,7 @@ public static class DiscoverySettings { private final Collection pluginLifecycleComponents; private final LocalNodeFactory localNodeFactory; private final NodeService nodeService; + private final TracerFactory tracerFactory; final NamedWriteableRegistry namedWriteableRegistry; private final AtomicReference runnableTaskListener; private FileCache fileCache; @@ -1021,6 +1028,16 @@ protected Node( searchModule.getIndexSearcherExecutor(threadPool) ); + if (FeatureFlags.isEnabled(TELEMETRY)) { + final TelemetrySettings telemetrySettings = new TelemetrySettings(settings, clusterService.getClusterSettings()); + List telemetryPlugins = pluginsService.filterPlugins(TelemetryPlugin.class); + TelemetryModule telemetryModule = new TelemetryModule(telemetryPlugins, telemetrySettings); + tracerFactory = new TracerFactory(telemetrySettings, telemetryModule.getTelemetry(), threadPool.getThreadContext()); + } else { + tracerFactory = new NoopTracerFactory(); + } + resourcesToClose.add(tracerFactory::close); + final List> tasksExecutors = pluginsService.filterPlugins(PersistentTaskPlugin.class) .stream() .map( @@ -1126,6 +1143,7 @@ protected Node( b.bind(FsHealthService.class).toInstance(fsHealthService); b.bind(SystemIndices.class).toInstance(systemIndices); b.bind(IdentityService.class).toInstance(identityService); + b.bind(TracerFactory.class).toInstance(this.tracerFactory); }); injector = modules.createInjector(); @@ -1481,6 +1499,9 @@ public synchronized void close() throws IOException { toClose.add(() -> stopWatch.stop().start("node_environment")); toClose.add(injector.getInstance(NodeEnvironment.class)); toClose.add(stopWatch::stop); + if (FeatureFlags.isEnabled(TELEMETRY)) { + toClose.add(() -> injector.getInstance(TracerFactory.class)); + } if (logger.isTraceEnabled()) { toClose.add(() -> logger.trace("Close times for each service:\n{}", stopWatch.prettyPrint())); diff --git a/server/src/main/java/org/opensearch/plugins/TelemetryPlugin.java b/server/src/main/java/org/opensearch/plugins/TelemetryPlugin.java new file mode 100644 index 0000000000000..33dc9b7a0c843 --- /dev/null +++ b/server/src/main/java/org/opensearch/plugins/TelemetryPlugin.java @@ -0,0 +1,25 @@ +/* + * 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.plugins; + +import org.opensearch.telemetry.Telemetry; +import org.opensearch.telemetry.TelemetrySettings; + +import java.util.Optional; + +/** + * Plugin for extending telemetry related classes + */ +public interface TelemetryPlugin { + + Optional getTelemetry(TelemetrySettings settings); + + String getName(); + +} diff --git a/server/src/main/java/org/opensearch/tasks/TaskThreadContextStatePropagator.java b/server/src/main/java/org/opensearch/tasks/TaskThreadContextStatePropagator.java new file mode 100644 index 0000000000000..ed111b34f048f --- /dev/null +++ b/server/src/main/java/org/opensearch/tasks/TaskThreadContextStatePropagator.java @@ -0,0 +1,38 @@ +/* + * 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.tasks; + +import org.opensearch.common.util.concurrent.ThreadContextStatePropagator; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.opensearch.tasks.TaskResourceTrackingService.TASK_ID; + +/** + * Propagates TASK_ID across thread contexts + */ +public class TaskThreadContextStatePropagator implements ThreadContextStatePropagator { + @Override + public Map transients(Map source) { + final Map transients = new HashMap<>(); + + if (source.containsKey(TASK_ID)) { + transients.put(TASK_ID, source.get(TASK_ID)); + } + + return transients; + } + + @Override + public Map headers(Map source) { + return Collections.emptyMap(); + } +} diff --git a/server/src/main/java/org/opensearch/telemetry/TelemetryModule.java b/server/src/main/java/org/opensearch/telemetry/TelemetryModule.java new file mode 100644 index 0000000000000..604c111b0720c --- /dev/null +++ b/server/src/main/java/org/opensearch/telemetry/TelemetryModule.java @@ -0,0 +1,47 @@ +/* + * 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.telemetry; + +import org.opensearch.plugins.TelemetryPlugin; + +import java.util.List; +import java.util.Optional; + +/** + * A module for loading classes for telemetry + * + * @opensearch.internal + */ +public class TelemetryModule { + + private Telemetry telemetry; + + public TelemetryModule(List telemetryPlugins, TelemetrySettings telemetrySettings) { + + for (TelemetryPlugin telemetryPlugin : telemetryPlugins) { + Optional telemetry = telemetryPlugin.getTelemetry(telemetrySettings); + if (telemetry.isPresent()) { + registerTelemetry(telemetry.get()); + } + } + } + + public Optional getTelemetry() { + return Optional.ofNullable(telemetry); + } + + private void registerTelemetry(Telemetry factory) { + if (telemetry == null) { + telemetry = factory; + } else { + throw new IllegalArgumentException("Cannot register more than one telemetry"); + } + } + +} diff --git a/server/src/main/java/org/opensearch/telemetry/TelemetrySettings.java b/server/src/main/java/org/opensearch/telemetry/TelemetrySettings.java new file mode 100644 index 0000000000000..7c9e0d5ac8097 --- /dev/null +++ b/server/src/main/java/org/opensearch/telemetry/TelemetrySettings.java @@ -0,0 +1,42 @@ +/* + * 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.telemetry; + +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Setting; +import org.opensearch.common.settings.Settings; + +/** + * Wrapper class to encapsulate tracing related settings + */ +public class TelemetrySettings { + public static final Setting TRACER_ENABLED_SETTING = Setting.boolSetting( + "telemetry.tracer.enabled", + false, + Setting.Property.NodeScope, + Setting.Property.Dynamic + ); + + private volatile boolean tracingEnabled; + + public TelemetrySettings(Settings settings, ClusterSettings clusterSettings) { + this.tracingEnabled = TRACER_ENABLED_SETTING.get(settings); + + clusterSettings.addSettingsUpdateConsumer(TRACER_ENABLED_SETTING, this::setTracingEnabled); + } + + public void setTracingEnabled(boolean tracingEnabled) { + this.tracingEnabled = tracingEnabled; + } + + public boolean isTracingEnabled() { + return tracingEnabled; + } + +} diff --git a/server/src/main/java/org/opensearch/telemetry/package-info.java b/server/src/main/java/org/opensearch/telemetry/package-info.java new file mode 100644 index 0000000000000..4545f0ef5990e --- /dev/null +++ b/server/src/main/java/org/opensearch/telemetry/package-info.java @@ -0,0 +1,12 @@ +/* + * 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. + */ + +/** + * This package contains classes needed for telemetry. + */ +package org.opensearch.telemetry; diff --git a/server/src/main/java/org/opensearch/telemetry/tracing/NoopTracerFactory.java b/server/src/main/java/org/opensearch/telemetry/tracing/NoopTracerFactory.java new file mode 100644 index 0000000000000..3d7f8133788ce --- /dev/null +++ b/server/src/main/java/org/opensearch/telemetry/tracing/NoopTracerFactory.java @@ -0,0 +1,33 @@ +/* + * 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.telemetry.tracing; + +import org.opensearch.telemetry.tracing.noop.NoopTracer; + +import java.util.Optional; + +/** + * No-op implementation of TracerFactory + */ +public class NoopTracerFactory extends TracerFactory { + public NoopTracerFactory() { + super(null, Optional.empty(), null); + } + + @Override + public Tracer getTracer() { + return NoopTracer.INSTANCE; + } + + @Override + public void close() { + + } + +} diff --git a/server/src/main/java/org/opensearch/telemetry/tracing/ThreadContextBasedTracerContextStorage.java b/server/src/main/java/org/opensearch/telemetry/tracing/ThreadContextBasedTracerContextStorage.java new file mode 100644 index 0000000000000..0d0b795fdc715 --- /dev/null +++ b/server/src/main/java/org/opensearch/telemetry/tracing/ThreadContextBasedTracerContextStorage.java @@ -0,0 +1,93 @@ +/* + * 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.telemetry.tracing; + +import org.opensearch.common.util.concurrent.ThreadContext; +import org.opensearch.common.util.concurrent.ThreadContextStatePropagator; + +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; + +/** + * Core's ThreadContext based TracerContextStorage implementation + */ +public class ThreadContextBasedTracerContextStorage implements TracerContextStorage, ThreadContextStatePropagator { + + private final ThreadContext threadContext; + + private final TracingTelemetry tracingTelemetry; + + public ThreadContextBasedTracerContextStorage(ThreadContext threadContext, TracingTelemetry tracingTelemetry) { + this.threadContext = Objects.requireNonNull(threadContext); + this.tracingTelemetry = Objects.requireNonNull(tracingTelemetry); + this.threadContext.registerThreadContextStatePropagator(this); + } + + @Override + public Span get(String key) { + return getCurrentSpan(key); + } + + @Override + public void put(String key, Span span) { + if (span == null) { + return; + } + SpanReference currentSpanRef = threadContext.getTransient(key); + if (currentSpanRef == null) { + threadContext.putTransient(key, new SpanReference(span)); + } else { + currentSpanRef.setSpan(span); + } + } + + @Override + public Map transients(Map source) { + final Map transients = new HashMap<>(); + + if (source.containsKey(CURRENT_SPAN)) { + final SpanReference current = (SpanReference) source.get(CURRENT_SPAN); + if (current != null) { + transients.put(CURRENT_SPAN, new SpanReference(current.getSpan())); + } + } + + return transients; + } + + @Override + public Map headers(Map source) { + final Map headers = new HashMap<>(); + + if (source.containsKey(CURRENT_SPAN)) { + final SpanReference current = (SpanReference) source.get(CURRENT_SPAN); + if (current != null) { + tracingTelemetry.getContextPropagator().inject(current.getSpan(), (key, value) -> headers.put(key, value)); + } + } + + return headers; + } + + Span getCurrentSpan(String key) { + Optional optionalSpanFromContext = spanFromThreadContext(key); + return optionalSpanFromContext.orElse(spanFromHeader()); + } + + private Optional spanFromThreadContext(String key) { + SpanReference currentSpanRef = threadContext.getTransient(key); + return (currentSpanRef == null) ? Optional.empty() : Optional.ofNullable(currentSpanRef.getSpan()); + } + + private Span spanFromHeader() { + return tracingTelemetry.getContextPropagator().extract(threadContext.getHeaders()); + } +} diff --git a/server/src/main/java/org/opensearch/telemetry/tracing/TracerFactory.java b/server/src/main/java/org/opensearch/telemetry/tracing/TracerFactory.java new file mode 100644 index 0000000000000..8228cded4c822 --- /dev/null +++ b/server/src/main/java/org/opensearch/telemetry/tracing/TracerFactory.java @@ -0,0 +1,70 @@ +/* + * 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.telemetry.tracing; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.opensearch.common.util.concurrent.ThreadContext; +import org.opensearch.telemetry.Telemetry; +import org.opensearch.telemetry.TelemetrySettings; +import org.opensearch.telemetry.tracing.noop.NoopTracer; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Optional; + +/** + * TracerManager represents a single global class that is used to access tracers. + * + * The Tracer singleton object can be retrieved using tracerManager.getTracer(). The TracerManager object + * is created during class initialization and cannot subsequently be changed. + */ +public class TracerFactory implements Closeable { + + private static final Logger logger = LogManager.getLogger(TracerFactory.class); + + private final TelemetrySettings telemetrySettings; + private final Tracer defaultTracer; + + public TracerFactory(TelemetrySettings telemetrySettings, Optional telemetry, ThreadContext threadContext) { + this.telemetrySettings = telemetrySettings; + this.defaultTracer = telemetry.map(Telemetry::getTracingTelemetry) + .map(tracingTelemetry -> createDefaultTracer(tracingTelemetry, threadContext)) + .orElse(NoopTracer.INSTANCE); + } + + /** + * Returns the tracer instance + * @return tracer instance + */ + public Tracer getTracer() { + return telemetrySettings.isTracingEnabled() ? defaultTracer : NoopTracer.INSTANCE; + } + + /** + * Closes the {@link Tracer} + */ + @Override + public void close() { + try { + defaultTracer.close(); + } catch (IOException e) { + logger.warn("Error closing tracer", e); + } + } + + private Tracer createDefaultTracer(TracingTelemetry tracingTelemetry, ThreadContext threadContext) { + TracerContextStorage tracerContextStorage = new ThreadContextBasedTracerContextStorage( + threadContext, + tracingTelemetry + ); + return new DefaultTracer(tracingTelemetry, tracerContextStorage); + } + +} diff --git a/server/src/main/java/org/opensearch/telemetry/tracing/package-info.java b/server/src/main/java/org/opensearch/telemetry/tracing/package-info.java new file mode 100644 index 0000000000000..4ac1e4c212c81 --- /dev/null +++ b/server/src/main/java/org/opensearch/telemetry/tracing/package-info.java @@ -0,0 +1,12 @@ +/* + * 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. + */ + +/** + * This package contains classes needed for tracing requests. + */ +package org.opensearch.telemetry.tracing; diff --git a/server/src/test/java/org/opensearch/telemetry/TelemetryModuleTests.java b/server/src/test/java/org/opensearch/telemetry/TelemetryModuleTests.java new file mode 100644 index 0000000000000..45344ab4253f7 --- /dev/null +++ b/server/src/test/java/org/opensearch/telemetry/TelemetryModuleTests.java @@ -0,0 +1,65 @@ +/* + * 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.telemetry; + +import org.opensearch.plugins.TelemetryPlugin; +import org.opensearch.test.OpenSearchTestCase; + +import java.util.List; +import java.util.Optional; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class TelemetryModuleTests extends OpenSearchTestCase { + + public void testGetTelemetrySuccess() { + TelemetryPlugin telemetryPlugin = mock(TelemetryPlugin.class); + when(telemetryPlugin.getName()).thenReturn("otel"); + Telemetry mockTelemetry = mock(Telemetry.class); + when(telemetryPlugin.getTelemetry(any())).thenReturn(Optional.of(mockTelemetry)); + List telemetryPlugins = List.of(telemetryPlugin); + + TelemetryModule telemetryModule = new TelemetryModule(telemetryPlugins, any()); + + assertTrue(telemetryModule.getTelemetry().isPresent()); + assertEquals(mockTelemetry, telemetryModule.getTelemetry().get()); + } + + public void testGetTelemetryWithMultipleInstalledPlugins() { + TelemetryPlugin telemetryPlugin1 = mock(TelemetryPlugin.class); + TelemetryPlugin telemetryPlugin2 = mock(TelemetryPlugin.class); + when(telemetryPlugin1.getName()).thenReturn("otel"); + Telemetry mockTelemetry1 = mock(Telemetry.class); + Telemetry mockTelemetry2 = mock(Telemetry.class); + + when(telemetryPlugin1.getTelemetry(any())).thenReturn(Optional.of(mockTelemetry1)); + when(telemetryPlugin2.getTelemetry(any())).thenReturn(Optional.of(mockTelemetry2)); + + List telemetryPlugins = List.of(telemetryPlugin1, telemetryPlugin2); + + try { + TelemetryModule telemetryModule = new TelemetryModule(telemetryPlugins, any()); + } catch (Exception e) { + assertEquals("Cannot register more than one telemetry", e.getMessage()); + } + + } + + public void testGetTelemetryWithNoPlugins() { + TelemetryPlugin telemetryPlugin = mock(TelemetryPlugin.class); + when(telemetryPlugin.getName()).thenReturn("otel"); + TelemetryModule telemetryModule = new TelemetryModule(List.of(telemetryPlugin), any()); + + assertFalse(telemetryModule.getTelemetry().isPresent()); + + } + +} diff --git a/server/src/test/java/org/opensearch/telemetry/tracing/TracerFactoryTests.java b/server/src/test/java/org/opensearch/telemetry/tracing/TracerFactoryTests.java new file mode 100644 index 0000000000000..7968c6c43afb4 --- /dev/null +++ b/server/src/test/java/org/opensearch/telemetry/tracing/TracerFactoryTests.java @@ -0,0 +1,68 @@ +/* + * 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.telemetry.tracing; + +import org.junit.After; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Setting; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.FeatureFlags; +import org.opensearch.common.util.concurrent.ThreadContext; +import org.opensearch.telemetry.Telemetry; +import org.opensearch.telemetry.TelemetrySettings; +import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.telemetry.tracing.noop.NoopTracer; + +import java.util.HashSet; +import java.util.List; +import java.util.Optional; +import java.util.Set; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class TracerFactoryTests extends OpenSearchTestCase { + + private TracerFactory tracerFactory; + + @After + public void close() { + tracerFactory.close(); + } + + public void testGetTracerWithTracingDisabledReturnsNoopTracer() { + Settings settings = Settings.builder().put(TelemetrySettings.TRACER_ENABLED_SETTING.getKey(), false).build(); + TelemetrySettings telemetrySettings = new TelemetrySettings(settings, new ClusterSettings(settings, getClusterSettings())); + Telemetry mockTelemetry = mock(Telemetry.class); + when(mockTelemetry.getTracingTelemetry()).thenReturn(mock(TracingTelemetry.class)); + tracerFactory = new TracerFactory(telemetrySettings, Optional.of(mockTelemetry), new ThreadContext(Settings.EMPTY)); + + Tracer tracer = tracerFactory.getTracer(); + assertTrue(tracer instanceof NoopTracer); + assertTrue(tracer.startSpan("foo") == Scope.NO_OP); + } + + public void testGetTracerWithTracingEnabledReturnsDefaultTracer() { + Settings settings = Settings.builder().put(TelemetrySettings.TRACER_ENABLED_SETTING.getKey(), true).build(); + TelemetrySettings telemetrySettings = new TelemetrySettings(settings, new ClusterSettings(settings, getClusterSettings())); + Telemetry mockTelemetry = mock(Telemetry.class); + when(mockTelemetry.getTracingTelemetry()).thenReturn(mock(TracingTelemetry.class)); + tracerFactory = new TracerFactory(telemetrySettings, Optional.of(mockTelemetry), new ThreadContext(Settings.EMPTY)); + + Tracer tracer = tracerFactory.getTracer(); + assertTrue(tracer instanceof DefaultTracer); + + } + + private Set> getClusterSettings() { + Set> allTracerSettings = new HashSet<>(); + ClusterSettings.FEATURE_FLAGGED_CLUSTER_SETTINGS.get(List.of(FeatureFlags.TELEMETRY)).stream().forEach((allTracerSettings::add)); + return allTracerSettings; + } +} From 3a1dd3ac2acfc5f989f43276eb1612ac3d354ac2 Mon Sep 17 00:00:00 2001 From: Owais Kazi Date: Wed, 28 Jun 2023 09:09:12 -0700 Subject: [PATCH 079/109] Added release notes for 1.3.11 (#8302) Signed-off-by: Owais Kazi --- release-notes/opensearch.release-notes-1.3.11.md | 13 +++++++++++++ 1 file changed, 13 insertions(+) create mode 100644 release-notes/opensearch.release-notes-1.3.11.md diff --git a/release-notes/opensearch.release-notes-1.3.11.md b/release-notes/opensearch.release-notes-1.3.11.md new file mode 100644 index 0000000000000..b2589d94b4f57 --- /dev/null +++ b/release-notes/opensearch.release-notes-1.3.11.md @@ -0,0 +1,13 @@ +## 2023-06-27 Version 1.3.11 Release Notes + +### Upgrades +- Upgrade `netty` from 4.1.91.Final to 4.1.93.Final ([#7901](https://github.com/opensearch-project/OpenSearch/pull/7901)) +- Upgrade `spock-core` from 1.3-groovy-2.5 to 2.3-groovy-2.5 ([#8119](https://github.com/opensearch-project/OpenSearch/pull/8119)) +- Upgrade `com.google.guava:guava` from 31.0.1-jre to 32.0.1-jre ([#8107](https://github.com/opensearch-project/OpenSearch/pull/8107)) +- Upgrade versions of gradle-info-plugin and nebula-publishing-plugin ([#8150](https://github.com/opensearch-project/OpenSearch/pull/8150)) +- Upgrade `json-smart` from 2.4.7 to 2.4.11 in plugins/repository-hdfs/ ([#8160](https://github.com/opensearch-project/OpenSearch/pull/8160)) +- Upgrade `netty` from 4.1.93.Final to 4.1.94.Final ([#8191](https://github.com/opensearch-project/OpenSearch/pull/8191)) + +### Bug Fixes +- Fix mapping char_filter when mapping a hashtag ([#7591](https://github.com/opensearch-project/OpenSearch/pull/7591)) +- Force nebula plugins to use latest org.bouncycastle:* artifacts ([#8233](https://github.com/opensearch-project/OpenSearch/pull/8233)) From b594f52f655e3ec5de57cb827b9b02800ce1666b Mon Sep 17 00:00:00 2001 From: Harish Bhakuni Date: Wed, 28 Jun 2023 10:03:17 -0700 Subject: [PATCH 080/109] [Snapshot Interop] Change Version Checks from 3.0 to 2.9 in Restore Snapshot Changes. (#8319) Signed-off-by: Harish Bhakuni Co-authored-by: Harish Bhakuni --- .../cluster/snapshots/restore/RestoreSnapshotRequest.java | 4 ++-- .../java/org/opensearch/cluster/routing/RecoverySource.java | 4 ++-- .../main/java/org/opensearch/snapshots/RestoreService.java | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) 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 7ff16fcf6fc03..9ab66d726854e 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 @@ -150,7 +150,7 @@ public RestoreSnapshotRequest(StreamInput in) throws IOException { if (in.getVersion().onOrAfter(Version.V_2_7_0)) { storageType = in.readEnum(StorageType.class); } - if (in.getVersion().onOrAfter(Version.V_3_0_0)) { + if (in.getVersion().onOrAfter(Version.V_2_9_0)) { sourceRemoteStoreRepository = in.readOptionalString(); } } @@ -174,7 +174,7 @@ public void writeTo(StreamOutput out) throws IOException { if (out.getVersion().onOrAfter(Version.V_2_7_0)) { out.writeEnum(storageType); } - if (out.getVersion().onOrAfter(Version.V_3_0_0)) { + if (out.getVersion().onOrAfter(Version.V_2_9_0)) { out.writeOptionalString(sourceRemoteStoreRepository); } } diff --git a/server/src/main/java/org/opensearch/cluster/routing/RecoverySource.java b/server/src/main/java/org/opensearch/cluster/routing/RecoverySource.java index 1af4fbe8ffb45..41e06ac2a8b24 100644 --- a/server/src/main/java/org/opensearch/cluster/routing/RecoverySource.java +++ b/server/src/main/java/org/opensearch/cluster/routing/RecoverySource.java @@ -293,7 +293,7 @@ public SnapshotRecoverySource( } else { isSearchableSnapshot = false; } - if (in.getVersion().onOrAfter(Version.V_3_0_0)) { + if (in.getVersion().onOrAfter(Version.V_2_9_0)) { remoteStoreIndexShallowCopy = in.readBoolean(); sourceRemoteStoreRepository = in.readOptionalString(); } else { @@ -345,7 +345,7 @@ protected void writeAdditionalFields(StreamOutput out) throws IOException { if (out.getVersion().onOrAfter(Version.V_2_7_0)) { out.writeBoolean(isSearchableSnapshot); } - if (out.getVersion().onOrAfter(Version.V_3_0_0)) { + if (out.getVersion().onOrAfter(Version.V_2_9_0)) { out.writeBoolean(remoteStoreIndexShallowCopy); out.writeOptionalString(sourceRemoteStoreRepository); } diff --git a/server/src/main/java/org/opensearch/snapshots/RestoreService.java b/server/src/main/java/org/opensearch/snapshots/RestoreService.java index bd162914e830e..ebd0e59599c21 100644 --- a/server/src/main/java/org/opensearch/snapshots/RestoreService.java +++ b/server/src/main/java/org/opensearch/snapshots/RestoreService.java @@ -453,7 +453,7 @@ public ClusterState execute(ClusterState currentState) { final boolean isRemoteStoreShallowCopy = Boolean.TRUE.equals( snapshotInfo.isRemoteStoreIndexShallowCopyEnabled() ) && metadata.index(index).getSettings().getAsBoolean(SETTING_REMOTE_STORE_ENABLED, false); - if (isRemoteStoreShallowCopy && !currentState.getNodes().getMinNodeVersion().onOrAfter(Version.V_3_0_0)) { + if (isRemoteStoreShallowCopy && !currentState.getNodes().getMinNodeVersion().onOrAfter(Version.V_2_9_0)) { throw new SnapshotRestoreException( snapshot, "cannot restore shallow copy snapshot for index [" From 270abeba62a78d4ab0aba58854ae4458d2f6bd90 Mon Sep 17 00:00:00 2001 From: Suraj Singh Date: Wed, 28 Jun 2023 11:47:01 -0700 Subject: [PATCH 081/109] [Segment Replication] Update bwc test to rely on segments for verification (#8267) * [Segment Replication] Use _cat/segments vs index stats + _search to verify doc count Signed-off-by: Suraj Singh Self review Signed-off-by: Suraj Singh remove unused imports Signed-off-by: Suraj Singh Handle 0 doc count segments Signed-off-by: Suraj Singh * Use 1 minute timeout for assertBusy validations and comments Signed-off-by: Suraj Singh --------- Signed-off-by: Suraj Singh --- .../org/opensearch/upgrades/IndexingIT.java | 92 +++++++++---------- 1 file changed, 43 insertions(+), 49 deletions(-) diff --git a/qa/rolling-upgrade/src/test/java/org/opensearch/upgrades/IndexingIT.java b/qa/rolling-upgrade/src/test/java/org/opensearch/upgrades/IndexingIT.java index 173aa9f6557d2..93c0bc96a5183 100644 --- a/qa/rolling-upgrade/src/test/java/org/opensearch/upgrades/IndexingIT.java +++ b/qa/rolling-upgrade/src/test/java/org/opensearch/upgrades/IndexingIT.java @@ -48,16 +48,12 @@ import java.io.IOException; import java.net.URISyntaxException; import java.nio.charset.StandardCharsets; -import java.util.Arrays; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; import static org.opensearch.cluster.routing.UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING; import static org.opensearch.rest.action.search.RestSearchAction.TOTAL_HITS_AS_INT_PARAM; -import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.either; /** * Basic test that indexed documents survive the rolling restart. See @@ -88,52 +84,51 @@ private void printClusterNodes() throws IOException, ParseException, URISyntaxEx } // Verifies that for each shard copy holds same document count across all containing nodes. - private void waitForSearchableDocs(String index, int shardCount) throws Exception { - Map primaryShardToNodeIDMap = new HashMap<>(); - Map replicaShardToNodeIDMap = new HashMap<>(); + private void waitForSearchableDocs(String index, int shardCount, int replicaCount) throws Exception { + assertTrue(shardCount > 0); + assertTrue(replicaCount > 0); waitForClusterHealthWithNoShardMigration(index, "green"); logger.info("--> _cat/shards before search \n{}", EntityUtils.toString(client().performRequest(new Request("GET", "/_cat/shards?v")).getEntity())); + // Verify segment replication stats verifySegmentStats(index); - Request request = new Request("GET", index + "/_stats"); - request.addParameter("level", "shards"); - Response response = client().performRequest(request); - for (int shardNumber = 0; shardNumber < shardCount; shardNumber++) { - List shardStats = ObjectPath.createFromResponse(response).evaluate("indices." + index + ".shards." + shardNumber); - for (Object shard : shardStats) { - final String nodeId = ObjectPath.evaluate(shard, "routing.node"); - final Boolean primary = ObjectPath.evaluate(shard, "routing.primary"); - if (primary) { - primaryShardToNodeIDMap.putIfAbsent(shardNumber, nodeId); - } else { - replicaShardToNodeIDMap.putIfAbsent(shardNumber, nodeId); + + // Verify segment store + assertBusy(() -> { + /** + * Use default tabular output and sort response based on shard,segment,primaryOrReplica columns to allow line by + * line parsing where records related to a segment (e.g. _0) are chunked together with first record belonging + * to primary while remaining *replicaCount* records belongs to replica copies + * */ + Request segrepStatsRequest = new Request("GET", "/_cat/segments/" + index + "?s=shard,segment,primaryOrReplica"); + segrepStatsRequest.addParameter("h", "index,shard,primaryOrReplica,segment,docs.count"); + Response segrepStatsResponse = client().performRequest(segrepStatsRequest); + logger.info("--> _cat/segments response\n {}", EntityUtils.toString(segrepStatsResponse.getEntity())); + List responseList = Streams.readAllLines(segrepStatsResponse.getEntity().getContent()); + for (int segmentsIndex=0; segmentsIndex < responseList.size();) { + String[] primaryRow = responseList.get(segmentsIndex++).split(" +"); + String shardId = primaryRow[0] + primaryRow[1]; + assertTrue(primaryRow[2].equals("p")); + for(int replicaIndex = 1; replicaIndex <= replicaCount; replicaIndex++) { + String[] replicaRow = responseList.get(segmentsIndex).split(" +"); + String replicaShardId = replicaRow[0] + replicaRow[1]; + // When segment has 0 doc count, not all replica copies posses that segment. Skip to next segment + if (replicaRow[2].equals("p")) { + assertTrue(primaryRow[4].equals("0")); + break; + } + // verify same shard id + assertTrue(replicaShardId.equals(shardId)); + // verify replica row + assertTrue(replicaRow[2].equals("r")); + // Verify segment name matches e.g. _0 + assertTrue(replicaRow[3].equals(primaryRow[3])); + // Verify doc count matches + assertTrue(replicaRow[4].equals(primaryRow[4])); + segmentsIndex++; } } - } - logger.info("--> primaryShardToNodeIDMap {}", primaryShardToNodeIDMap); - logger.info("--> replicaShardToNodeIDMap {}", replicaShardToNodeIDMap); - - for (int shardNumber = 0; shardNumber < shardCount; shardNumber++) { - Request searchTestIndexRequest = new Request("POST", "/" + index + "/_search"); - searchTestIndexRequest.addParameter(TOTAL_HITS_AS_INT_PARAM, "true"); - searchTestIndexRequest.addParameter("filter_path", "hits.total"); - searchTestIndexRequest.addParameter("preference", "_shards:" + shardNumber + "|_only_nodes:" + primaryShardToNodeIDMap.get(shardNumber)); - Response searchTestIndexResponse = client().performRequest(searchTestIndexRequest); - final int primaryHits = ObjectPath.createFromResponse(searchTestIndexResponse).evaluate("hits.total"); - final int shardNum = shardNumber; - // Verify replica shard doc count only when available. - if (replicaShardToNodeIDMap.get(shardNum) != null) { - assertBusy(() -> { - Request replicaRequest = new Request("POST", "/" + index + "/_search"); - replicaRequest.addParameter(TOTAL_HITS_AS_INT_PARAM, "true"); - replicaRequest.addParameter("filter_path", "hits.total"); - replicaRequest.addParameter("preference", "_shards:" + shardNum + "|_only_nodes:" + replicaShardToNodeIDMap.get(shardNum)); - Response replicaResponse = client().performRequest(replicaRequest); - int replicaHits = ObjectPath.createFromResponse(replicaResponse).evaluate("hits.total"); - assertEquals("Doc count mismatch for shard " + shardNum + ". Primary hits " + primaryHits + " Replica hits " + replicaHits, primaryHits, replicaHits); - }, 1, TimeUnit.MINUTES); - } - } + }, 1, TimeUnit.MINUTES); } private void waitForClusterHealthWithNoShardMigration(String indexName, String status) throws IOException { @@ -156,7 +151,7 @@ private void verifySegmentStats(String indexName) throws Exception { String[] elements = statLine.split(" +"); assertEquals("Replica shard " + elements[0] + "not upto date with primary ", 0, Integer.parseInt(elements[2])); } - }); + }, 1, TimeUnit.MINUTES); } public void testIndexing() throws IOException, ParseException { @@ -307,7 +302,7 @@ public void testIndexingWithSegRep() throws Exception { throw new UnsupportedOperationException("Unknown cluster type [" + CLUSTER_TYPE + "]"); } - waitForSearchableDocs(indexName, shardCount); + waitForSearchableDocs(indexName, shardCount, replicaCount); assertCount(indexName, expectedCount); if (CLUSTER_TYPE != ClusterType.OLD) { @@ -318,17 +313,16 @@ public void testIndexingWithSegRep() throws Exception { toBeDeleted.addParameter("refresh", "true"); toBeDeleted.setJsonEntity("{\"f1\": \"delete-me\"}"); client().performRequest(toBeDeleted); - waitForSearchableDocs(indexName, shardCount); + waitForSearchableDocs(indexName, shardCount, replicaCount); assertCount(indexName, expectedCount + 6); logger.info("--> Delete previously added doc and verify doc count"); Request delete = new Request("DELETE", "/" + indexName + "/_doc/to_be_deleted"); delete.addParameter("refresh", "true"); client().performRequest(delete); - waitForSearchableDocs(indexName, shardCount); + waitForSearchableDocs(indexName, shardCount, replicaCount); assertCount(indexName, expectedCount + 5); } - logger.info("--> _cat/shards post execution \n{}", EntityUtils.toString(client().performRequest(new Request("GET", "/_cat/shards?v")).getEntity())); } public void testAutoIdWithOpTypeCreate() throws IOException { From 9aeb15699ef30312715a624af81aa826c4e4c42e Mon Sep 17 00:00:00 2001 From: Nick Knize Date: Wed, 28 Jun 2023 14:57:12 -0500 Subject: [PATCH 082/109] [Mute] SegRepUsingRemoteStoreIT.testPressureServiceStats (#8324) Mutes repeated failing test. Signed-off-by: Nicholas Walter Knize --- .../remotestore/SegmentReplicationUsingRemoteStoreIT.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationUsingRemoteStoreIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationUsingRemoteStoreIT.java index ad6e358cb9da1..01fb91f83aa02 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationUsingRemoteStoreIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/SegmentReplicationUsingRemoteStoreIT.java @@ -63,4 +63,10 @@ public void setup() { public void teardown() { assertAcked(clusterAdmin().prepareDeleteRepository(REPOSITORY_NAME)); } + + @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/7592") + @Override + public void testPressureServiceStats() throws Exception { + super.testPressureServiceStats(); + } } From 9856cb77bc1bf6908210d91af7f95cc29d41b3a5 Mon Sep 17 00:00:00 2001 From: Andrey Pleskach Date: Wed, 28 Jun 2023 23:12:09 +0200 Subject: [PATCH 083/109] Bump BouncyCastle from jdk15on to jdk15to18 (#8247) jdk15on are not supported anymore since based development was moved from jsk15on to jdk18on. jdk15to18 contains fixed for: - CVE-2023-33201 - CVE-2022-45146 Signed-off-by: Andrey Pleskach --- CHANGELOG.md | 5 +++++ buildSrc/version.properties | 2 +- plugins/identity-shiro/build.gradle | 2 +- .../identity-shiro/licenses/bcprov-jdk15on-1.70.jar.sha1 | 1 - .../identity-shiro/licenses/bcprov-jdk15to18-1.75.jar.sha1 | 1 + ...rov-jdk15on-LICENSE.txt => bcprov-jdk15to18-LICENSE.txt} | 0 ...cprov-jdk15on-NOTICE.txt => bcprov-jdk15to18-NOTICE.txt} | 0 plugins/ingest-attachment/build.gradle | 6 +++--- .../ingest-attachment/licenses/bcmail-jdk15on-1.70.jar.sha1 | 1 - .../licenses/bcmail-jdk15to18-1.75.jar.sha1 | 1 + ...ail-jdk15on-LICENSE.txt => bcmail-jdk15to18-LICENSE.txt} | 0 ...cmail-jdk15on-NOTICE.txt => bcmail-jdk15to18-NOTICE.txt} | 0 .../ingest-attachment/licenses/bcpkix-jdk15on-1.70.jar.sha1 | 1 - .../licenses/bcpkix-jdk15to18-1.75.jar.sha1 | 1 + ...kix-jdk15on-LICENSE.txt => bcpkix-jdk15to18-LICENSE.txt} | 0 ...cpkix-jdk15on-NOTICE.txt => bcpkix-jdk15to18-NOTICE.txt} | 0 .../ingest-attachment/licenses/bcprov-jdk15on-1.70.jar.sha1 | 1 - .../licenses/bcprov-jdk15to18-1.75.jar.sha1 | 1 + ...rov-jdk15on-LICENSE.txt => bcprov-jdk15to18-LICENSE.txt} | 0 ...cprov-jdk15on-NOTICE.txt => bcprov-jdk15to18-NOTICE.txt} | 0 .../src/main/plugin-metadata/plugin-security.policy | 3 --- test/fixtures/hdfs-fixture/build.gradle | 2 +- 22 files changed, 15 insertions(+), 13 deletions(-) delete mode 100644 plugins/identity-shiro/licenses/bcprov-jdk15on-1.70.jar.sha1 create mode 100644 plugins/identity-shiro/licenses/bcprov-jdk15to18-1.75.jar.sha1 rename plugins/identity-shiro/licenses/{bcprov-jdk15on-LICENSE.txt => bcprov-jdk15to18-LICENSE.txt} (100%) rename plugins/identity-shiro/licenses/{bcprov-jdk15on-NOTICE.txt => bcprov-jdk15to18-NOTICE.txt} (100%) delete mode 100644 plugins/ingest-attachment/licenses/bcmail-jdk15on-1.70.jar.sha1 create mode 100644 plugins/ingest-attachment/licenses/bcmail-jdk15to18-1.75.jar.sha1 rename plugins/ingest-attachment/licenses/{bcmail-jdk15on-LICENSE.txt => bcmail-jdk15to18-LICENSE.txt} (100%) rename plugins/ingest-attachment/licenses/{bcmail-jdk15on-NOTICE.txt => bcmail-jdk15to18-NOTICE.txt} (100%) delete mode 100644 plugins/ingest-attachment/licenses/bcpkix-jdk15on-1.70.jar.sha1 create mode 100644 plugins/ingest-attachment/licenses/bcpkix-jdk15to18-1.75.jar.sha1 rename plugins/ingest-attachment/licenses/{bcpkix-jdk15on-LICENSE.txt => bcpkix-jdk15to18-LICENSE.txt} (100%) rename plugins/ingest-attachment/licenses/{bcpkix-jdk15on-NOTICE.txt => bcpkix-jdk15to18-NOTICE.txt} (100%) delete mode 100644 plugins/ingest-attachment/licenses/bcprov-jdk15on-1.70.jar.sha1 create mode 100644 plugins/ingest-attachment/licenses/bcprov-jdk15to18-1.75.jar.sha1 rename plugins/ingest-attachment/licenses/{bcprov-jdk15on-LICENSE.txt => bcprov-jdk15to18-LICENSE.txt} (100%) rename plugins/ingest-attachment/licenses/{bcprov-jdk15on-NOTICE.txt => bcprov-jdk15to18-NOTICE.txt} (100%) diff --git a/CHANGELOG.md b/CHANGELOG.md index 8c370dbe7b0d9..7f4a6f38b578e 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -36,6 +36,11 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - OpenJDK Update (April 2023 Patch releases) ([#7344](https://github.com/opensearch-project/OpenSearch/pull/7344) - Bump `com.google.http-client:google-http-client:1.43.2` from 1.42.0 to 1.43.2 ([7928](https://github.com/opensearch-project/OpenSearch/pull/7928))) - Add Opentelemetry dependencies ([#7543](https://github.com/opensearch-project/OpenSearch/issues/7543)) +- Bump `org.bouncycastle:bcprov-jdk15on` to `org.bouncycastle:bcprov-jdk15to18` version 1.75 ([8247](https://github.com/opensearch-project/OpenSearch/pull/8247)) +- Bump `org.bouncycastle:bcmail-jdk15on` to `org.bouncycastle:bcmail-jdk15to18` version 1.75 ([8247](https://github.com/opensearch-project/OpenSearch/pull/8247)) +- Bump `org.bouncycastle:bcpkix-jdk15on` to `org.bouncycastle:bcpkix-jdk15to18` version 1.75 ([8247](https://github.com/opensearch-project/OpenSearch/pull/8247)) + + ### Changed - [CCR] Add getHistoryOperationsFromTranslog method to fetch the history snapshot from translogs ([#3948](https://github.com/opensearch-project/OpenSearch/pull/3948)) diff --git a/buildSrc/version.properties b/buildSrc/version.properties index 735a9fe4f8255..f9eac9516cb18 100644 --- a/buildSrc/version.properties +++ b/buildSrc/version.properties @@ -48,7 +48,7 @@ reactivestreams = 1.0.4 # when updating this version, you need to ensure compatibility with: # - plugins/ingest-attachment (transitive dependency, check the upstream POM) # - distribution/tools/plugin-cli -bouncycastle=1.70 +bouncycastle=1.75 # test dependencies randomizedrunner = 2.7.1 junit = 4.13.2 diff --git a/plugins/identity-shiro/build.gradle b/plugins/identity-shiro/build.gradle index 22dc21864b620..baa3464d0a98e 100644 --- a/plugins/identity-shiro/build.gradle +++ b/plugins/identity-shiro/build.gradle @@ -28,7 +28,7 @@ dependencies { implementation 'org.passay:passay:1.6.3' - implementation "org.bouncycastle:bcprov-jdk15on:${versions.bouncycastle}" + implementation "org.bouncycastle:bcprov-jdk15to18:${versions.bouncycastle}" testImplementation project(path: ':modules:transport-netty4') // for http testImplementation project(path: ':plugins:transport-nio') // for http diff --git a/plugins/identity-shiro/licenses/bcprov-jdk15on-1.70.jar.sha1 b/plugins/identity-shiro/licenses/bcprov-jdk15on-1.70.jar.sha1 deleted file mode 100644 index f5e89c0f5ed45..0000000000000 --- a/plugins/identity-shiro/licenses/bcprov-jdk15on-1.70.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -4636a0d01f74acaf28082fb62b317f1080118371 \ No newline at end of file diff --git a/plugins/identity-shiro/licenses/bcprov-jdk15to18-1.75.jar.sha1 b/plugins/identity-shiro/licenses/bcprov-jdk15to18-1.75.jar.sha1 new file mode 100644 index 0000000000000..9911bb75f9209 --- /dev/null +++ b/plugins/identity-shiro/licenses/bcprov-jdk15to18-1.75.jar.sha1 @@ -0,0 +1 @@ +df22e1b6a9f6b218913f5b68dd16641344397fe0 \ No newline at end of file diff --git a/plugins/identity-shiro/licenses/bcprov-jdk15on-LICENSE.txt b/plugins/identity-shiro/licenses/bcprov-jdk15to18-LICENSE.txt similarity index 100% rename from plugins/identity-shiro/licenses/bcprov-jdk15on-LICENSE.txt rename to plugins/identity-shiro/licenses/bcprov-jdk15to18-LICENSE.txt diff --git a/plugins/identity-shiro/licenses/bcprov-jdk15on-NOTICE.txt b/plugins/identity-shiro/licenses/bcprov-jdk15to18-NOTICE.txt similarity index 100% rename from plugins/identity-shiro/licenses/bcprov-jdk15on-NOTICE.txt rename to plugins/identity-shiro/licenses/bcprov-jdk15to18-NOTICE.txt diff --git a/plugins/ingest-attachment/build.gradle b/plugins/ingest-attachment/build.gradle index 1f3c80909733c..62651216c8144 100644 --- a/plugins/ingest-attachment/build.gradle +++ b/plugins/ingest-attachment/build.gradle @@ -71,9 +71,9 @@ dependencies { api "org.apache.pdfbox:fontbox:${versions.pdfbox}" api "org.apache.pdfbox:jempbox:1.8.17" api "commons-logging:commons-logging:${versions.commonslogging}" - api "org.bouncycastle:bcmail-jdk15on:${versions.bouncycastle}" - api "org.bouncycastle:bcprov-jdk15on:${versions.bouncycastle}" - api "org.bouncycastle:bcpkix-jdk15on:${versions.bouncycastle}" + api "org.bouncycastle:bcmail-jdk15to18:${versions.bouncycastle}" + api "org.bouncycastle:bcprov-jdk15to18:${versions.bouncycastle}" + api "org.bouncycastle:bcpkix-jdk15to18:${versions.bouncycastle}" // OpenOffice api "org.apache.poi:poi-ooxml:${versions.poi}" api "org.apache.poi:poi:${versions.poi}" diff --git a/plugins/ingest-attachment/licenses/bcmail-jdk15on-1.70.jar.sha1 b/plugins/ingest-attachment/licenses/bcmail-jdk15on-1.70.jar.sha1 deleted file mode 100644 index 672e479eda8d7..0000000000000 --- a/plugins/ingest-attachment/licenses/bcmail-jdk15on-1.70.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -08f4aafad90f6cc7f16b9992279828ae848c9e0d \ No newline at end of file diff --git a/plugins/ingest-attachment/licenses/bcmail-jdk15to18-1.75.jar.sha1 b/plugins/ingest-attachment/licenses/bcmail-jdk15to18-1.75.jar.sha1 new file mode 100644 index 0000000000000..e6840a9b02b38 --- /dev/null +++ b/plugins/ingest-attachment/licenses/bcmail-jdk15to18-1.75.jar.sha1 @@ -0,0 +1 @@ +b316bcd094e3917b1ece93a6edbab93f8315fb3b \ No newline at end of file diff --git a/plugins/ingest-attachment/licenses/bcmail-jdk15on-LICENSE.txt b/plugins/ingest-attachment/licenses/bcmail-jdk15to18-LICENSE.txt similarity index 100% rename from plugins/ingest-attachment/licenses/bcmail-jdk15on-LICENSE.txt rename to plugins/ingest-attachment/licenses/bcmail-jdk15to18-LICENSE.txt diff --git a/plugins/ingest-attachment/licenses/bcmail-jdk15on-NOTICE.txt b/plugins/ingest-attachment/licenses/bcmail-jdk15to18-NOTICE.txt similarity index 100% rename from plugins/ingest-attachment/licenses/bcmail-jdk15on-NOTICE.txt rename to plugins/ingest-attachment/licenses/bcmail-jdk15to18-NOTICE.txt diff --git a/plugins/ingest-attachment/licenses/bcpkix-jdk15on-1.70.jar.sha1 b/plugins/ingest-attachment/licenses/bcpkix-jdk15on-1.70.jar.sha1 deleted file mode 100644 index e348463a21257..0000000000000 --- a/plugins/ingest-attachment/licenses/bcpkix-jdk15on-1.70.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -f81e5af49571a9d5a109a88f239a73ce87055417 \ No newline at end of file diff --git a/plugins/ingest-attachment/licenses/bcpkix-jdk15to18-1.75.jar.sha1 b/plugins/ingest-attachment/licenses/bcpkix-jdk15to18-1.75.jar.sha1 new file mode 100644 index 0000000000000..9181b1c3ab1b6 --- /dev/null +++ b/plugins/ingest-attachment/licenses/bcpkix-jdk15to18-1.75.jar.sha1 @@ -0,0 +1 @@ +f16e5252ad7a46d5eaf255231b0a5da307599082 \ No newline at end of file diff --git a/plugins/ingest-attachment/licenses/bcpkix-jdk15on-LICENSE.txt b/plugins/ingest-attachment/licenses/bcpkix-jdk15to18-LICENSE.txt similarity index 100% rename from plugins/ingest-attachment/licenses/bcpkix-jdk15on-LICENSE.txt rename to plugins/ingest-attachment/licenses/bcpkix-jdk15to18-LICENSE.txt diff --git a/plugins/ingest-attachment/licenses/bcpkix-jdk15on-NOTICE.txt b/plugins/ingest-attachment/licenses/bcpkix-jdk15to18-NOTICE.txt similarity index 100% rename from plugins/ingest-attachment/licenses/bcpkix-jdk15on-NOTICE.txt rename to plugins/ingest-attachment/licenses/bcpkix-jdk15to18-NOTICE.txt diff --git a/plugins/ingest-attachment/licenses/bcprov-jdk15on-1.70.jar.sha1 b/plugins/ingest-attachment/licenses/bcprov-jdk15on-1.70.jar.sha1 deleted file mode 100644 index f5e89c0f5ed45..0000000000000 --- a/plugins/ingest-attachment/licenses/bcprov-jdk15on-1.70.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -4636a0d01f74acaf28082fb62b317f1080118371 \ No newline at end of file diff --git a/plugins/ingest-attachment/licenses/bcprov-jdk15to18-1.75.jar.sha1 b/plugins/ingest-attachment/licenses/bcprov-jdk15to18-1.75.jar.sha1 new file mode 100644 index 0000000000000..9911bb75f9209 --- /dev/null +++ b/plugins/ingest-attachment/licenses/bcprov-jdk15to18-1.75.jar.sha1 @@ -0,0 +1 @@ +df22e1b6a9f6b218913f5b68dd16641344397fe0 \ No newline at end of file diff --git a/plugins/ingest-attachment/licenses/bcprov-jdk15on-LICENSE.txt b/plugins/ingest-attachment/licenses/bcprov-jdk15to18-LICENSE.txt similarity index 100% rename from plugins/ingest-attachment/licenses/bcprov-jdk15on-LICENSE.txt rename to plugins/ingest-attachment/licenses/bcprov-jdk15to18-LICENSE.txt diff --git a/plugins/ingest-attachment/licenses/bcprov-jdk15on-NOTICE.txt b/plugins/ingest-attachment/licenses/bcprov-jdk15to18-NOTICE.txt similarity index 100% rename from plugins/ingest-attachment/licenses/bcprov-jdk15on-NOTICE.txt rename to plugins/ingest-attachment/licenses/bcprov-jdk15to18-NOTICE.txt diff --git a/plugins/ingest-attachment/src/main/plugin-metadata/plugin-security.policy b/plugins/ingest-attachment/src/main/plugin-metadata/plugin-security.policy index 0fa85f6f040f6..4b90f9a21aae4 100644 --- a/plugins/ingest-attachment/src/main/plugin-metadata/plugin-security.policy +++ b/plugins/ingest-attachment/src/main/plugin-metadata/plugin-security.policy @@ -35,9 +35,6 @@ grant { // needed to apply additional sandboxing to tika parsing permission java.security.SecurityPermission "createAccessControlContext"; - // TODO: fix PDFBox not to actually install bouncy castle like this - permission java.security.SecurityPermission "putProviderProperty.BC"; - permission java.security.SecurityPermission "insertProvider"; // TODO: fix POI XWPF to not do this: https://bz.apache.org/bugzilla/show_bug.cgi?id=58597 permission java.lang.reflect.ReflectPermission "suppressAccessChecks"; // needed by xmlbeans, as part of POI for MS xml docs diff --git a/test/fixtures/hdfs-fixture/build.gradle b/test/fixtures/hdfs-fixture/build.gradle index de6f69a4fd4ce..310b088674d5c 100644 --- a/test/fixtures/hdfs-fixture/build.gradle +++ b/test/fixtures/hdfs-fixture/build.gradle @@ -51,7 +51,7 @@ dependencies { api "org.apache.logging.log4j:log4j-core:${versions.log4j}" api "io.netty:netty-all:${versions.netty}" api 'com.google.code.gson:gson:2.10.1' - api "org.bouncycastle:bcpkix-jdk15on:${versions.bouncycastle}" + api "org.bouncycastle:bcpkix-jdk15to18:${versions.bouncycastle}" api "com.fasterxml.jackson.jaxrs:jackson-jaxrs-json-provider:${versions.jackson}" api "com.fasterxml.jackson.core:jackson-databind:${versions.jackson_databind}" api "com.fasterxml.woodstox:woodstox-core:${versions.woodstox}" From 46c9a211b6b9490f6a7ac9425e946986cd51bed2 Mon Sep 17 00:00:00 2001 From: Michael Froh Date: Wed, 28 Jun 2023 14:26:53 -0700 Subject: [PATCH 084/109] [Search Pipelines] Add stats for search pipelines (#8053) * [Search Pipelines] Add stats for search pipelines This adds statistics on executions and time spent on search pipeline operations, similar to the stats that are available for ingest pipelines. Signed-off-by: Michael Froh * Compare parsed JSON structure, not exact JSON string As @lukas-vlcek pointed out, asserting equality with an exact JSON string is sensitive to formatting, which makes the test brittle. Instead, we can parse the expected JSON and compare as Maps. Signed-off-by: Michael Froh * Refactor to common stats/metrics classes Search pipelines and ingest pipelines had identical functionality for tracking metrics around operations and converting those to immutable "stats" objects. That approach isn't even really specific to pipelines, but can be used to track metrics on any repeated operation, so I moved that common logic to the common.metrics package. Signed-off-by: Michael Froh * Split pipeline metrics tracking into its own class Thanks @saratvemulapalli for the suggestion! This lets the Pipeline class focus on transforming requests / responses, while the subclass focuses on tracking and managing metrics. Signed-off-by: Michael Froh --------- Signed-off-by: Michael Froh --- CHANGELOG.md | 1 + .../ingest/common/IngestRestartIT.java | 2 +- .../admin/cluster/node/stats/NodeStats.java | 24 +- .../cluster/node/stats/NodesStatsRequest.java | 3 +- .../node/stats/TransportNodesStatsAction.java | 3 +- .../cluster/stats/ClusterStatsNodes.java | 19 +- .../stats/TransportClusterStatsAction.java | 1 + .../opensearch/common/metrics/MeanMetric.java | 5 + .../common/metrics/OperationMetrics.java | 68 ++++ .../common/metrics/OperationStats.java | 107 +++++ .../opensearch/ingest/CompoundProcessor.java | 17 +- .../ingest/ConditionalProcessor.java | 13 +- .../org/opensearch/ingest/IngestMetric.java | 112 ------ .../org/opensearch/ingest/IngestService.java | 39 +- .../org/opensearch/ingest/IngestStats.java | 131 +------ .../java/org/opensearch/ingest/Pipeline.java | 13 +- .../java/org/opensearch/node/NodeService.java | 6 +- .../opensearch/search/pipeline/Pipeline.java | 170 ++++---- .../search/pipeline/PipelineWithMetrics.java | 227 +++++++++++ .../pipeline/SearchPipelineService.java | 52 ++- .../search/pipeline/SearchPipelineStats.java | 367 ++++++++++++++++++ .../cluster/node/stats/NodeStatsTests.java | 49 +-- .../cluster/stats/ClusterStatsNodesTests.java | 16 +- .../opensearch/cluster/DiskUsageTests.java | 6 + .../ingest/CompoundProcessorTests.java | 11 +- .../ingest/ConditionalProcessorTests.java | 11 +- .../opensearch/ingest/IngestServiceTests.java | 13 +- .../opensearch/ingest/IngestStatsTests.java | 27 +- .../ingest/PipelineProcessorTests.java | 31 +- .../pipeline/SearchPipelineServiceTests.java | 123 ++++++ .../pipeline/SearchPipelineStatsTests.java | 185 +++++++++ .../MockInternalClusterInfoService.java | 3 +- .../opensearch/test/InternalTestCluster.java | 1 + 33 files changed, 1398 insertions(+), 458 deletions(-) create mode 100644 server/src/main/java/org/opensearch/common/metrics/OperationMetrics.java create mode 100644 server/src/main/java/org/opensearch/common/metrics/OperationStats.java delete mode 100644 server/src/main/java/org/opensearch/ingest/IngestMetric.java create mode 100644 server/src/main/java/org/opensearch/search/pipeline/PipelineWithMetrics.java create mode 100644 server/src/main/java/org/opensearch/search/pipeline/SearchPipelineStats.java create mode 100644 server/src/test/java/org/opensearch/search/pipeline/SearchPipelineStatsTests.java diff --git a/CHANGELOG.md b/CHANGELOG.md index 7f4a6f38b578e..109401a46e5b7 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -86,6 +86,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Implement concurrent aggregations support without profile option ([#7514](https://github.com/opensearch-project/OpenSearch/pull/7514)) - Add dynamic index and cluster setting for concurrent segment search ([#7956](https://github.com/opensearch-project/OpenSearch/pull/7956)) - Add descending order search optimization through reverse segment read. ([#7967](https://github.com/opensearch-project/OpenSearch/pull/7967)) +- [Search pipelines] Added search pipelines output to node stats ([#8053](https://github.com/opensearch-project/OpenSearch/pull/8053)) - Update components of segrep backpressure to support remote store. ([#8020](https://github.com/opensearch-project/OpenSearch/pull/8020)) - Make remote cluster connection setup in async ([#8038](https://github.com/opensearch-project/OpenSearch/pull/8038)) - Add API to initialize extensions ([#8029]()https://github.com/opensearch-project/OpenSearch/pull/8029) diff --git a/modules/ingest-common/src/internalClusterTest/java/org/opensearch/ingest/common/IngestRestartIT.java b/modules/ingest-common/src/internalClusterTest/java/org/opensearch/ingest/common/IngestRestartIT.java index 784dad8cea49f..1f0680b27796d 100644 --- a/modules/ingest-common/src/internalClusterTest/java/org/opensearch/ingest/common/IngestRestartIT.java +++ b/modules/ingest-common/src/internalClusterTest/java/org/opensearch/ingest/common/IngestRestartIT.java @@ -132,7 +132,7 @@ public void testFailureInConditionalProcessor() { for (int k = 0; k < nodeCount; k++) { List stats = r.getNodes().get(k).getIngestStats().getProcessorStats().get(pipelineId); for (IngestStats.ProcessorStat st : stats) { - assertThat(st.getStats().getIngestCurrent(), greaterThanOrEqualTo(0L)); + assertThat(st.getStats().getCurrent(), greaterThanOrEqualTo(0L)); } } } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodeStats.java b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodeStats.java index c183562e2e85a..6b8e06594acb7 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodeStats.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodeStats.java @@ -59,6 +59,7 @@ import org.opensearch.script.ScriptCacheStats; import org.opensearch.script.ScriptStats; import org.opensearch.search.backpressure.stats.SearchBackpressureStats; +import org.opensearch.search.pipeline.SearchPipelineStats; import org.opensearch.tasks.TaskCancellationStats; import org.opensearch.threadpool.ThreadPoolStats; import org.opensearch.transport.TransportStats; @@ -138,6 +139,9 @@ public class NodeStats extends BaseNodeResponse implements ToXContentFragment { @Nullable private TaskCancellationStats taskCancellationStats; + @Nullable + private SearchPipelineStats searchPipelineStats; + public NodeStats(StreamInput in) throws IOException { super(in); timestamp = in.readVLong(); @@ -189,6 +193,11 @@ public NodeStats(StreamInput in) throws IOException { } else { taskCancellationStats = null; } + if (in.getVersion().onOrAfter(Version.V_3_0_0)) { // TODO Update to 2_9_0 when we backport to 2.x + searchPipelineStats = in.readOptionalWriteable(SearchPipelineStats::new); + } else { + searchPipelineStats = null; + } } public NodeStats( @@ -214,7 +223,8 @@ public NodeStats( @Nullable ClusterManagerThrottlingStats clusterManagerThrottlingStats, @Nullable WeightedRoutingStats weightedRoutingStats, @Nullable FileCacheStats fileCacheStats, - @Nullable TaskCancellationStats taskCancellationStats + @Nullable TaskCancellationStats taskCancellationStats, + @Nullable SearchPipelineStats searchPipelineStats ) { super(node); this.timestamp = timestamp; @@ -239,6 +249,7 @@ public NodeStats( this.weightedRoutingStats = weightedRoutingStats; this.fileCacheStats = fileCacheStats; this.taskCancellationStats = taskCancellationStats; + this.searchPipelineStats = searchPipelineStats; } public long getTimestamp() { @@ -371,6 +382,11 @@ public TaskCancellationStats getTaskCancellationStats() { return taskCancellationStats; } + @Nullable + public SearchPipelineStats getSearchPipelineStats() { + return searchPipelineStats; + } + @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); @@ -411,6 +427,9 @@ public void writeTo(StreamOutput out) throws IOException { if (out.getVersion().onOrAfter(Version.V_2_9_0)) { out.writeOptionalWriteable(taskCancellationStats); } + if (out.getVersion().onOrAfter(Version.V_3_0_0)) { // TODO: Update to 2_9_0 once we backport to 2.x + out.writeOptionalWriteable(searchPipelineStats); + } } @Override @@ -498,6 +517,9 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws if (getTaskCancellationStats() != null) { getTaskCancellationStats().toXContent(builder, params); } + if (getSearchPipelineStats() != null) { + getSearchPipelineStats().toXContent(builder, params); + } return builder; } diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodesStatsRequest.java b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodesStatsRequest.java index 68f391b91507c..f37a837c6f0ef 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodesStatsRequest.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/NodesStatsRequest.java @@ -211,7 +211,8 @@ public enum Metric { CLUSTER_MANAGER_THROTTLING("cluster_manager_throttling"), WEIGHTED_ROUTING_STATS("weighted_routing"), FILE_CACHE_STATS("file_cache"), - TASK_CANCELLATION("task_cancellation"); + TASK_CANCELLATION("task_cancellation"), + SEARCH_PIPELINE("search_pipeline"); private String metricName; diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/TransportNodesStatsAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/TransportNodesStatsAction.java index 6aadf546d30f7..660142f05bab2 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/TransportNodesStatsAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/node/stats/TransportNodesStatsAction.java @@ -123,7 +123,8 @@ protected NodeStats nodeOperation(NodeStatsRequest nodeStatsRequest) { NodesStatsRequest.Metric.CLUSTER_MANAGER_THROTTLING.containedIn(metrics), NodesStatsRequest.Metric.WEIGHTED_ROUTING_STATS.containedIn(metrics), NodesStatsRequest.Metric.FILE_CACHE_STATS.containedIn(metrics), - NodesStatsRequest.Metric.TASK_CANCELLATION.containedIn(metrics) + NodesStatsRequest.Metric.TASK_CANCELLATION.containedIn(metrics), + NodesStatsRequest.Metric.SEARCH_PIPELINE.containedIn(metrics) ); } 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 828f3a2e5e842..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,6 +40,7 @@ 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.metrics.OperationStats; import org.opensearch.common.network.NetworkModule; import org.opensearch.common.settings.Settings; import org.opensearch.common.transport.TransportAddress; @@ -800,18 +801,18 @@ static class IngestStats implements ToXContentFragment { pipelineIds.add(processorStats.getKey()); for (org.opensearch.ingest.IngestStats.ProcessorStat stat : processorStats.getValue()) { stats.compute(stat.getType(), (k, v) -> { - org.opensearch.ingest.IngestStats.Stats nodeIngestStats = stat.getStats(); + OperationStats nodeIngestStats = stat.getStats(); if (v == null) { return new long[] { - nodeIngestStats.getIngestCount(), - nodeIngestStats.getIngestFailedCount(), - nodeIngestStats.getIngestCurrent(), - nodeIngestStats.getIngestTimeInMillis() }; + nodeIngestStats.getCount(), + nodeIngestStats.getFailedCount(), + nodeIngestStats.getCurrent(), + nodeIngestStats.getTotalTimeInMillis() }; } else { - v[0] += nodeIngestStats.getIngestCount(); - v[1] += nodeIngestStats.getIngestFailedCount(); - v[2] += nodeIngestStats.getIngestCurrent(); - v[3] += nodeIngestStats.getIngestTimeInMillis(); + v[0] += nodeIngestStats.getCount(); + v[1] += nodeIngestStats.getFailedCount(); + v[2] += nodeIngestStats.getCurrent(); + v[3] += nodeIngestStats.getTotalTimeInMillis(); return v; } }); diff --git a/server/src/main/java/org/opensearch/action/admin/cluster/stats/TransportClusterStatsAction.java b/server/src/main/java/org/opensearch/action/admin/cluster/stats/TransportClusterStatsAction.java index 726f8a0de19ae..aee6dfddd203e 100644 --- a/server/src/main/java/org/opensearch/action/admin/cluster/stats/TransportClusterStatsAction.java +++ b/server/src/main/java/org/opensearch/action/admin/cluster/stats/TransportClusterStatsAction.java @@ -167,6 +167,7 @@ protected ClusterStatsNodeResponse nodeOperation(ClusterStatsNodeRequest nodeReq false, false, false, + false, false ); List shardsStats = new ArrayList<>(); diff --git a/server/src/main/java/org/opensearch/common/metrics/MeanMetric.java b/server/src/main/java/org/opensearch/common/metrics/MeanMetric.java index 79c04d431e97b..33f12c8cb42d3 100644 --- a/server/src/main/java/org/opensearch/common/metrics/MeanMetric.java +++ b/server/src/main/java/org/opensearch/common/metrics/MeanMetric.java @@ -49,6 +49,11 @@ public void inc(long n) { sum.add(n); } + public void add(MeanMetric other) { + counter.add(other.counter.sum()); + sum.add(other.sum.sum()); + } + public void dec(long n) { counter.decrement(); sum.add(-n); diff --git a/server/src/main/java/org/opensearch/common/metrics/OperationMetrics.java b/server/src/main/java/org/opensearch/common/metrics/OperationMetrics.java new file mode 100644 index 0000000000000..97fbbc2ce5cde --- /dev/null +++ b/server/src/main/java/org/opensearch/common/metrics/OperationMetrics.java @@ -0,0 +1,68 @@ +/* + * 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.common.metrics; + +import java.util.concurrent.atomic.AtomicLong; + +/** + * Mutable tracker of a repeated operation. + * + * @opensearch.internal + */ +public class OperationMetrics { + /** + * The mean time it takes to complete the measured item. + */ + private final MeanMetric time = new MeanMetric(); + /** + * The current count of things being measured. + * Useful when aggregating multiple metrics to see how many things are in flight. + */ + private final AtomicLong current = new AtomicLong(); + /** + * The non-decreasing count of failures + */ + private final CounterMetric failed = new CounterMetric(); + + /** + * Invoked before the given operation begins. + */ + public void before() { + current.incrementAndGet(); + } + + /** + * Invoked upon completion (success or failure) of the given operation + * @param currentTime elapsed time of the operation + */ + public void after(long currentTime) { + current.decrementAndGet(); + time.inc(currentTime); + } + + /** + * Invoked upon failure of the operation. + */ + public void failed() { + failed.inc(); + } + + public void add(OperationMetrics other) { + // Don't try copying over current, since in-flight requests will be linked to the existing metrics instance. + failed.inc(other.failed.count()); + time.add(other.time); + } + + /** + * @return an immutable snapshot of the current metric values. + */ + public OperationStats createStats() { + return new OperationStats(time.count(), time.sum(), current.get(), failed.count()); + } +} diff --git a/server/src/main/java/org/opensearch/common/metrics/OperationStats.java b/server/src/main/java/org/opensearch/common/metrics/OperationStats.java new file mode 100644 index 0000000000000..a820f848393bb --- /dev/null +++ b/server/src/main/java/org/opensearch/common/metrics/OperationStats.java @@ -0,0 +1,107 @@ +/* + * 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.common.metrics; + +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.xcontent.ToXContentFragment; +import org.opensearch.core.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.Objects; +import java.util.concurrent.TimeUnit; + +/** + * An immutable representation of a {@link OperationMetrics} + */ +public class OperationStats implements Writeable, ToXContentFragment { + private final long count; + private final long totalTimeInMillis; + private final long current; + private final long failedCount; + + public OperationStats(long count, long totalTimeInMillis, long current, long failedCount) { + this.count = count; + this.totalTimeInMillis = totalTimeInMillis; + this.current = current; + this.failedCount = failedCount; + } + + /** + * Read from a stream. + */ + public OperationStats(StreamInput in) throws IOException { + count = in.readVLong(); + totalTimeInMillis = in.readVLong(); + current = in.readVLong(); + failedCount = in.readVLong(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVLong(count); + out.writeVLong(totalTimeInMillis); + out.writeVLong(current); + out.writeVLong(failedCount); + } + + /** + * @return The total number of executed operations. + */ + public long getCount() { + return count; + } + + /** + * @return The total time spent of in millis. + */ + public long getTotalTimeInMillis() { + return totalTimeInMillis; + } + + /** + * @return The total number of operations currently executing. + */ + public long getCurrent() { + return current; + } + + /** + * @return The total number of operations that have failed. + */ + public long getFailedCount() { + return failedCount; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return builder.field("count", count) + .humanReadableField("time_in_millis", "time", new TimeValue(totalTimeInMillis, TimeUnit.MILLISECONDS)) + .field("current", current) + .field("failed", failedCount); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + OperationStats that = (OperationStats) o; + return Objects.equals(count, that.count) + && Objects.equals(totalTimeInMillis, that.totalTimeInMillis) + && Objects.equals(failedCount, that.failedCount) + && Objects.equals(current, that.current); + } + + @Override + public int hashCode() { + return Objects.hash(count, totalTimeInMillis, failedCount, current); + } +} diff --git a/server/src/main/java/org/opensearch/ingest/CompoundProcessor.java b/server/src/main/java/org/opensearch/ingest/CompoundProcessor.java index 8cdbc487dc137..a5f4870029e87 100644 --- a/server/src/main/java/org/opensearch/ingest/CompoundProcessor.java +++ b/server/src/main/java/org/opensearch/ingest/CompoundProcessor.java @@ -34,6 +34,7 @@ import org.opensearch.OpenSearchException; import org.opensearch.common.collect.Tuple; +import org.opensearch.common.metrics.OperationMetrics; import java.util.ArrayList; import java.util.Arrays; @@ -60,7 +61,7 @@ public class CompoundProcessor implements Processor { private final boolean ignoreFailure; private final List processors; private final List onFailureProcessors; - private final List> processorsWithMetrics; + private final List> processorsWithMetrics; private final LongSupplier relativeTimeProvider; CompoundProcessor(LongSupplier relativeTimeProvider, Processor... processor) { @@ -87,10 +88,10 @@ public CompoundProcessor(boolean ignoreFailure, List processors, List this.onFailureProcessors = onFailureProcessors; this.relativeTimeProvider = relativeTimeProvider; this.processorsWithMetrics = new ArrayList<>(processors.size()); - processors.forEach(p -> processorsWithMetrics.add(new Tuple<>(p, new IngestMetric()))); + processors.forEach(p -> processorsWithMetrics.add(new Tuple<>(p, new OperationMetrics()))); } - List> getProcessorsWithMetrics() { + List> getProcessorsWithMetrics() { return processorsWithMetrics; } @@ -155,17 +156,17 @@ void innerExecute(int currentProcessor, IngestDocument ingestDocument, BiConsume return; } - Tuple processorWithMetric = processorsWithMetrics.get(currentProcessor); + Tuple processorWithMetric = processorsWithMetrics.get(currentProcessor); final Processor processor = processorWithMetric.v1(); - final IngestMetric metric = processorWithMetric.v2(); + final OperationMetrics metric = processorWithMetric.v2(); final long startTimeInNanos = relativeTimeProvider.getAsLong(); - metric.preIngest(); + metric.before(); processor.execute(ingestDocument, (result, e) -> { long ingestTimeInMillis = TimeUnit.NANOSECONDS.toMillis(relativeTimeProvider.getAsLong() - startTimeInNanos); - metric.postIngest(ingestTimeInMillis); + metric.after(ingestTimeInMillis); if (e != null) { - metric.ingestFailed(); + metric.failed(); if (ignoreFailure) { innerExecute(currentProcessor + 1, ingestDocument, handler); } else { diff --git a/server/src/main/java/org/opensearch/ingest/ConditionalProcessor.java b/server/src/main/java/org/opensearch/ingest/ConditionalProcessor.java index 591a71fd72b8f..8bf489805f7ca 100644 --- a/server/src/main/java/org/opensearch/ingest/ConditionalProcessor.java +++ b/server/src/main/java/org/opensearch/ingest/ConditionalProcessor.java @@ -32,6 +32,7 @@ package org.opensearch.ingest; +import org.opensearch.common.metrics.OperationMetrics; import org.opensearch.script.IngestConditionalScript; import org.opensearch.script.Script; import org.opensearch.script.ScriptException; @@ -66,7 +67,7 @@ public class ConditionalProcessor extends AbstractProcessor implements WrappingP private final Script condition; private final ScriptService scriptService; private final Processor processor; - private final IngestMetric metric; + private final OperationMetrics metric; private final LongSupplier relativeTimeProvider; private final IngestConditionalScript precompiledConditionScript; @@ -86,7 +87,7 @@ public class ConditionalProcessor extends AbstractProcessor implements WrappingP this.condition = script; this.scriptService = scriptService; this.processor = processor; - this.metric = new IngestMetric(); + this.metric = new OperationMetrics(); this.relativeTimeProvider = relativeTimeProvider; try { @@ -114,12 +115,12 @@ public void execute(IngestDocument ingestDocument, BiConsumer { long ingestTimeInMillis = TimeUnit.NANOSECONDS.toMillis(relativeTimeProvider.getAsLong() - startTimeInNanos); - metric.postIngest(ingestTimeInMillis); + metric.after(ingestTimeInMillis); if (e != null) { - metric.ingestFailed(); + metric.failed(); handler.accept(null, e); } else { handler.accept(result, null); @@ -148,7 +149,7 @@ public Processor getInnerProcessor() { return processor; } - IngestMetric getMetric() { + OperationMetrics getMetric() { return metric; } diff --git a/server/src/main/java/org/opensearch/ingest/IngestMetric.java b/server/src/main/java/org/opensearch/ingest/IngestMetric.java deleted file mode 100644 index 2d4a1dc9cfdee..0000000000000 --- a/server/src/main/java/org/opensearch/ingest/IngestMetric.java +++ /dev/null @@ -1,112 +0,0 @@ -/* - * 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.ingest; - -import org.opensearch.common.metrics.CounterMetric; -import org.opensearch.common.metrics.MeanMetric; - -import java.util.concurrent.atomic.AtomicLong; - -/** - *

Metrics to measure ingest actions. - *

This counts measure documents and timings for a given scope. - * The scope is determined by the calling code. For example you can use this class to count all documents across all pipeline, - * or you can use this class to count documents for a given pipeline or a specific processor. - * This class does not make assumptions about it's given scope. - * - * @opensearch.internal - */ -class IngestMetric { - - /** - * The time it takes to complete the measured item. - */ - private final MeanMetric ingestTime = new MeanMetric(); - /** - * The current count of things being measure. Should most likely ever be 0 or 1. - * Useful when aggregating multiple metrics to see how many things are in flight. - */ - private final AtomicLong ingestCurrent = new AtomicLong(); - /** - * The ever increasing count of things being measured - */ - private final CounterMetric ingestCount = new CounterMetric(); - /** - * The only increasing count of failures - */ - private final CounterMetric ingestFailed = new CounterMetric(); - - /** - * Call this prior to the ingest action. - */ - void preIngest() { - ingestCurrent.incrementAndGet(); - } - - /** - * Call this after the performing the ingest action, even if the action failed. - * @param ingestTimeInMillis The time it took to perform the action. - */ - void postIngest(long ingestTimeInMillis) { - ingestCurrent.decrementAndGet(); - ingestTime.inc(ingestTimeInMillis); - ingestCount.inc(); - } - - /** - * Call this if the ingest action failed. - */ - void ingestFailed() { - ingestFailed.inc(); - } - - /** - *

Add two sets of metrics together. - *

Note - this method does not add the current count values. - * The current count value is ephemeral and requires a increase/decrease operation pairs to keep the value correct. - * - * @param metrics The metric to add. - */ - void add(IngestMetric metrics) { - ingestCount.inc(metrics.ingestCount.count()); - ingestTime.inc(metrics.ingestTime.sum()); - ingestFailed.inc(metrics.ingestFailed.count()); - } - - /** - * Creates a serializable representation for these metrics. - */ - IngestStats.Stats createStats() { - return new IngestStats.Stats(ingestCount.count(), ingestTime.sum(), ingestCurrent.get(), ingestFailed.count()); - } -} diff --git a/server/src/main/java/org/opensearch/ingest/IngestService.java b/server/src/main/java/org/opensearch/ingest/IngestService.java index b9785d9ec036f..0984046ca3077 100644 --- a/server/src/main/java/org/opensearch/ingest/IngestService.java +++ b/server/src/main/java/org/opensearch/ingest/IngestService.java @@ -60,6 +60,7 @@ import org.opensearch.cluster.service.ClusterManagerTaskThrottler; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.collect.Tuple; +import org.opensearch.common.metrics.OperationMetrics; import org.opensearch.common.regex.Regex; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; @@ -114,7 +115,7 @@ public class IngestService implements ClusterStateApplier, ReportingService pipelines = Collections.emptyMap(); private final ThreadPool threadPool; - private final IngestMetric totalMetrics = new IngestMetric(); + private final OperationMetrics totalMetrics = new OperationMetrics(); private final List> ingestClusterStateListeners = new CopyOnWriteArrayList<>(); private final ClusterManagerTaskThrottler.ThrottlingKey putPipelineTaskKey; private final ClusterManagerTaskThrottler.ThrottlingKey deletePipelineTaskKey; @@ -440,17 +441,17 @@ Map pipelines() { * Recursive method to obtain all of the non-failure processors for given compoundProcessor. Since conditionals are implemented as * wrappers to the actual processor, always prefer the actual processor's metric over the conditional processor's metric. * @param compoundProcessor The compound processor to start walking the non-failure processors - * @param processorMetrics The list of {@link Processor} {@link IngestMetric} tuples. + * @param processorMetrics The list of {@link Processor} {@link OperationMetrics} tuples. * @return the processorMetrics for all non-failure processor that belong to the original compoundProcessor */ - private static List> getProcessorMetrics( + private static List> getProcessorMetrics( CompoundProcessor compoundProcessor, - List> processorMetrics + List> processorMetrics ) { // only surface the top level non-failure processors, on-failure processor times will be included in the top level non-failure - for (Tuple processorWithMetric : compoundProcessor.getProcessorsWithMetrics()) { + for (Tuple processorWithMetric : compoundProcessor.getProcessorsWithMetrics()) { Processor processor = processorWithMetric.v1(); - IngestMetric metric = processorWithMetric.v2(); + OperationMetrics metric = processorWithMetric.v2(); if (processor instanceof CompoundProcessor) { getProcessorMetrics((CompoundProcessor) processor, processorMetrics); } else { @@ -614,7 +615,7 @@ private void executePipelines( if (Objects.equals(originalIndex, newIndex) == false) { if (hasFinalPipeline && it.hasNext() == false) { - totalMetrics.ingestFailed(); + totalMetrics.failed(); onFailure.accept( slot, new IllegalStateException("final pipeline [" + pipelineId + "] can't change the target index") @@ -680,11 +681,11 @@ public IngestStats stats() { Pipeline pipeline = holder.pipeline; CompoundProcessor rootProcessor = pipeline.getCompoundProcessor(); statsBuilder.addPipelineMetrics(id, pipeline.getMetrics()); - List> processorMetrics = new ArrayList<>(); + List> processorMetrics = new ArrayList<>(); getProcessorMetrics(rootProcessor, processorMetrics); processorMetrics.forEach(t -> { Processor processor = t.v1(); - IngestMetric processorMetric = t.v2(); + OperationMetrics processorMetric = t.v2(); statsBuilder.addProcessorMetrics(id, getProcessorName(processor), processor.getType(), processorMetric); }); }); @@ -739,7 +740,7 @@ private void innerExecute( long startTimeInNanos = System.nanoTime(); // the pipeline specific stat holder may not exist and that is fine: // (e.g. the pipeline may have been removed while we're ingesting a document - totalMetrics.preIngest(); + totalMetrics.before(); String index = indexRequest.index(); String id = indexRequest.id(); String routing = indexRequest.routing(); @@ -749,9 +750,9 @@ private void innerExecute( IngestDocument ingestDocument = new IngestDocument(index, id, routing, version, versionType, sourceAsMap); ingestDocument.executePipeline(pipeline, (result, e) -> { long ingestTimeInMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeInNanos); - totalMetrics.postIngest(ingestTimeInMillis); + totalMetrics.after(ingestTimeInMillis); if (e != null) { - totalMetrics.ingestFailed(); + totalMetrics.failed(); handler.accept(e); } else if (result == null) { itemDroppedHandler.accept(slot); @@ -835,22 +836,22 @@ void innerUpdatePipelines(IngestMetadata newIngestMetadata) { } Pipeline oldPipeline = previous.pipeline; newPipeline.getMetrics().add(oldPipeline.getMetrics()); - List> oldPerProcessMetrics = new ArrayList<>(); - List> newPerProcessMetrics = new ArrayList<>(); + List> oldPerProcessMetrics = new ArrayList<>(); + List> newPerProcessMetrics = new ArrayList<>(); getProcessorMetrics(oldPipeline.getCompoundProcessor(), oldPerProcessMetrics); getProcessorMetrics(newPipeline.getCompoundProcessor(), newPerProcessMetrics); // Best attempt to populate new processor metrics using a parallel array of the old metrics. This is not ideal since // the per processor metrics may get reset when the arrays don't match. However, to get to an ideal model, unique and // consistent id's per processor and/or semantic equals for each processor will be needed. if (newPerProcessMetrics.size() == oldPerProcessMetrics.size()) { - Iterator> oldMetricsIterator = oldPerProcessMetrics.iterator(); - for (Tuple compositeMetric : newPerProcessMetrics) { + Iterator> oldMetricsIterator = oldPerProcessMetrics.iterator(); + for (Tuple compositeMetric : newPerProcessMetrics) { String type = compositeMetric.v1().getType(); - IngestMetric metric = compositeMetric.v2(); + OperationMetrics metric = compositeMetric.v2(); if (oldMetricsIterator.hasNext()) { - Tuple oldCompositeMetric = oldMetricsIterator.next(); + Tuple oldCompositeMetric = oldMetricsIterator.next(); String oldType = oldCompositeMetric.v1().getType(); - IngestMetric oldMetric = oldCompositeMetric.v2(); + OperationMetrics oldMetric = oldCompositeMetric.v2(); if (type.equals(oldType)) { metric.add(oldMetric); } diff --git a/server/src/main/java/org/opensearch/ingest/IngestStats.java b/server/src/main/java/org/opensearch/ingest/IngestStats.java index 0f6209d1c005e..ac06d779bdf2d 100644 --- a/server/src/main/java/org/opensearch/ingest/IngestStats.java +++ b/server/src/main/java/org/opensearch/ingest/IngestStats.java @@ -35,7 +35,8 @@ 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.common.metrics.OperationMetrics; +import org.opensearch.common.metrics.OperationStats; import org.opensearch.core.xcontent.ToXContentFragment; import org.opensearch.core.xcontent.XContentBuilder; @@ -46,15 +47,14 @@ import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.concurrent.TimeUnit; /** - * Stats for an ingest processor pipeline + * OperationStats for an ingest processor pipeline * * @opensearch.internal */ public class IngestStats implements Writeable, ToXContentFragment { - private final Stats totalStats; + private final OperationStats totalStats; private final List pipelineStats; private final Map> processorStats; @@ -64,7 +64,7 @@ public class IngestStats implements Writeable, ToXContentFragment { * @param pipelineStats - The stats for a given ingest pipeline. * @param processorStats - The per-processor stats for a given pipeline. A map keyed by the pipeline identifier. */ - public IngestStats(Stats totalStats, List pipelineStats, Map> processorStats) { + public IngestStats(OperationStats totalStats, List pipelineStats, Map> processorStats) { this.totalStats = totalStats; this.pipelineStats = pipelineStats; this.processorStats = processorStats; @@ -74,13 +74,13 @@ public IngestStats(Stats totalStats, List pipelineStats, Map(size); this.processorStats = new HashMap<>(size); for (int i = 0; i < size; i++) { String pipelineId = in.readString(); - Stats pipelineStat = new Stats(in); + OperationStats pipelineStat = new OperationStats(in); this.pipelineStats.add(new PipelineStat(pipelineId, pipelineStat)); int processorsSize = in.readVInt(); List processorStatsPerPipeline = new ArrayList<>(processorsSize); @@ -88,7 +88,7 @@ public IngestStats(StreamInput in) throws IOException { String processorName = in.readString(); String processorType = "_NOT_AVAILABLE"; processorType = in.readString(); - Stats processorStat = new Stats(in); + OperationStats processorStat = new OperationStats(in); processorStatsPerPipeline.add(new ProcessorStat(processorName, processorType, processorStat)); } this.processorStats.put(pipelineId, processorStatsPerPipeline); @@ -148,7 +148,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws return builder; } - public Stats getTotalStats() { + public OperationStats getTotalStats() { return totalStats; } @@ -176,115 +176,24 @@ public int hashCode() { } /** - * The ingest statistics. - * - * @opensearch.internal - */ - public static class Stats implements Writeable, ToXContentFragment { - - private final long ingestCount; - private final long ingestTimeInMillis; - private final long ingestCurrent; - private final long ingestFailedCount; - - public Stats(long ingestCount, long ingestTimeInMillis, long ingestCurrent, long ingestFailedCount) { - this.ingestCount = ingestCount; - this.ingestTimeInMillis = ingestTimeInMillis; - this.ingestCurrent = ingestCurrent; - this.ingestFailedCount = ingestFailedCount; - } - - /** - * Read from a stream. - */ - public Stats(StreamInput in) throws IOException { - ingestCount = in.readVLong(); - ingestTimeInMillis = in.readVLong(); - ingestCurrent = in.readVLong(); - ingestFailedCount = in.readVLong(); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeVLong(ingestCount); - out.writeVLong(ingestTimeInMillis); - out.writeVLong(ingestCurrent); - out.writeVLong(ingestFailedCount); - } - - /** - * @return The total number of executed ingest preprocessing operations. - */ - public long getIngestCount() { - return ingestCount; - } - - /** - * @return The total time spent of ingest preprocessing in millis. - */ - public long getIngestTimeInMillis() { - return ingestTimeInMillis; - } - - /** - * @return The total number of ingest preprocessing operations currently executing. - */ - public long getIngestCurrent() { - return ingestCurrent; - } - - /** - * @return The total number of ingest preprocessing operations that have failed. - */ - public long getIngestFailedCount() { - return ingestFailedCount; - } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - builder.field("count", ingestCount); - builder.humanReadableField("time_in_millis", "time", new TimeValue(ingestTimeInMillis, TimeUnit.MILLISECONDS)); - builder.field("current", ingestCurrent); - builder.field("failed", ingestFailedCount); - return builder; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - IngestStats.Stats that = (IngestStats.Stats) o; - return Objects.equals(ingestCount, that.ingestCount) - && Objects.equals(ingestTimeInMillis, that.ingestTimeInMillis) - && Objects.equals(ingestFailedCount, that.ingestFailedCount) - && Objects.equals(ingestCurrent, that.ingestCurrent); - } - - @Override - public int hashCode() { - return Objects.hash(ingestCount, ingestTimeInMillis, ingestFailedCount, ingestCurrent); - } - } - - /** - * Easy conversion from scoped {@link IngestMetric} objects to a serializable Stats objects + * Easy conversion from scoped {@link OperationMetrics} objects to a serializable OperationStats objects */ static class Builder { - private Stats totalStats; + private OperationStats totalStats; private List pipelineStats = new ArrayList<>(); private Map> processorStats = new HashMap<>(); - Builder addTotalMetrics(IngestMetric totalMetric) { + Builder addTotalMetrics(OperationMetrics totalMetric) { this.totalStats = totalMetric.createStats(); return this; } - Builder addPipelineMetrics(String pipelineId, IngestMetric pipelineMetric) { + Builder addPipelineMetrics(String pipelineId, OperationMetrics pipelineMetric) { this.pipelineStats.add(new PipelineStat(pipelineId, pipelineMetric.createStats())); return this; } - Builder addProcessorMetrics(String pipelineId, String processorName, String processorType, IngestMetric metric) { + Builder addProcessorMetrics(String pipelineId, String processorName, String processorType, OperationMetrics metric) { this.processorStats.computeIfAbsent(pipelineId, k -> new ArrayList<>()) .add(new ProcessorStat(processorName, processorType, metric.createStats())); return this; @@ -300,9 +209,9 @@ IngestStats build() { */ public static class PipelineStat { private final String pipelineId; - private final Stats stats; + private final OperationStats stats; - public PipelineStat(String pipelineId, Stats stats) { + public PipelineStat(String pipelineId, OperationStats stats) { this.pipelineId = pipelineId; this.stats = stats; } @@ -311,7 +220,7 @@ public String getPipelineId() { return pipelineId; } - public Stats getStats() { + public OperationStats getStats() { return stats; } @@ -335,9 +244,9 @@ public int hashCode() { public static class ProcessorStat { private final String name; private final String type; - private final Stats stats; + private final OperationStats stats; - public ProcessorStat(String name, String type, Stats stats) { + public ProcessorStat(String name, String type, OperationStats stats) { this.name = name; this.type = type; this.stats = stats; @@ -351,7 +260,7 @@ public String getType() { return type; } - public Stats getStats() { + public OperationStats getStats() { return stats; } diff --git a/server/src/main/java/org/opensearch/ingest/Pipeline.java b/server/src/main/java/org/opensearch/ingest/Pipeline.java index 9b3725fd65d9d..766fb9cd66777 100644 --- a/server/src/main/java/org/opensearch/ingest/Pipeline.java +++ b/server/src/main/java/org/opensearch/ingest/Pipeline.java @@ -43,6 +43,7 @@ import java.util.function.BiConsumer; import java.util.function.LongSupplier; +import org.opensearch.common.metrics.OperationMetrics; import org.opensearch.script.ScriptService; /** @@ -63,7 +64,7 @@ public final class Pipeline { @Nullable private final Integer version; private final CompoundProcessor compoundProcessor; - private final IngestMetric metrics; + private final OperationMetrics metrics; private final LongSupplier relativeTimeProvider; public Pipeline(String id, @Nullable String description, @Nullable Integer version, CompoundProcessor compoundProcessor) { @@ -82,7 +83,7 @@ public Pipeline(String id, @Nullable String description, @Nullable Integer versi this.description = description; this.compoundProcessor = compoundProcessor; this.version = version; - this.metrics = new IngestMetric(); + this.metrics = new OperationMetrics(); this.relativeTimeProvider = relativeTimeProvider; } @@ -129,12 +130,12 @@ public static Pipeline create( */ public void execute(IngestDocument ingestDocument, BiConsumer handler) { final long startTimeInNanos = relativeTimeProvider.getAsLong(); - metrics.preIngest(); + metrics.before(); compoundProcessor.execute(ingestDocument, (result, e) -> { long ingestTimeInMillis = TimeUnit.NANOSECONDS.toMillis(relativeTimeProvider.getAsLong() - startTimeInNanos); - metrics.postIngest(ingestTimeInMillis); + metrics.after(ingestTimeInMillis); if (e != null) { - metrics.ingestFailed(); + metrics.failed(); } handler.accept(result, e); }); @@ -198,7 +199,7 @@ public List flattenAllProcessors() { /** * The metrics associated with this pipeline. */ - public IngestMetric getMetrics() { + public OperationMetrics getMetrics() { return metrics; } } diff --git a/server/src/main/java/org/opensearch/node/NodeService.java b/server/src/main/java/org/opensearch/node/NodeService.java index 9382746081c18..6f4fe1e083ad7 100644 --- a/server/src/main/java/org/opensearch/node/NodeService.java +++ b/server/src/main/java/org/opensearch/node/NodeService.java @@ -216,7 +216,8 @@ public NodeStats stats( boolean clusterManagerThrottling, boolean weightedRoutingStats, boolean fileCacheStats, - boolean taskCancellation + boolean taskCancellation, + boolean searchPipelineStats ) { // for indices stats we want to include previous allocated shards stats as well (it will // only be applied to the sensible ones to use, like refresh/merge/flush/indexing stats) @@ -243,7 +244,8 @@ public NodeStats stats( clusterManagerThrottling ? this.clusterService.getClusterManagerService().getThrottlingStats() : null, weightedRoutingStats ? WeightedRoutingStats.getInstance() : null, fileCacheStats && fileCache != null ? fileCache.fileCacheStats() : null, - taskCancellation ? this.taskCancellationMonitoringService.stats() : null + taskCancellation ? this.taskCancellationMonitoringService.stats() : null, + searchPipelineStats ? this.searchPipelineService.stats() : null ); } diff --git a/server/src/main/java/org/opensearch/search/pipeline/Pipeline.java b/server/src/main/java/org/opensearch/search/pipeline/Pipeline.java index c9a5f865d507e..6f44daf48ed21 100644 --- a/server/src/main/java/org/opensearch/search/pipeline/Pipeline.java +++ b/server/src/main/java/org/opensearch/search/pipeline/Pipeline.java @@ -8,7 +8,6 @@ package org.opensearch.search.pipeline; -import org.opensearch.OpenSearchParseException; import org.opensearch.action.search.SearchRequest; import org.opensearch.action.search.SearchResponse; import org.opensearch.common.Nullable; @@ -16,17 +15,11 @@ import org.opensearch.common.io.stream.NamedWriteableAwareStreamInput; import org.opensearch.common.io.stream.NamedWriteableRegistry; import org.opensearch.common.io.stream.StreamInput; -import org.opensearch.ingest.ConfigurationUtils; -import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.Map; - -import static org.opensearch.ingest.ConfigurationUtils.TAG_KEY; -import static org.opensearch.ingest.Pipeline.DESCRIPTION_KEY; -import static org.opensearch.ingest.Pipeline.VERSION_KEY; +import java.util.concurrent.TimeUnit; +import java.util.function.LongSupplier; /** * Concrete representation of a search pipeline, holding multiple processors. @@ -45,73 +38,24 @@ class Pipeline { private final List searchResponseProcessors; private final NamedWriteableRegistry namedWriteableRegistry; + private final LongSupplier relativeTimeSupplier; - private Pipeline( + Pipeline( String id, @Nullable String description, @Nullable Integer version, List requestProcessors, List responseProcessors, - NamedWriteableRegistry namedWriteableRegistry + NamedWriteableRegistry namedWriteableRegistry, + LongSupplier relativeTimeSupplier ) { this.id = id; this.description = description; this.version = version; - this.searchRequestProcessors = requestProcessors; - this.searchResponseProcessors = responseProcessors; + this.searchRequestProcessors = Collections.unmodifiableList(requestProcessors); + this.searchResponseProcessors = Collections.unmodifiableList(responseProcessors); this.namedWriteableRegistry = namedWriteableRegistry; - } - - static Pipeline create( - String id, - Map config, - Map> requestProcessorFactories, - Map> responseProcessorFactories, - NamedWriteableRegistry namedWriteableRegistry - ) throws Exception { - String description = ConfigurationUtils.readOptionalStringProperty(null, null, config, DESCRIPTION_KEY); - Integer version = ConfigurationUtils.readIntProperty(null, null, config, VERSION_KEY, null); - List> requestProcessorConfigs = ConfigurationUtils.readOptionalList(null, null, config, REQUEST_PROCESSORS_KEY); - List requestProcessors = readProcessors(requestProcessorFactories, requestProcessorConfigs); - List> responseProcessorConfigs = ConfigurationUtils.readOptionalList( - null, - null, - config, - RESPONSE_PROCESSORS_KEY - ); - List responseProcessors = readProcessors(responseProcessorFactories, responseProcessorConfigs); - if (config.isEmpty() == false) { - throw new OpenSearchParseException( - "pipeline [" - + id - + "] doesn't support one or more provided configuration parameters " - + Arrays.toString(config.keySet().toArray()) - ); - } - return new Pipeline(id, description, version, requestProcessors, responseProcessors, namedWriteableRegistry); - } - - private static List readProcessors( - Map> processorFactories, - List> requestProcessorConfigs - ) throws Exception { - List processors = new ArrayList<>(); - if (requestProcessorConfigs == null) { - return processors; - } - for (Map processorConfigWithKey : requestProcessorConfigs) { - for (Map.Entry entry : processorConfigWithKey.entrySet()) { - String type = entry.getKey(); - if (!processorFactories.containsKey(type)) { - throw new IllegalArgumentException("Invalid processor type " + type); - } - Map config = (Map) entry.getValue(); - String tag = ConfigurationUtils.readOptionalStringProperty(null, null, config, TAG_KEY); - String description = ConfigurationUtils.readOptionalStringProperty(null, tag, config, DESCRIPTION_KEY); - processors.add(processorFactories.get(type).create(processorFactories, tag, description, config)); - } - } - return Collections.unmodifiableList(processors); + this.relativeTimeSupplier = relativeTimeSupplier; } String getId() { @@ -134,32 +78,94 @@ List getSearchResponseProcessors() { return searchResponseProcessors; } - SearchRequest transformRequest(SearchRequest request) throws Exception { + protected void beforeTransformRequest() {} + + protected void afterTransformRequest(long timeInNanos) {} + + protected void onTransformRequestFailure() {} + + protected void beforeRequestProcessor(Processor processor) {} + + protected void afterRequestProcessor(Processor processor, long timeInNanos) {} + + protected void onRequestProcessorFailed(Processor processor) {} + + protected void beforeTransformResponse() {} + + protected void afterTransformResponse(long timeInNanos) {} + + protected void onTransformResponseFailure() {} + + protected void beforeResponseProcessor(Processor processor) {} + + protected void afterResponseProcessor(Processor processor, long timeInNanos) {} + + protected void onResponseProcessorFailed(Processor processor) {} + + SearchRequest transformRequest(SearchRequest request) throws SearchPipelineProcessingException { if (searchRequestProcessors.isEmpty() == false) { - try (BytesStreamOutput bytesStreamOutput = new BytesStreamOutput()) { - request.writeTo(bytesStreamOutput); - try (StreamInput in = bytesStreamOutput.bytes().streamInput()) { - try (StreamInput input = new NamedWriteableAwareStreamInput(in, namedWriteableRegistry)) { - request = new SearchRequest(input); + long pipelineStart = relativeTimeSupplier.getAsLong(); + beforeTransformRequest(); + try { + try (BytesStreamOutput bytesStreamOutput = new BytesStreamOutput()) { + request.writeTo(bytesStreamOutput); + try (StreamInput in = bytesStreamOutput.bytes().streamInput()) { + try (StreamInput input = new NamedWriteableAwareStreamInput(in, namedWriteableRegistry)) { + request = new SearchRequest(input); + } } } - } - for (SearchRequestProcessor searchRequestProcessor : searchRequestProcessors) { - request = searchRequestProcessor.processRequest(request); + for (SearchRequestProcessor processor : searchRequestProcessors) { + beforeRequestProcessor(processor); + long start = relativeTimeSupplier.getAsLong(); + try { + request = processor.processRequest(request); + } catch (Exception e) { + onRequestProcessorFailed(processor); + throw e; + } finally { + long took = TimeUnit.NANOSECONDS.toMillis(relativeTimeSupplier.getAsLong() - start); + afterRequestProcessor(processor, took); + } + } + } catch (Exception e) { + onTransformRequestFailure(); + throw new SearchPipelineProcessingException(e); + } finally { + long took = TimeUnit.NANOSECONDS.toMillis(relativeTimeSupplier.getAsLong() - pipelineStart); + afterTransformRequest(took); } } return request; } SearchResponse transformResponse(SearchRequest request, SearchResponse response) throws SearchPipelineProcessingException { - try { - for (SearchResponseProcessor responseProcessor : searchResponseProcessors) { - response = responseProcessor.processResponse(request, response); + if (searchResponseProcessors.isEmpty() == false) { + long pipelineStart = relativeTimeSupplier.getAsLong(); + beforeTransformResponse(); + try { + for (SearchResponseProcessor processor : searchResponseProcessors) { + beforeResponseProcessor(processor); + long start = relativeTimeSupplier.getAsLong(); + try { + response = processor.processResponse(request, response); + } catch (Exception e) { + onResponseProcessorFailed(processor); + throw e; + } finally { + long took = TimeUnit.NANOSECONDS.toMillis(relativeTimeSupplier.getAsLong() - start); + afterResponseProcessor(processor, took); + } + } + } catch (Exception e) { + onTransformResponseFailure(); + throw new SearchPipelineProcessingException(e); + } finally { + long took = TimeUnit.NANOSECONDS.toMillis(relativeTimeSupplier.getAsLong() - pipelineStart); + afterTransformResponse(took); } - return response; - } catch (Exception e) { - throw new SearchPipelineProcessingException(e); } + return response; } static final Pipeline NO_OP_PIPELINE = new Pipeline( @@ -168,6 +174,8 @@ SearchResponse transformResponse(SearchRequest request, SearchResponse response) 0, Collections.emptyList(), Collections.emptyList(), - null + null, + () -> 0L ); + } diff --git a/server/src/main/java/org/opensearch/search/pipeline/PipelineWithMetrics.java b/server/src/main/java/org/opensearch/search/pipeline/PipelineWithMetrics.java new file mode 100644 index 0000000000000..662473f190006 --- /dev/null +++ b/server/src/main/java/org/opensearch/search/pipeline/PipelineWithMetrics.java @@ -0,0 +1,227 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.search.pipeline; + +import org.opensearch.OpenSearchParseException; +import org.opensearch.common.io.stream.NamedWriteableRegistry; +import org.opensearch.common.metrics.OperationMetrics; +import org.opensearch.ingest.ConfigurationUtils; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.LongSupplier; + +import static org.opensearch.ingest.ConfigurationUtils.TAG_KEY; +import static org.opensearch.ingest.Pipeline.DESCRIPTION_KEY; +import static org.opensearch.ingest.Pipeline.VERSION_KEY; + +/** + * Specialization of {@link Pipeline} that adds metrics to track executions of the pipeline and individual processors. + */ +class PipelineWithMetrics extends Pipeline { + + private final OperationMetrics totalRequestMetrics; + private final OperationMetrics totalResponseMetrics; + private final OperationMetrics pipelineRequestMetrics = new OperationMetrics(); + private final OperationMetrics pipelineResponseMetrics = new OperationMetrics(); + private final Map requestProcessorMetrics = new HashMap<>(); + private final Map responseProcessorMetrics = new HashMap<>(); + + PipelineWithMetrics( + String id, + String description, + Integer version, + List requestProcessors, + List responseProcessors, + NamedWriteableRegistry namedWriteableRegistry, + OperationMetrics totalRequestMetrics, + OperationMetrics totalResponseMetrics, + LongSupplier relativeTimeSupplier + ) { + super(id, description, version, requestProcessors, responseProcessors, namedWriteableRegistry, relativeTimeSupplier); + this.totalRequestMetrics = totalRequestMetrics; + this.totalResponseMetrics = totalResponseMetrics; + for (Processor requestProcessor : getSearchRequestProcessors()) { + requestProcessorMetrics.putIfAbsent(getProcessorKey(requestProcessor), new OperationMetrics()); + } + for (Processor responseProcessor : getSearchResponseProcessors()) { + responseProcessorMetrics.putIfAbsent(getProcessorKey(responseProcessor), new OperationMetrics()); + } + } + + static PipelineWithMetrics create( + String id, + Map config, + Map> requestProcessorFactories, + Map> responseProcessorFactories, + NamedWriteableRegistry namedWriteableRegistry, + OperationMetrics totalRequestProcessingMetrics, + OperationMetrics totalResponseProcessingMetrics + ) throws Exception { + String description = ConfigurationUtils.readOptionalStringProperty(null, null, config, DESCRIPTION_KEY); + Integer version = ConfigurationUtils.readIntProperty(null, null, config, VERSION_KEY, null); + List> requestProcessorConfigs = ConfigurationUtils.readOptionalList(null, null, config, REQUEST_PROCESSORS_KEY); + List requestProcessors = readProcessors(requestProcessorFactories, requestProcessorConfigs); + List> responseProcessorConfigs = ConfigurationUtils.readOptionalList( + null, + null, + config, + RESPONSE_PROCESSORS_KEY + ); + List responseProcessors = readProcessors(responseProcessorFactories, responseProcessorConfigs); + if (config.isEmpty() == false) { + throw new OpenSearchParseException( + "pipeline [" + + id + + "] doesn't support one or more provided configuration parameters " + + Arrays.toString(config.keySet().toArray()) + ); + } + return new PipelineWithMetrics( + id, + description, + version, + requestProcessors, + responseProcessors, + namedWriteableRegistry, + totalRequestProcessingMetrics, + totalResponseProcessingMetrics, + System::nanoTime + ); + + } + + private static List readProcessors( + Map> processorFactories, + List> requestProcessorConfigs + ) throws Exception { + List processors = new ArrayList<>(); + if (requestProcessorConfigs == null) { + return processors; + } + for (Map processorConfigWithKey : requestProcessorConfigs) { + for (Map.Entry entry : processorConfigWithKey.entrySet()) { + String type = entry.getKey(); + if (!processorFactories.containsKey(type)) { + throw new IllegalArgumentException("Invalid processor type " + type); + } + Map config = (Map) entry.getValue(); + String tag = ConfigurationUtils.readOptionalStringProperty(null, null, config, TAG_KEY); + String description = ConfigurationUtils.readOptionalStringProperty(null, tag, config, DESCRIPTION_KEY); + processors.add(processorFactories.get(type).create(processorFactories, tag, description, config)); + } + } + return Collections.unmodifiableList(processors); + } + + @Override + protected void beforeTransformRequest() { + super.beforeTransformRequest(); + totalRequestMetrics.before(); + pipelineRequestMetrics.before(); + } + + @Override + protected void afterTransformRequest(long timeInNanos) { + super.afterTransformRequest(timeInNanos); + totalRequestMetrics.after(timeInNanos); + pipelineRequestMetrics.after(timeInNanos); + } + + @Override + protected void onTransformRequestFailure() { + super.onTransformRequestFailure(); + totalRequestMetrics.failed(); + pipelineRequestMetrics.failed(); + } + + protected void beforeRequestProcessor(Processor processor) { + requestProcessorMetrics.get(getProcessorKey(processor)).before(); + } + + protected void afterRequestProcessor(Processor processor, long timeInNanos) { + requestProcessorMetrics.get(getProcessorKey(processor)).after(timeInNanos); + } + + protected void onRequestProcessorFailed(Processor processor) { + requestProcessorMetrics.get(getProcessorKey(processor)).failed(); + } + + protected void beforeTransformResponse() { + super.beforeTransformRequest(); + totalResponseMetrics.before(); + pipelineResponseMetrics.before(); + } + + protected void afterTransformResponse(long timeInNanos) { + super.afterTransformResponse(timeInNanos); + totalResponseMetrics.after(timeInNanos); + pipelineResponseMetrics.after(timeInNanos); + } + + protected void onTransformResponseFailure() { + super.onTransformResponseFailure(); + totalResponseMetrics.failed(); + pipelineResponseMetrics.failed(); + } + + protected void beforeResponseProcessor(Processor processor) { + responseProcessorMetrics.get(getProcessorKey(processor)).before(); + } + + protected void afterResponseProcessor(Processor processor, long timeInNanos) { + responseProcessorMetrics.get(getProcessorKey(processor)).after(timeInNanos); + } + + protected void onResponseProcessorFailed(Processor processor) { + responseProcessorMetrics.get(getProcessorKey(processor)).failed(); + } + + void copyMetrics(PipelineWithMetrics oldPipeline) { + pipelineRequestMetrics.add(oldPipeline.pipelineRequestMetrics); + pipelineResponseMetrics.add(oldPipeline.pipelineResponseMetrics); + copyProcessorMetrics(requestProcessorMetrics, oldPipeline.requestProcessorMetrics); + copyProcessorMetrics(responseProcessorMetrics, oldPipeline.responseProcessorMetrics); + } + + private static void copyProcessorMetrics( + Map newProcessorMetrics, + Map oldProcessorMetrics + ) { + for (Map.Entry oldProcessorMetric : oldProcessorMetrics.entrySet()) { + if (newProcessorMetrics.containsKey(oldProcessorMetric.getKey())) { + newProcessorMetrics.get(oldProcessorMetric.getKey()).add(oldProcessorMetric.getValue()); + } + } + } + + private static String getProcessorKey(Processor processor) { + String key = processor.getType(); + if (processor.getTag() != null) { + return key + ":" + processor.getTag(); + } + return key; + } + + void populateStats(SearchPipelineStats.Builder statsBuilder) { + statsBuilder.addPipelineStats(getId(), pipelineRequestMetrics, pipelineResponseMetrics); + for (Processor processor : getSearchRequestProcessors()) { + String key = getProcessorKey(processor); + statsBuilder.addRequestProcessorStats(getId(), key, processor.getType(), requestProcessorMetrics.get(key)); + } + for (Processor processor : getSearchResponseProcessors()) { + String key = getProcessorKey(processor); + statsBuilder.addResponseProcessorStats(getId(), key, processor.getType(), responseProcessorMetrics.get(key)); + } + } +} diff --git a/server/src/main/java/org/opensearch/search/pipeline/SearchPipelineService.java b/server/src/main/java/org/opensearch/search/pipeline/SearchPipelineService.java index 87c09bd971284..434c8fbfacc74 100644 --- a/server/src/main/java/org/opensearch/search/pipeline/SearchPipelineService.java +++ b/server/src/main/java/org/opensearch/search/pipeline/SearchPipelineService.java @@ -30,6 +30,7 @@ import org.opensearch.cluster.service.ClusterManagerTaskThrottler; import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.io.stream.NamedWriteableRegistry; +import org.opensearch.common.metrics.OperationMetrics; import org.opensearch.common.regex.Regex; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; @@ -80,6 +81,9 @@ public class SearchPipelineService implements ClusterStateApplier, ReportingServ private final NamedWriteableRegistry namedWriteableRegistry; private volatile ClusterState state; + private final OperationMetrics totalRequestProcessingMetrics = new OperationMetrics(); + private final OperationMetrics totalResponseProcessingMetrics = new OperationMetrics(); + private final boolean isEnabled; public SearchPipelineService( @@ -172,26 +176,26 @@ void innerUpdatePipelines(SearchPipelineMetadata newSearchPipelineMetadata) { newPipelines = new HashMap<>(existingPipelines); } try { - Pipeline newPipeline = Pipeline.create( + PipelineWithMetrics newPipeline = PipelineWithMetrics.create( newConfiguration.getId(), newConfiguration.getConfigAsMap(), requestProcessorFactories, responseProcessorFactories, - namedWriteableRegistry + namedWriteableRegistry, + totalRequestProcessingMetrics, + totalResponseProcessingMetrics ); newPipelines.put(newConfiguration.getId(), new PipelineHolder(newConfiguration, newPipeline)); - if (previous == null) { - continue; + if (previous != null) { + newPipeline.copyMetrics(previous.pipeline); } - // TODO -- once we add in pipeline metrics (like in ingest pipelines), we will need to deep-copy - // the old pipeline's metrics into the new pipeline. } catch (Exception e) { OpenSearchParseException parseException = new OpenSearchParseException( "Error updating pipeline with id [" + newConfiguration.getId() + "]", e ); - // TODO -- replace pipeline with one that throws an exception when we try to use it + // TODO -- replace pipeline with one that throws this exception when we try to use it if (exceptions == null) { exceptions = new ArrayList<>(); } @@ -271,12 +275,14 @@ void validatePipeline(Map searchPipelineInfos throw new IllegalStateException("Search pipeline info is empty"); } Map pipelineConfig = XContentHelper.convertToMap(request.getSource(), false, request.getXContentType()).v2(); - Pipeline pipeline = Pipeline.create( + Pipeline pipeline = PipelineWithMetrics.create( request.getId(), pipelineConfig, requestProcessorFactories, responseProcessorFactories, - namedWriteableRegistry + namedWriteableRegistry, + new OperationMetrics(), // Use ephemeral metrics for validation + new OperationMetrics() ); List exceptions = new ArrayList<>(); for (SearchRequestProcessor processor : pipeline.getSearchRequestProcessors()) { @@ -367,12 +373,14 @@ public PipelinedRequest resolvePipeline(SearchRequest searchRequest) throws Exce ); } try { - pipeline = Pipeline.create( + pipeline = PipelineWithMetrics.create( AD_HOC_PIPELINE_ID, searchRequest.source().searchPipelineSource(), requestProcessorFactories, responseProcessorFactories, - namedWriteableRegistry + namedWriteableRegistry, + totalRequestProcessingMetrics, + totalResponseProcessingMetrics ); } catch (Exception e) { throw new SearchPipelineProcessingException(e); @@ -400,12 +408,8 @@ public PipelinedRequest resolvePipeline(SearchRequest searchRequest) throws Exce pipeline = pipelineHolder.pipeline; } } - try { - SearchRequest transformedRequest = pipeline.transformRequest(searchRequest); - return new PipelinedRequest(pipeline, transformedRequest); - } catch (Exception e) { - throw new SearchPipelineProcessingException(e); - } + SearchRequest transformedRequest = pipeline.transformRequest(searchRequest); + return new PipelinedRequest(pipeline, transformedRequest); } Map> getRequestProcessorFactories() { @@ -431,6 +435,16 @@ public SearchPipelineInfo info() { ); } + public SearchPipelineStats stats() { + SearchPipelineStats.Builder builder = new SearchPipelineStats.Builder(); + builder.withTotalStats(totalRequestProcessingMetrics, totalResponseProcessingMetrics); + for (PipelineHolder pipelineHolder : pipelines.values()) { + PipelineWithMetrics pipeline = pipelineHolder.pipeline; + pipeline.populateStats(builder); + } + return builder.build(); + } + public static List getPipelines(ClusterState clusterState, String... ids) { SearchPipelineMetadata metadata = clusterState.getMetadata().custom(SearchPipelineMetadata.TYPE); return innerGetPipelines(metadata, ids); @@ -474,9 +488,9 @@ Map getPipelines() { static class PipelineHolder { final PipelineConfiguration configuration; - final Pipeline pipeline; + final PipelineWithMetrics pipeline; - PipelineHolder(PipelineConfiguration configuration, Pipeline pipeline) { + PipelineHolder(PipelineConfiguration configuration, PipelineWithMetrics pipeline) { this.configuration = Objects.requireNonNull(configuration); this.pipeline = Objects.requireNonNull(pipeline); } diff --git a/server/src/main/java/org/opensearch/search/pipeline/SearchPipelineStats.java b/server/src/main/java/org/opensearch/search/pipeline/SearchPipelineStats.java new file mode 100644 index 0000000000000..4261bfe99160a --- /dev/null +++ b/server/src/main/java/org/opensearch/search/pipeline/SearchPipelineStats.java @@ -0,0 +1,367 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.search.pipeline; + +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.metrics.OperationMetrics; +import org.opensearch.common.metrics.OperationStats; +import org.opensearch.core.xcontent.ToXContentFragment; +import org.opensearch.core.xcontent.XContentBuilder; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.TreeMap; + +import static java.util.Collections.emptyList; +import static java.util.Collections.unmodifiableList; +import static java.util.Collections.unmodifiableMap; + +/** + * Serializable, immutable search pipeline statistics to be returned via stats APIs. + * + * @opensearch.internal + */ +public class SearchPipelineStats implements Writeable, ToXContentFragment { + + private final OperationStats totalRequestStats; + private final OperationStats totalResponseStats; + private final List perPipelineStats; + private final Map perPipelineProcessorStats; + + public SearchPipelineStats( + OperationStats totalRequestStats, + OperationStats totalResponseStats, + List perPipelineStats, + Map perPipelineProcessorStats + ) { + this.totalRequestStats = totalRequestStats; + this.totalResponseStats = totalResponseStats; + this.perPipelineStats = perPipelineStats; + this.perPipelineProcessorStats = perPipelineProcessorStats; + } + + public SearchPipelineStats(StreamInput in) throws IOException { + this.totalRequestStats = new OperationStats(in); + this.totalResponseStats = new OperationStats(in); + int size = in.readVInt(); + List perPipelineStats = new ArrayList<>(size); + Map pipelineDetailStatsMap = new TreeMap<>(); + for (int i = 0; i < size; i++) { + String pipelineId = in.readString(); + OperationStats pipelineRequestStats = new OperationStats(in); + OperationStats pipelineResponseStats = new OperationStats(in); + perPipelineStats.add(new PerPipelineStats(pipelineId, pipelineRequestStats, pipelineResponseStats)); + int numRequestProcessors = in.readVInt(); + List requestProcessorStats = new ArrayList<>(numRequestProcessors); + for (int j = 0; j < numRequestProcessors; j++) { + String processorName = in.readString(); + String processorType = in.readString(); + OperationStats processorStats = new OperationStats(in); + requestProcessorStats.add(new ProcessorStats(processorName, processorType, processorStats)); + } + int numResponseProcessors = in.readVInt(); + List responseProcessorStats = new ArrayList<>(numResponseProcessors); + for (int j = 0; j < numResponseProcessors; j++) { + String processorName = in.readString(); + String processorType = in.readString(); + OperationStats processorStats = new OperationStats(in); + responseProcessorStats.add(new ProcessorStats(processorName, processorType, processorStats)); + } + pipelineDetailStatsMap.put( + pipelineId, + new PipelineDetailStats(unmodifiableList(requestProcessorStats), unmodifiableList(responseProcessorStats)) + ); + } + this.perPipelineStats = unmodifiableList(perPipelineStats); + this.perPipelineProcessorStats = unmodifiableMap(pipelineDetailStatsMap); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject("search_pipeline"); + builder.startObject("total_request"); + totalRequestStats.toXContent(builder, params); + builder.endObject(); + builder.startObject("total_response"); + totalResponseStats.toXContent(builder, params); + builder.endObject(); + builder.startObject("pipelines"); + for (PerPipelineStats pipelineStat : perPipelineStats) { + builder.startObject(pipelineStat.pipelineId); + builder.startObject("request"); + pipelineStat.requestStats.toXContent(builder, params); + builder.endObject(); + builder.startObject("response"); + pipelineStat.responseStats.toXContent(builder, params); + builder.endObject(); + + PipelineDetailStats pipelineDetailStats = perPipelineProcessorStats.get(pipelineStat.pipelineId); + builder.startArray("request_processors"); + for (ProcessorStats processorStats : pipelineDetailStats.requestProcessorStats) { + builder.startObject(); + processorStats.toXContent(builder, params); + builder.endObject(); + } + builder.endArray(); + builder.startArray("response_processors"); + for (ProcessorStats processorStats : pipelineDetailStats.responseProcessorStats) { + builder.startObject(); + processorStats.toXContent(builder, params); + builder.endObject(); + } + builder.endArray(); + builder.endObject(); + } + builder.endObject(); + builder.endObject(); + return builder; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + totalRequestStats.writeTo(out); + totalResponseStats.writeTo(out); + out.writeVInt(perPipelineStats.size()); + for (PerPipelineStats pipelineStat : perPipelineStats) { + out.writeString(pipelineStat.pipelineId); + pipelineStat.requestStats.writeTo(out); + pipelineStat.responseStats.writeTo(out); + PipelineDetailStats pipelineDetailStats = perPipelineProcessorStats.get(pipelineStat.pipelineId); + out.writeVInt(pipelineDetailStats.requestProcessorStats.size()); + for (ProcessorStats processorStats : pipelineDetailStats.requestProcessorStats) { + out.writeString(processorStats.processorName); + out.writeString(processorStats.processorType); + processorStats.stats.writeTo(out); + } + out.writeVInt(pipelineDetailStats.responseProcessorStats.size()); + for (ProcessorStats processorStats : pipelineDetailStats.responseProcessorStats) { + out.writeString(processorStats.processorName); + out.writeString(processorStats.processorType); + processorStats.stats.writeTo(out); + } + } + } + + static class Builder { + private OperationStats totalRequestStats; + private OperationStats totalResponseStats; + private final List perPipelineStats = new ArrayList<>(); + private final Map> requestProcessorStatsPerPipeline = new HashMap<>(); + private final Map> responseProcessorStatsPerPipeline = new HashMap<>(); + + Builder withTotalStats(OperationMetrics totalRequestMetrics, OperationMetrics totalResponseMetrics) { + this.totalRequestStats = totalRequestMetrics.createStats(); + this.totalResponseStats = totalResponseMetrics.createStats(); + return this; + } + + Builder addPipelineStats(String pipelineId, OperationMetrics pipelineRequestMetrics, OperationMetrics pipelineResponseMetrics) { + this.perPipelineStats.add( + new PerPipelineStats(pipelineId, pipelineRequestMetrics.createStats(), pipelineResponseMetrics.createStats()) + ); + return this; + } + + Builder addRequestProcessorStats(String pipelineId, String processorName, String processorType, OperationMetrics processorMetrics) { + this.requestProcessorStatsPerPipeline.computeIfAbsent(pipelineId, k -> new ArrayList<>()) + .add(new ProcessorStats(processorName, processorType, processorMetrics.createStats())); + return this; + } + + Builder addResponseProcessorStats( + String pipelineId, + String processorName, + String processorType, + OperationMetrics processorMetrics + ) { + this.responseProcessorStatsPerPipeline.computeIfAbsent(pipelineId, k -> new ArrayList<>()) + .add(new ProcessorStats(processorName, processorType, processorMetrics.createStats())); + return this; + } + + SearchPipelineStats build() { + Map pipelineDetailStatsMap = new TreeMap<>(); + for (PerPipelineStats pipelineStat : perPipelineStats) { + List requestProcessorStats = requestProcessorStatsPerPipeline.getOrDefault( + pipelineStat.pipelineId, + emptyList() + ); + List responseProcessorStats = responseProcessorStatsPerPipeline.getOrDefault( + pipelineStat.pipelineId, + emptyList() + ); + PipelineDetailStats pipelineDetailStats = new PipelineDetailStats( + unmodifiableList(requestProcessorStats), + unmodifiableList(responseProcessorStats) + ); + pipelineDetailStatsMap.put(pipelineStat.pipelineId, pipelineDetailStats); + } + return new SearchPipelineStats( + totalRequestStats, + totalResponseStats, + unmodifiableList(perPipelineStats), + unmodifiableMap(pipelineDetailStatsMap) + ); + } + } + + static class PerPipelineStats { + private final String pipelineId; + private final OperationStats requestStats; + private final OperationStats responseStats; + + public PerPipelineStats(String pipelineId, OperationStats requestStats, OperationStats responseStats) { + this.pipelineId = pipelineId; + this.requestStats = requestStats; + this.responseStats = responseStats; + } + + public String getPipelineId() { + return pipelineId; + } + + public OperationStats getRequestStats() { + return requestStats; + } + + public OperationStats getResponseStats() { + return responseStats; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + PerPipelineStats that = (PerPipelineStats) o; + return pipelineId.equals(that.pipelineId) && requestStats.equals(that.requestStats) && responseStats.equals(that.responseStats); + } + + @Override + public int hashCode() { + return Objects.hash(pipelineId, requestStats, responseStats); + } + } + + static class PipelineDetailStats { + private final List requestProcessorStats; + private final List responseProcessorStats; + + public PipelineDetailStats(List requestProcessorStats, List responseProcessorStats) { + this.requestProcessorStats = requestProcessorStats; + this.responseProcessorStats = responseProcessorStats; + } + + public List requestProcessorStats() { + return requestProcessorStats; + } + + public List responseProcessorStats() { + return responseProcessorStats; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + PipelineDetailStats that = (PipelineDetailStats) o; + return requestProcessorStats.equals(that.requestProcessorStats) && responseProcessorStats.equals(that.responseProcessorStats); + } + + @Override + public int hashCode() { + return Objects.hash(requestProcessorStats, responseProcessorStats); + } + } + + static class ProcessorStats implements ToXContentFragment { + private final String processorName; // type:tag + private final String processorType; + private final OperationStats stats; + + public ProcessorStats(String processorName, String processorType, OperationStats stats) { + this.processorName = processorName; + this.processorType = processorType; + this.stats = stats; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ProcessorStats that = (ProcessorStats) o; + return processorName.equals(that.processorName) && processorType.equals(that.processorType) && stats.equals(that.stats); + } + + @Override + public int hashCode() { + return Objects.hash(processorName, processorType, stats); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(processorName); + builder.field("type", processorType); + builder.startObject("stats"); + stats.toXContent(builder, params); + builder.endObject(); + builder.endObject(); + return builder; + } + + String getProcessorName() { + return processorName; + } + + String getProcessorType() { + return processorType; + } + + OperationStats getStats() { + return stats; + } + } + + OperationStats getTotalRequestStats() { + return totalRequestStats; + } + + OperationStats getTotalResponseStats() { + return totalResponseStats; + } + + List getPipelineStats() { + return perPipelineStats; + } + + Map getPerPipelineProcessorStats() { + return perPipelineProcessorStats; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + SearchPipelineStats stats = (SearchPipelineStats) o; + return totalRequestStats.equals(stats.totalRequestStats) + && totalResponseStats.equals(stats.totalResponseStats) + && perPipelineStats.equals(stats.perPipelineStats) + && perPipelineProcessorStats.equals(stats.perPipelineProcessorStats); + } + + @Override + public int hashCode() { + return Objects.hash(totalRequestStats, totalResponseStats, perPipelineStats, perPipelineProcessorStats); + } +} diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java b/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java index d99b93b780140..1e2e085333e50 100644 --- a/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java +++ b/server/src/test/java/org/opensearch/action/admin/cluster/node/stats/NodeStatsTests.java @@ -37,6 +37,7 @@ import org.opensearch.cluster.service.ClusterManagerThrottlingStats; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.common.metrics.OperationStats; import org.opensearch.discovery.DiscoveryStats; import org.opensearch.cluster.coordination.PendingClusterStateStats; import org.opensearch.cluster.coordination.PublishClusterStateStats; @@ -338,40 +339,31 @@ public void testSerialization() throws IOException { if (ingestStats == null) { assertNull(deserializedIngestStats); } else { - IngestStats.Stats totalStats = ingestStats.getTotalStats(); - assertEquals(totalStats.getIngestCount(), deserializedIngestStats.getTotalStats().getIngestCount()); - assertEquals(totalStats.getIngestCurrent(), deserializedIngestStats.getTotalStats().getIngestCurrent()); - assertEquals(totalStats.getIngestFailedCount(), deserializedIngestStats.getTotalStats().getIngestFailedCount()); - assertEquals(totalStats.getIngestTimeInMillis(), deserializedIngestStats.getTotalStats().getIngestTimeInMillis()); + OperationStats totalStats = ingestStats.getTotalStats(); + assertEquals(totalStats.getCount(), deserializedIngestStats.getTotalStats().getCount()); + assertEquals(totalStats.getCurrent(), deserializedIngestStats.getTotalStats().getCurrent()); + assertEquals(totalStats.getFailedCount(), deserializedIngestStats.getTotalStats().getFailedCount()); + assertEquals(totalStats.getTotalTimeInMillis(), deserializedIngestStats.getTotalStats().getTotalTimeInMillis()); assertEquals(ingestStats.getPipelineStats().size(), deserializedIngestStats.getPipelineStats().size()); for (IngestStats.PipelineStat pipelineStat : ingestStats.getPipelineStats()) { String pipelineId = pipelineStat.getPipelineId(); - IngestStats.Stats deserializedPipelineStats = getPipelineStats( - deserializedIngestStats.getPipelineStats(), - pipelineId - ); - assertEquals(pipelineStat.getStats().getIngestFailedCount(), deserializedPipelineStats.getIngestFailedCount()); - assertEquals(pipelineStat.getStats().getIngestTimeInMillis(), deserializedPipelineStats.getIngestTimeInMillis()); - assertEquals(pipelineStat.getStats().getIngestCurrent(), deserializedPipelineStats.getIngestCurrent()); - assertEquals(pipelineStat.getStats().getIngestCount(), deserializedPipelineStats.getIngestCount()); + OperationStats deserializedPipelineStats = getPipelineStats(deserializedIngestStats.getPipelineStats(), pipelineId); + assertEquals(pipelineStat.getStats().getFailedCount(), deserializedPipelineStats.getFailedCount()); + assertEquals(pipelineStat.getStats().getTotalTimeInMillis(), deserializedPipelineStats.getTotalTimeInMillis()); + assertEquals(pipelineStat.getStats().getCurrent(), deserializedPipelineStats.getCurrent()); + assertEquals(pipelineStat.getStats().getCount(), deserializedPipelineStats.getCount()); List processorStats = ingestStats.getProcessorStats().get(pipelineId); // intentionally validating identical order Iterator it = deserializedIngestStats.getProcessorStats().get(pipelineId).iterator(); for (IngestStats.ProcessorStat processorStat : processorStats) { IngestStats.ProcessorStat deserializedProcessorStat = it.next(); + assertEquals(processorStat.getStats().getFailedCount(), deserializedProcessorStat.getStats().getFailedCount()); assertEquals( - processorStat.getStats().getIngestFailedCount(), - deserializedProcessorStat.getStats().getIngestFailedCount() - ); - assertEquals( - processorStat.getStats().getIngestTimeInMillis(), - deserializedProcessorStat.getStats().getIngestTimeInMillis() + processorStat.getStats().getTotalTimeInMillis(), + deserializedProcessorStat.getStats().getTotalTimeInMillis() ); - assertEquals( - processorStat.getStats().getIngestCurrent(), - deserializedProcessorStat.getStats().getIngestCurrent() - ); - assertEquals(processorStat.getStats().getIngestCount(), deserializedProcessorStat.getStats().getIngestCount()); + assertEquals(processorStat.getStats().getCurrent(), deserializedProcessorStat.getStats().getCurrent()); + assertEquals(processorStat.getStats().getCount(), deserializedProcessorStat.getStats().getCount()); } assertFalse(it.hasNext()); } @@ -650,7 +642,7 @@ public static NodeStats createNodeStats() { : null; IngestStats ingestStats = null; if (frequently()) { - IngestStats.Stats totalStats = new IngestStats.Stats( + OperationStats totalStats = new OperationStats( randomNonNegativeLong(), randomNonNegativeLong(), randomNonNegativeLong(), @@ -665,7 +657,7 @@ public static NodeStats createNodeStats() { ingestPipelineStats.add( new IngestStats.PipelineStat( pipelineId, - new IngestStats.Stats( + new OperationStats( randomNonNegativeLong(), randomNonNegativeLong(), randomNonNegativeLong(), @@ -676,7 +668,7 @@ public static NodeStats createNodeStats() { List processorPerPipeline = new ArrayList<>(numProcessors); for (int j = 0; j < numProcessors; j++) { - IngestStats.Stats processorStats = new IngestStats.Stats( + OperationStats processorStats = new OperationStats( randomNonNegativeLong(), randomNonNegativeLong(), randomNonNegativeLong(), @@ -750,11 +742,12 @@ public static NodeStats createNodeStats() { clusterManagerThrottlingStats, weightedRoutingStats, null, + null, null ); } - private IngestStats.Stats getPipelineStats(List pipelineStats, String id) { + private OperationStats getPipelineStats(List pipelineStats, String id) { return pipelineStats.stream().filter(p1 -> p1.getPipelineId().equals(id)).findFirst().map(p2 -> p2.getStats()).orElse(null); } } diff --git a/server/src/test/java/org/opensearch/action/admin/cluster/stats/ClusterStatsNodesTests.java b/server/src/test/java/org/opensearch/action/admin/cluster/stats/ClusterStatsNodesTests.java index d3a40868bc389..627ada7092273 100644 --- a/server/src/test/java/org/opensearch/action/admin/cluster/stats/ClusterStatsNodesTests.java +++ b/server/src/test/java/org/opensearch/action/admin/cluster/stats/ClusterStatsNodesTests.java @@ -89,15 +89,15 @@ public void testIngestStats() throws Exception { processorStats.compute(stat.getType(), (key, value) -> { if (value == null) { return new long[] { - stat.getStats().getIngestCount(), - stat.getStats().getIngestFailedCount(), - stat.getStats().getIngestCurrent(), - stat.getStats().getIngestTimeInMillis() }; + stat.getStats().getCount(), + stat.getStats().getFailedCount(), + stat.getStats().getCurrent(), + stat.getStats().getTotalTimeInMillis() }; } else { - value[0] += stat.getStats().getIngestCount(); - value[1] += stat.getStats().getIngestFailedCount(); - value[2] += stat.getStats().getIngestCurrent(); - value[3] += stat.getStats().getIngestTimeInMillis(); + value[0] += stat.getStats().getCount(); + value[1] += stat.getStats().getFailedCount(); + value[2] += stat.getStats().getCurrent(); + value[3] += stat.getStats().getTotalTimeInMillis(); return value; } }); diff --git a/server/src/test/java/org/opensearch/cluster/DiskUsageTests.java b/server/src/test/java/org/opensearch/cluster/DiskUsageTests.java index 73349d45bd5c7..e5833ea619774 100644 --- a/server/src/test/java/org/opensearch/cluster/DiskUsageTests.java +++ b/server/src/test/java/org/opensearch/cluster/DiskUsageTests.java @@ -189,6 +189,7 @@ public void testFillDiskUsage() { null, null, null, + null, null ), new NodeStats( @@ -214,6 +215,7 @@ public void testFillDiskUsage() { null, null, null, + null, null ), new NodeStats( @@ -239,6 +241,7 @@ public void testFillDiskUsage() { null, null, null, + null, null ) ); @@ -295,6 +298,7 @@ public void testFillDiskUsageSomeInvalidValues() { null, null, null, + null, null ), new NodeStats( @@ -320,6 +324,7 @@ public void testFillDiskUsageSomeInvalidValues() { null, null, null, + null, null ), new NodeStats( @@ -345,6 +350,7 @@ public void testFillDiskUsageSomeInvalidValues() { null, null, null, + null, null ) ); diff --git a/server/src/test/java/org/opensearch/ingest/CompoundProcessorTests.java b/server/src/test/java/org/opensearch/ingest/CompoundProcessorTests.java index b299ac4d66996..76301acac0c19 100644 --- a/server/src/test/java/org/opensearch/ingest/CompoundProcessorTests.java +++ b/server/src/test/java/org/opensearch/ingest/CompoundProcessorTests.java @@ -33,6 +33,7 @@ package org.opensearch.ingest; import org.opensearch.OpenSearchException; +import org.opensearch.common.metrics.OperationStats; import org.opensearch.test.OpenSearchTestCase; import org.junit.Before; @@ -433,10 +434,10 @@ private void assertStats(CompoundProcessor compoundProcessor, long count, long f } private void assertStats(int processor, CompoundProcessor compoundProcessor, long current, long count, long failed, long time) { - IngestStats.Stats stats = compoundProcessor.getProcessorsWithMetrics().get(processor).v2().createStats(); - assertThat(stats.getIngestCount(), equalTo(count)); - assertThat(stats.getIngestCurrent(), equalTo(current)); - assertThat(stats.getIngestFailedCount(), equalTo(failed)); - assertThat(stats.getIngestTimeInMillis(), equalTo(time)); + OperationStats stats = compoundProcessor.getProcessorsWithMetrics().get(processor).v2().createStats(); + assertThat(stats.getCount(), equalTo(count)); + assertThat(stats.getCurrent(), equalTo(current)); + assertThat(stats.getFailedCount(), equalTo(failed)); + assertThat(stats.getTotalTimeInMillis(), equalTo(time)); } } diff --git a/server/src/test/java/org/opensearch/ingest/ConditionalProcessorTests.java b/server/src/test/java/org/opensearch/ingest/ConditionalProcessorTests.java index a383ab9b97918..921ac10c02862 100644 --- a/server/src/test/java/org/opensearch/ingest/ConditionalProcessorTests.java +++ b/server/src/test/java/org/opensearch/ingest/ConditionalProcessorTests.java @@ -32,6 +32,7 @@ package org.opensearch.ingest; +import org.opensearch.common.metrics.OperationStats; import org.opensearch.common.settings.Settings; import org.opensearch.script.IngestConditionalScript; import org.opensearch.script.MockScriptEngine; @@ -250,10 +251,10 @@ private static void assertMutatingCtxThrows(Consumer> mutati } private static void assertStats(ConditionalProcessor conditionalProcessor, long count, long failed, long time) { - IngestStats.Stats stats = conditionalProcessor.getMetric().createStats(); - assertThat(stats.getIngestCount(), equalTo(count)); - assertThat(stats.getIngestCurrent(), equalTo(0L)); - assertThat(stats.getIngestFailedCount(), equalTo(failed)); - assertThat(stats.getIngestTimeInMillis(), greaterThanOrEqualTo(time)); + OperationStats stats = conditionalProcessor.getMetric().createStats(); + assertThat(stats.getCount(), equalTo(count)); + assertThat(stats.getCurrent(), equalTo(0L)); + assertThat(stats.getFailedCount(), equalTo(failed)); + assertThat(stats.getTotalTimeInMillis(), greaterThanOrEqualTo(time)); } } diff --git a/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java b/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java index 4176a32e32ad3..19fef468c529e 100644 --- a/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java +++ b/server/src/test/java/org/opensearch/ingest/IngestServiceTests.java @@ -58,6 +58,7 @@ import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.SetOnce; import org.opensearch.common.bytes.BytesArray; +import org.opensearch.common.metrics.OperationStats; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.concurrent.OpenSearchExecutors; import org.opensearch.core.xcontent.XContentBuilder; @@ -1739,14 +1740,14 @@ private void assertPipelineStats(List pipelineStats, S assertStats(getPipelineStats(pipelineStats, pipelineId), count, failed, time); } - private void assertStats(IngestStats.Stats stats, long count, long failed, long time) { - assertThat(stats.getIngestCount(), equalTo(count)); - assertThat(stats.getIngestCurrent(), equalTo(0L)); - assertThat(stats.getIngestFailedCount(), equalTo(failed)); - assertThat(stats.getIngestTimeInMillis(), greaterThanOrEqualTo(time)); + private void assertStats(OperationStats stats, long count, long failed, long time) { + assertThat(stats.getCount(), equalTo(count)); + assertThat(stats.getCurrent(), equalTo(0L)); + assertThat(stats.getFailedCount(), equalTo(failed)); + assertThat(stats.getTotalTimeInMillis(), greaterThanOrEqualTo(time)); } - private IngestStats.Stats getPipelineStats(List pipelineStats, String id) { + private OperationStats getPipelineStats(List pipelineStats, String id) { return pipelineStats.stream().filter(p1 -> p1.getPipelineId().equals(id)).findFirst().map(p2 -> p2.getStats()).orElse(null); } } diff --git a/server/src/test/java/org/opensearch/ingest/IngestStatsTests.java b/server/src/test/java/org/opensearch/ingest/IngestStatsTests.java index b5c74f0ee5d16..b17e24ee5424d 100644 --- a/server/src/test/java/org/opensearch/ingest/IngestStatsTests.java +++ b/server/src/test/java/org/opensearch/ingest/IngestStatsTests.java @@ -35,6 +35,7 @@ import org.opensearch.common.collect.MapBuilder; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.common.metrics.OperationStats; import org.opensearch.test.OpenSearchTestCase; import java.io.IOException; @@ -48,7 +49,7 @@ public class IngestStatsTests extends OpenSearchTestCase { public void testSerialization() throws IOException { - IngestStats.Stats totalStats = new IngestStats.Stats(50, 100, 200, 300); + OperationStats totalStats = new OperationStats(50, 100, 200, 300); List pipelineStats = createPipelineStats(); Map> processorStats = createProcessorStats(pipelineStats); IngestStats ingestStats = new IngestStats(totalStats, pipelineStats, processorStats); @@ -57,20 +58,20 @@ public void testSerialization() throws IOException { } private List createPipelineStats() { - IngestStats.PipelineStat pipeline1Stats = new IngestStats.PipelineStat("pipeline1", new IngestStats.Stats(3, 3, 3, 3)); - IngestStats.PipelineStat pipeline2Stats = new IngestStats.PipelineStat("pipeline2", new IngestStats.Stats(47, 97, 197, 297)); - IngestStats.PipelineStat pipeline3Stats = new IngestStats.PipelineStat("pipeline3", new IngestStats.Stats(0, 0, 0, 0)); + IngestStats.PipelineStat pipeline1Stats = new IngestStats.PipelineStat("pipeline1", new OperationStats(3, 3, 3, 3)); + IngestStats.PipelineStat pipeline2Stats = new IngestStats.PipelineStat("pipeline2", new OperationStats(47, 97, 197, 297)); + IngestStats.PipelineStat pipeline3Stats = new IngestStats.PipelineStat("pipeline3", new OperationStats(0, 0, 0, 0)); return Stream.of(pipeline1Stats, pipeline2Stats, pipeline3Stats).collect(Collectors.toList()); } private Map> createProcessorStats(List pipelineStats) { assert (pipelineStats.size() >= 2); - IngestStats.ProcessorStat processor1Stat = new IngestStats.ProcessorStat("processor1", "type", new IngestStats.Stats(1, 1, 1, 1)); - IngestStats.ProcessorStat processor2Stat = new IngestStats.ProcessorStat("processor2", "type", new IngestStats.Stats(2, 2, 2, 2)); + IngestStats.ProcessorStat processor1Stat = new IngestStats.ProcessorStat("processor1", "type", new OperationStats(1, 1, 1, 1)); + IngestStats.ProcessorStat processor2Stat = new IngestStats.ProcessorStat("processor2", "type", new OperationStats(2, 2, 2, 2)); IngestStats.ProcessorStat processor3Stat = new IngestStats.ProcessorStat( "processor3", "type", - new IngestStats.Stats(47, 97, 197, 297) + new OperationStats(47, 97, 197, 297) ); // pipeline1 -> processor1,processor2; pipeline2 -> processor3 return MapBuilder.>newMapBuilder() @@ -132,14 +133,14 @@ private void assertIngestStats( } - private void assertStats(IngestStats.Stats fromObject, IngestStats.Stats fromStream) { - assertEquals(fromObject.getIngestCount(), fromStream.getIngestCount()); - assertEquals(fromObject.getIngestFailedCount(), fromStream.getIngestFailedCount()); - assertEquals(fromObject.getIngestTimeInMillis(), fromStream.getIngestTimeInMillis()); - assertEquals(fromObject.getIngestCurrent(), fromStream.getIngestCurrent()); + private void assertStats(OperationStats fromObject, OperationStats fromStream) { + assertEquals(fromObject.getCount(), fromStream.getCount()); + assertEquals(fromObject.getFailedCount(), fromStream.getFailedCount()); + assertEquals(fromObject.getTotalTimeInMillis(), fromStream.getTotalTimeInMillis()); + assertEquals(fromObject.getCurrent(), fromStream.getCurrent()); } - private IngestStats.Stats getPipelineStats(List pipelineStats, String id) { + private OperationStats getPipelineStats(List pipelineStats, String id) { return pipelineStats.stream().filter(p1 -> p1.getPipelineId().equals(id)).findFirst().map(p2 -> p2.getStats()).orElse(null); } } diff --git a/server/src/test/java/org/opensearch/ingest/PipelineProcessorTests.java b/server/src/test/java/org/opensearch/ingest/PipelineProcessorTests.java index 9f8dda15eeb65..3708b5bc32955 100644 --- a/server/src/test/java/org/opensearch/ingest/PipelineProcessorTests.java +++ b/server/src/test/java/org/opensearch/ingest/PipelineProcessorTests.java @@ -32,6 +32,7 @@ package org.opensearch.ingest; import org.opensearch.OpenSearchException; +import org.opensearch.common.metrics.OperationStats; import org.opensearch.script.ScriptService; import org.opensearch.script.TemplateScript; import org.opensearch.test.OpenSearchTestCase; @@ -192,29 +193,29 @@ public void testPipelineProcessorWithPipelineChain() throws Exception { assertNotNull(ingestDocument.getSourceAndMetadata().get(key1)); // check the stats - IngestStats.Stats pipeline1Stats = pipeline1.getMetrics().createStats(); - IngestStats.Stats pipeline2Stats = pipeline2.getMetrics().createStats(); - IngestStats.Stats pipeline3Stats = pipeline3.getMetrics().createStats(); + OperationStats pipeline1Stats = pipeline1.getMetrics().createStats(); + OperationStats pipeline2Stats = pipeline2.getMetrics().createStats(); + OperationStats pipeline3Stats = pipeline3.getMetrics().createStats(); // current - assertThat(pipeline1Stats.getIngestCurrent(), equalTo(0L)); - assertThat(pipeline2Stats.getIngestCurrent(), equalTo(0L)); - assertThat(pipeline3Stats.getIngestCurrent(), equalTo(0L)); + assertThat(pipeline1Stats.getCurrent(), equalTo(0L)); + assertThat(pipeline2Stats.getCurrent(), equalTo(0L)); + assertThat(pipeline3Stats.getCurrent(), equalTo(0L)); // count - assertThat(pipeline1Stats.getIngestCount(), equalTo(1L)); - assertThat(pipeline2Stats.getIngestCount(), equalTo(1L)); - assertThat(pipeline3Stats.getIngestCount(), equalTo(1L)); + assertThat(pipeline1Stats.getCount(), equalTo(1L)); + assertThat(pipeline2Stats.getCount(), equalTo(1L)); + assertThat(pipeline3Stats.getCount(), equalTo(1L)); // time - assertThat(pipeline1Stats.getIngestTimeInMillis(), equalTo(0L)); - assertThat(pipeline2Stats.getIngestTimeInMillis(), equalTo(3L)); - assertThat(pipeline3Stats.getIngestTimeInMillis(), equalTo(2L)); + assertThat(pipeline1Stats.getTotalTimeInMillis(), equalTo(0L)); + assertThat(pipeline2Stats.getTotalTimeInMillis(), equalTo(3L)); + assertThat(pipeline3Stats.getTotalTimeInMillis(), equalTo(2L)); // failure - assertThat(pipeline1Stats.getIngestFailedCount(), equalTo(0L)); - assertThat(pipeline2Stats.getIngestFailedCount(), equalTo(0L)); - assertThat(pipeline3Stats.getIngestFailedCount(), equalTo(1L)); + assertThat(pipeline1Stats.getFailedCount(), equalTo(0L)); + assertThat(pipeline2Stats.getFailedCount(), equalTo(0L)); + assertThat(pipeline3Stats.getFailedCount(), equalTo(1L)); } public void testIngestPipelineMetadata() { diff --git a/server/src/test/java/org/opensearch/search/pipeline/SearchPipelineServiceTests.java b/server/src/test/java/org/opensearch/search/pipeline/SearchPipelineServiceTests.java index d49d9fd41031c..219dddff40b35 100644 --- a/server/src/test/java/org/opensearch/search/pipeline/SearchPipelineServiceTests.java +++ b/server/src/test/java/org/opensearch/search/pipeline/SearchPipelineServiceTests.java @@ -30,6 +30,7 @@ import org.opensearch.cluster.service.ClusterService; import org.opensearch.common.bytes.BytesArray; import org.opensearch.common.io.stream.NamedWriteableRegistry; +import org.opensearch.common.metrics.OperationStats; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.concurrent.OpenSearchExecutors; import org.opensearch.common.xcontent.XContentType; @@ -786,4 +787,126 @@ public void testExceptionOnResponseProcessing() throws Exception { // Exception thrown when processing response expectThrows(SearchPipelineProcessingException.class, () -> pipelinedRequest.transformResponse(response)); } + + public void testStats() throws Exception { + SearchRequestProcessor throwingRequestProcessor = new FakeRequestProcessor("throwing_request", "1", null, r -> { + throw new RuntimeException(); + }); + Map> requestProcessors = Map.of( + "successful_request", + (pf, t, f, c) -> new FakeRequestProcessor("successful_request", "2", null, r -> {}), + "throwing_request", + (pf, t, f, c) -> throwingRequestProcessor + ); + SearchResponseProcessor throwingResponseProcessor = new FakeResponseProcessor("throwing_response", "3", null, r -> { + throw new RuntimeException(); + }); + Map> responseProcessors = Map.of( + "successful_response", + (pf, t, f, c) -> new FakeResponseProcessor("successful_response", "4", null, r -> {}), + "throwing_response", + (pf, t, f, c) -> throwingResponseProcessor + ); + SearchPipelineService searchPipelineService = createWithProcessors(requestProcessors, responseProcessors); + + SearchPipelineMetadata metadata = new SearchPipelineMetadata( + Map.of( + "good_response_pipeline", + new PipelineConfiguration( + "good_response_pipeline", + new BytesArray("{\"response_processors\" : [ { \"successful_response\": {} } ] }"), + XContentType.JSON + ), + "bad_response_pipeline", + new PipelineConfiguration( + "bad_response_pipeline", + new BytesArray("{\"response_processors\" : [ { \"throwing_response\": {} } ] }"), + XContentType.JSON + ), + "good_request_pipeline", + new PipelineConfiguration( + "good_request_pipeline", + new BytesArray("{\"request_processors\" : [ { \"successful_request\": {} } ] }"), + XContentType.JSON + ), + "bad_request_pipeline", + new PipelineConfiguration( + "bad_request_pipeline", + new BytesArray("{\"request_processors\" : [ { \"throwing_request\": {} } ] }"), + XContentType.JSON + ) + ) + ); + ClusterState clusterState = ClusterState.builder(new ClusterName("_name")).build(); + ClusterState previousState = clusterState; + clusterState = ClusterState.builder(clusterState) + .metadata(Metadata.builder().putCustom(SearchPipelineMetadata.TYPE, metadata)) + .build(); + searchPipelineService.applyClusterState(new ClusterChangedEvent("", clusterState, previousState)); + + SearchRequest request = new SearchRequest(); + SearchResponse response = new SearchResponse(null, null, 0, 0, 0, 0, null, null); + + searchPipelineService.resolvePipeline(request.pipeline("good_request_pipeline")).transformResponse(response); + expectThrows( + SearchPipelineProcessingException.class, + () -> searchPipelineService.resolvePipeline(request.pipeline("bad_request_pipeline")).transformResponse(response) + ); + searchPipelineService.resolvePipeline(request.pipeline("good_response_pipeline")).transformResponse(response); + expectThrows( + SearchPipelineProcessingException.class, + () -> searchPipelineService.resolvePipeline(request.pipeline("bad_response_pipeline")).transformResponse(response) + ); + + SearchPipelineStats stats = searchPipelineService.stats(); + assertPipelineStats(stats.getTotalRequestStats(), 2, 1); + assertPipelineStats(stats.getTotalResponseStats(), 2, 1); + for (SearchPipelineStats.PerPipelineStats perPipelineStats : stats.getPipelineStats()) { + SearchPipelineStats.PipelineDetailStats detailStats = stats.getPerPipelineProcessorStats() + .get(perPipelineStats.getPipelineId()); + switch (perPipelineStats.getPipelineId()) { + case "good_request_pipeline": + assertPipelineStats(perPipelineStats.getRequestStats(), 1, 0); + assertPipelineStats(perPipelineStats.getResponseStats(), 0, 0); + assertEquals(1, detailStats.requestProcessorStats().size()); + assertEquals(0, detailStats.responseProcessorStats().size()); + assertEquals("successful_request:2", detailStats.requestProcessorStats().get(0).getProcessorName()); + assertEquals("successful_request", detailStats.requestProcessorStats().get(0).getProcessorType()); + assertPipelineStats(detailStats.requestProcessorStats().get(0).getStats(), 1, 0); + break; + case "bad_request_pipeline": + assertPipelineStats(perPipelineStats.getRequestStats(), 1, 1); + assertPipelineStats(perPipelineStats.getResponseStats(), 0, 0); + assertEquals(1, detailStats.requestProcessorStats().size()); + assertEquals(0, detailStats.responseProcessorStats().size()); + assertEquals("throwing_request:1", detailStats.requestProcessorStats().get(0).getProcessorName()); + assertEquals("throwing_request", detailStats.requestProcessorStats().get(0).getProcessorType()); + assertPipelineStats(detailStats.requestProcessorStats().get(0).getStats(), 1, 1); + break; + case "good_response_pipeline": + assertPipelineStats(perPipelineStats.getRequestStats(), 0, 0); + assertPipelineStats(perPipelineStats.getResponseStats(), 1, 0); + assertEquals(0, detailStats.requestProcessorStats().size()); + assertEquals(1, detailStats.responseProcessorStats().size()); + assertEquals("successful_response:4", detailStats.responseProcessorStats().get(0).getProcessorName()); + assertEquals("successful_response", detailStats.responseProcessorStats().get(0).getProcessorType()); + assertPipelineStats(detailStats.responseProcessorStats().get(0).getStats(), 1, 0); + break; + case "bad_response_pipeline": + assertPipelineStats(perPipelineStats.getRequestStats(), 0, 0); + assertPipelineStats(perPipelineStats.getResponseStats(), 1, 1); + assertEquals(0, detailStats.requestProcessorStats().size()); + assertEquals(1, detailStats.responseProcessorStats().size()); + assertEquals("throwing_response:3", detailStats.responseProcessorStats().get(0).getProcessorName()); + assertEquals("throwing_response", detailStats.responseProcessorStats().get(0).getProcessorType()); + assertPipelineStats(detailStats.responseProcessorStats().get(0).getStats(), 1, 1); + break; + } + } + } + + private static void assertPipelineStats(OperationStats stats, long count, long failed) { + assertEquals(stats.getCount(), count); + assertEquals(stats.getFailedCount(), failed); + } } diff --git a/server/src/test/java/org/opensearch/search/pipeline/SearchPipelineStatsTests.java b/server/src/test/java/org/opensearch/search/pipeline/SearchPipelineStatsTests.java new file mode 100644 index 0000000000000..dac41f0db4e00 --- /dev/null +++ b/server/src/test/java/org/opensearch/search/pipeline/SearchPipelineStatsTests.java @@ -0,0 +1,185 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.search.pipeline; + +import org.opensearch.common.bytes.BytesReference; +import org.opensearch.common.io.stream.BytesStreamOutput; +import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.common.metrics.OperationStats; +import org.opensearch.common.xcontent.XContentHelper; +import org.opensearch.common.xcontent.XContentType; +import org.opensearch.common.xcontent.json.JsonXContent; +import org.opensearch.core.xcontent.DeprecationHandler; +import org.opensearch.core.xcontent.MediaType; +import org.opensearch.core.xcontent.XContentBuilder; +import org.opensearch.core.xcontent.XContentParser; +import org.opensearch.test.OpenSearchTestCase; + +import java.io.IOException; +import java.util.List; +import java.util.Map; + +public class SearchPipelineStatsTests extends OpenSearchTestCase { + public void testSerializationRoundtrip() throws IOException { + SearchPipelineStats stats = createStats(); + SearchPipelineStats deserialized; + try (BytesStreamOutput bytesStreamOutput = new BytesStreamOutput()) { + stats.writeTo(bytesStreamOutput); + try (StreamInput bytesStreamInput = bytesStreamOutput.bytes().streamInput()) { + deserialized = new SearchPipelineStats(bytesStreamInput); + } + } + assertEquals(stats, deserialized); + } + + private static SearchPipelineStats createStats() { + return new SearchPipelineStats( + new OperationStats(1, 2, 3, 4), + new OperationStats(5, 6, 7, 8), + List.of( + new SearchPipelineStats.PerPipelineStats("p1", new OperationStats(9, 10, 11, 12), new OperationStats(13, 14, 15, 16)), + new SearchPipelineStats.PerPipelineStats("p2", new OperationStats(17, 18, 19, 20), new OperationStats(21, 22, 23, 24)) + + ), + Map.of( + "p1", + new SearchPipelineStats.PipelineDetailStats( + List.of(new SearchPipelineStats.ProcessorStats("req1:a", "req1", new OperationStats(25, 26, 27, 28))), + List.of(new SearchPipelineStats.ProcessorStats("rsp1:a", "rsp1", new OperationStats(29, 30, 31, 32))) + ), + "p2", + new SearchPipelineStats.PipelineDetailStats( + List.of( + new SearchPipelineStats.ProcessorStats("req1:a", "req1", new OperationStats(33, 34, 35, 36)), + new SearchPipelineStats.ProcessorStats("req2", "req2", new OperationStats(37, 38, 39, 40)) + ), + List.of() + ) + ) + ); + } + + public void testToXContent() throws IOException { + XContentBuilder actualBuilder = XContentBuilder.builder(JsonXContent.jsonXContent); + actualBuilder.startObject(); + createStats().toXContent(actualBuilder, null); + actualBuilder.endObject(); + + String expected = "{" + + " \"search_pipeline\" : {" + + " \"total_request\" : {" + + " \"count\" : 1," + + " \"time_in_millis\" : 2," + + " \"current\" : 3," + + " \"failed\" : 4" + + " }," + + " \"total_response\" : {" + + " \"count\" : 5," + + " \"time_in_millis\" : 6," + + " \"current\" : 7," + + " \"failed\" : 8" + + " }," + + " \"pipelines\" : {" + + " \"p1\" : {" + + " \"request\" : {" + + " \"count\" : 9," + + " \"time_in_millis\" : 10," + + " \"current\" : 11," + + " \"failed\" : 12" + + " }," + + " \"response\" : {" + + " \"count\" : 13," + + " \"time_in_millis\" : 14," + + " \"current\" : 15," + + " \"failed\" : 16" + + " }," + + " \"request_processors\" : [" + + " {" + + " \"req1:a\" : {" + + " \"type\" : \"req1\"," + + " \"stats\" : {" + + " \"count\" : 25," + + " \"time_in_millis\" : 26," + + " \"current\" : 27," + + " \"failed\" : 28" + + " }" + + " }" + + " }" + + " ]," + + " \"response_processors\" : [" + + " {" + + " \"rsp1:a\" : {" + + " \"type\" : \"rsp1\"," + + " \"stats\" : {" + + " \"count\" : 29," + + " \"time_in_millis\" : 30," + + " \"current\" : 31," + + " \"failed\" : 32" + + " }" + + " }" + + " }" + + " ]" + + " }," + + " \"p2\" : {" + + " \"request\" : {" + + " \"count\" : 17," + + " \"time_in_millis\" : 18," + + " \"current\" : 19," + + " \"failed\" : 20" + + " }," + + " \"response\" : {" + + " \"count\" : 21," + + " \"time_in_millis\" : 22," + + " \"current\" : 23," + + " \"failed\" : 24" + + " }," + + " \"request_processors\" : [" + + " {" + + " \"req1:a\" : {" + + " \"type\" : \"req1\"," + + " \"stats\" : {" + + " \"count\" : 33," + + " \"time_in_millis\" : 34," + + " \"current\" : 35," + + " \"failed\" : 36" + + " }" + + " }" + + " }," + + " {" + + " \"req2\" : {" + + " \"type\" : \"req2\"," + + " \"stats\" : {" + + " \"count\" : 37," + + " \"time_in_millis\" : 38," + + " \"current\" : 39," + + " \"failed\" : 40" + + " }" + + " }" + + " }" + + " ]," + + " \"response_processors\" : [ ]" + + " }" + + " }" + + " }" + + "}"; + + XContentParser expectedParser = JsonXContent.jsonXContent.createParser( + this.xContentRegistry(), + DeprecationHandler.THROW_UNSUPPORTED_OPERATION, + expected + ); + XContentBuilder expectedBuilder = XContentBuilder.builder(JsonXContent.jsonXContent); + expectedBuilder.generator().copyCurrentStructure(expectedParser); + + assertEquals( + XContentHelper.convertToMap(BytesReference.bytes(expectedBuilder), false, (MediaType) XContentType.JSON), + XContentHelper.convertToMap(BytesReference.bytes(actualBuilder), false, (MediaType) XContentType.JSON) + ); + } +} diff --git a/test/framework/src/main/java/org/opensearch/cluster/MockInternalClusterInfoService.java b/test/framework/src/main/java/org/opensearch/cluster/MockInternalClusterInfoService.java index cf5f6613c3ea1..6634d1b4dbafc 100644 --- a/test/framework/src/main/java/org/opensearch/cluster/MockInternalClusterInfoService.java +++ b/test/framework/src/main/java/org/opensearch/cluster/MockInternalClusterInfoService.java @@ -119,7 +119,8 @@ List adjustNodesStats(List nodesStats) { nodeStats.getClusterManagerThrottlingStats(), nodeStats.getWeightedRoutingStats(), nodeStats.getFileCacheStats(), - nodeStats.getTaskCancellationStats() + nodeStats.getTaskCancellationStats(), + nodeStats.getSearchPipelineStats() ); }).collect(Collectors.toList()); } 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 38617d09b703d..49d8b64bc71cd 100644 --- a/test/framework/src/main/java/org/opensearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/opensearch/test/InternalTestCluster.java @@ -2685,6 +2685,7 @@ public void ensureEstimatedStats() { false, false, false, + false, false ); assertThat( From b7cace575c84d94b49c02cd8328835d3f8b1a0d0 Mon Sep 17 00:00:00 2001 From: Ketan Verma <9292653+ketanv3@users.noreply.github.com> Date: Thu, 29 Jun 2023 03:44:44 +0530 Subject: [PATCH 085/109] Self-organizing hash table to improve the performance of bucket aggregations (#7652) * Add self-organizing hash table to improve the performance of bucket aggregations Signed-off-by: Ketan Verma * Updated approach: PSL, fingerprint and recency information are embedded in the hash table itself Signed-off-by: Ketan Verma * Updated tests and added microbenchmarks Signed-off-by: Ketan Verma * Renamed FastLongHash to ReorganizingLongHash and updated the default initial capacity Signed-off-by: Ketan Verma --------- Signed-off-by: Ketan Verma --- CHANGELOG.md | 1 + .../common/util/LongHashBenchmark.java | 425 ++++++++++++++++++ .../common/util/ReorganizingLongHash.java | 307 +++++++++++++ .../bucket/terms/LongKeyedBucketOrds.java | 6 +- .../util/ReorganizingLongHashTests.java | 146 ++++++ 5 files changed, 882 insertions(+), 3 deletions(-) create mode 100644 benchmarks/src/main/java/org/opensearch/common/util/LongHashBenchmark.java create mode 100644 server/src/main/java/org/opensearch/common/util/ReorganizingLongHash.java create mode 100644 server/src/test/java/org/opensearch/common/util/ReorganizingLongHashTests.java diff --git a/CHANGELOG.md b/CHANGELOG.md index 109401a46e5b7..5b3b8e251f205 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -136,6 +136,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Compress and cache cluster state during validate join request ([#7321](https://github.com/opensearch-project/OpenSearch/pull/7321)) - [Snapshot Interop] Add Changes in Create Snapshot Flow for remote store interoperability. ([#7118](https://github.com/opensearch-project/OpenSearch/pull/7118)) - Allow insecure string settings to warn-log usage and advise to migration of a newer secure variant ([#5496](https://github.com/opensearch-project/OpenSearch/pull/5496)) +- Add self-organizing hash table to improve the performance of bucket aggregations ([#7652](https://github.com/opensearch-project/OpenSearch/pull/7652)) ### Deprecated diff --git a/benchmarks/src/main/java/org/opensearch/common/util/LongHashBenchmark.java b/benchmarks/src/main/java/org/opensearch/common/util/LongHashBenchmark.java new file mode 100644 index 0000000000000..fa75dd2c91f5a --- /dev/null +++ b/benchmarks/src/main/java/org/opensearch/common/util/LongHashBenchmark.java @@ -0,0 +1,425 @@ +/* + * 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.common.util; + +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; +import org.opensearch.common.lease.Releasable; + +import java.util.Random; +import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; + +@Fork(value = 3) +@Warmup(iterations = 1, time = 4) +@Measurement(iterations = 3, time = 2) +@BenchmarkMode(Mode.AverageTime) +@OutputTimeUnit(TimeUnit.MILLISECONDS) +public class LongHashBenchmark { + + @Benchmark + public void add(Blackhole bh, HashTableOptions tableOpts, WorkloadOptions workloadOpts) { + try (HashTable table = tableOpts.get(); WorkloadIterator iter = workloadOpts.iter()) { + while (iter.hasNext()) { + bh.consume(table.add(iter.next())); + } + } + } + + /** + * Creates a hash table with varying parameters. + */ + @State(Scope.Benchmark) + public static class HashTableOptions { + + @Param({ "LongHash", "ReorganizingLongHash" }) + public String type; + + @Param({ "1" }) + public long initialCapacity; + + @Param({ "0.6" }) + public float loadFactor; + + private Supplier supplier; + + @Setup + public void setup() { + switch (type) { + case "LongHash": + supplier = this::newLongHash; + break; + case "ReorganizingLongHash": + supplier = this::newReorganizingLongHash; + break; + default: + throw new IllegalArgumentException("invalid hash table type: " + type); + } + } + + public HashTable get() { + return supplier.get(); + } + + private HashTable newLongHash() { + return new HashTable() { + private final LongHash table = new LongHash(initialCapacity, loadFactor, BigArrays.NON_RECYCLING_INSTANCE); + + @Override + public long add(long key) { + return table.add(key); + } + + @Override + public void close() { + table.close(); + } + }; + } + + private HashTable newReorganizingLongHash() { + return new HashTable() { + private final ReorganizingLongHash table = new ReorganizingLongHash( + initialCapacity, + loadFactor, + BigArrays.NON_RECYCLING_INSTANCE + ); + + @Override + public long add(long key) { + return table.add(key); + } + + @Override + public void close() { + table.close(); + } + }; + } + } + + /** + * Creates a workload with varying parameters. + */ + @State(Scope.Benchmark) + public static class WorkloadOptions { + public static final int NUM_HITS = 20_000_000; + + /** + * Repeat the experiment with growing number of keys. + * These values are generated with an exponential growth pattern such that: + * value = ceil(previous_value * random_float_between(1.0, 1.14)) + */ + @Param({ + "1", + "2", + "3", + "4", + "5", + "6", + "7", + "8", + "9", + "10", + "11", + "13", + "15", + "17", + "18", + "19", + "20", + "21", + "23", + "26", + "27", + "30", + "32", + "35", + "41", + "45", + "50", + "53", + "54", + "55", + "57", + "63", + "64", + "69", + "74", + "80", + "84", + "91", + "98", + "101", + "111", + "114", + "124", + "128", + "139", + "148", + "161", + "162", + "176", + "190", + "204", + "216", + "240", + "257", + "269", + "291", + "302", + "308", + "327", + "341", + "374", + "402", + "412", + "438", + "443", + "488", + "505", + "558", + "612", + "621", + "623", + "627", + "642", + "717", + "765", + "787", + "817", + "915", + "962", + "1011", + "1083", + "1163", + "1237", + "1301", + "1424", + "1541", + "1716", + "1805", + "1817", + "1934", + "2024", + "2238", + "2281", + "2319", + "2527", + "2583", + "2639", + "2662", + "2692", + "2991", + "3201", + "3215", + "3517", + "3681", + "3710", + "4038", + "4060", + "4199", + "4509", + "4855", + "5204", + "5624", + "6217", + "6891", + "7569", + "8169", + "8929", + "9153", + "10005", + "10624", + "10931", + "12070", + "12370", + "13694", + "14227", + "15925", + "17295", + "17376", + "18522", + "19200", + "20108", + "21496", + "23427", + "24224", + "26759", + "29199", + "29897", + "32353", + "33104", + "36523", + "38480", + "38958", + "40020", + "44745", + "45396", + "47916", + "49745", + "49968", + "52231", + "53606" }) + public int size; + + @Param({ "correlated", "uncorrelated", "distinct" }) + public String dataset; + + private WorkloadIterator iterator; + + @Setup + public void setup() { + switch (dataset) { + case "correlated": + iterator = newCorrelatedWorkload(); + break; + case "uncorrelated": + iterator = newUncorrelatedWorkload(); + break; + case "distinct": + iterator = newDistinctWorkload(); + break; + default: + throw new IllegalArgumentException("invalid dataset: " + dataset); + } + } + + public WorkloadIterator iter() { + return iterator; + } + + /** + * Simulates monotonically increasing timestamp data with multiple hits mapping to the same key. + */ + private WorkloadIterator newCorrelatedWorkload() { + assert NUM_HITS >= size : "ensure hits >= size so that each key is used at least once"; + + final long[] data = new long[size]; + for (int i = 0; i < data.length; i++) { + data[i] = 1420070400000L + 3600000L * i; + } + + return new WorkloadIterator() { + private int count = 0; + private int index = 0; + private int remaining = NUM_HITS / data.length; + + @Override + public boolean hasNext() { + return count < NUM_HITS; + } + + @Override + public long next() { + if (--remaining <= 0) { + index = (index + 1) % data.length; + remaining = NUM_HITS / data.length; + } + count++; + return data[index]; + } + + @Override + public void reset() { + count = 0; + index = 0; + remaining = NUM_HITS / data.length; + } + }; + } + + /** + * Simulates uncorrelated data (such as travel distance / fare amount). + */ + private WorkloadIterator newUncorrelatedWorkload() { + assert NUM_HITS >= size : "ensure hits >= size so that each key is used at least once"; + + final Random random = new Random(0); // fixed seed for reproducible results + final long[] data = new long[size]; + for (int i = 0; i < data.length; i++) { + data[i] = Double.doubleToLongBits(20.0 + 80 * random.nextDouble()); + } + + return new WorkloadIterator() { + private int count = 0; + private int index = 0; + + @Override + public boolean hasNext() { + return count < NUM_HITS; + } + + @Override + public long next() { + count++; + index = (index + 1) % data.length; + return data[index]; + } + + @Override + public void reset() { + count = 0; + index = 0; + } + }; + } + + /** + * Simulates workload with high cardinality, i.e., each hit mapping to a different key. + */ + private WorkloadIterator newDistinctWorkload() { + return new WorkloadIterator() { + private int count = 0; + + @Override + public boolean hasNext() { + return count < size; + } + + @Override + public long next() { + return count++; + } + + @Override + public void reset() { + count = 0; + } + }; + } + } + + private interface HashTable extends Releasable { + long add(long key); + } + + private interface WorkloadIterator extends Releasable { + boolean hasNext(); + + long next(); + + void reset(); + + @Override + default void close() { + reset(); + } + } +} diff --git a/server/src/main/java/org/opensearch/common/util/ReorganizingLongHash.java b/server/src/main/java/org/opensearch/common/util/ReorganizingLongHash.java new file mode 100644 index 0000000000000..5789b47423c1d --- /dev/null +++ b/server/src/main/java/org/opensearch/common/util/ReorganizingLongHash.java @@ -0,0 +1,307 @@ +/* + * 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.common.util; + +import org.apache.lucene.util.hppc.BitMixer; +import org.opensearch.common.lease.Releasable; + +/** + * Specialized hash table implementation that maps a (primitive) long to long. + * + *

+ * It organizes itself by moving keys around dynamically in order to reduce the + * longest probe sequence length (PSL), which makes lookups faster as keys are likely to + * be found in the same CPU cache line. It also optimizes lookups for recently added keys, + * making it useful for aggregations where keys are correlated across consecutive hits. + * + *

+ * This class is not thread-safe. + * + * @opensearch.internal + */ +public class ReorganizingLongHash implements Releasable { + private static final long MAX_CAPACITY = 1L << 32; + private static final long DEFAULT_INITIAL_CAPACITY = 32; + private static final float DEFAULT_LOAD_FACTOR = 0.6f; + + /** + * Maximum load factor after which the capacity is doubled. + */ + private final float loadFactor; + + /** + * Utility class to allocate recyclable arrays. + */ + private final BigArrays bigArrays; + + /** + * Current capacity of the hash table. This must be a power of two so that the hash table slot + * can be identified quickly using bitmasks, thus avoiding expensive modulo or integer division. + */ + private long capacity; + + /** + * Bitmask to identify the hash table slot from a key's hash. + */ + private long mask; + + /** + * Size threshold after which the hash table needs to be doubled in capacity. + */ + private long grow; + + /** + * Current size of the hash table. + */ + private long size; + + /** + * Underlying array to store the hash table values. + * + *

+ * Each hash table value (64-bit) uses the following byte packing strategy: + *

+     * |=========|===============|================|================================|
+     * | Discard | PSL           | Fingerprint    | Ordinal                        |
+     * |    -    |---------------|----------------|--------------------------------|
+     * | 1 bit   | 15 bits       | 16 bits        | 32 bits                        |
+     * |=========|===============|================|================================|
+     * 
+ * + *

+ * This allows us to encode and manipulate additional information in the hash table + * itself without having to look elsewhere in the memory, which is much slower. + * + *

+ * Terminology: table[index] = value = (discard | psl | fingerprint | ordinal) + */ + private LongArray table; + + /** + * Underlying array to store the keys. + * + *

+ * Terminology: keys[ordinal] = key + */ + private LongArray keys; + + /** + * Bitmasks to manipulate the hash table values. + */ + private static final long MASK_ORDINAL = 0x00000000FFFFFFFFL; // extract ordinal + private static final long MASK_FINGERPRINT = 0x0000FFFF00000000L; // extract fingerprint + private static final long MASK_PSL = 0x7FFF000000000000L; // extract PSL + private static final long INCR_PSL = 0x0001000000000000L; // increment PSL by one + + public ReorganizingLongHash(final BigArrays bigArrays) { + this(DEFAULT_INITIAL_CAPACITY, DEFAULT_LOAD_FACTOR, bigArrays); + } + + public ReorganizingLongHash(final long initialCapacity, final float loadFactor, final BigArrays bigArrays) { + assert initialCapacity > 0 : "initial capacity must be greater than 0"; + assert loadFactor > 0 && loadFactor < 1 : "load factor must be between 0 and 1"; + + this.bigArrays = bigArrays; + this.loadFactor = loadFactor; + + capacity = nextPowerOfTwo((long) (initialCapacity / loadFactor)); + mask = capacity - 1; + grow = (long) (capacity * loadFactor); + size = 0; + + table = bigArrays.newLongArray(capacity, false); + table.fill(0, capacity, -1); // -1 represents an empty slot + keys = bigArrays.newLongArray(initialCapacity, false); + } + + /** + * Adds the given key to the hash table and returns its ordinal. + * If the key exists already, it returns (-1 - ordinal). + */ + public long add(final long key) { + final long ordinal = find(key); + if (ordinal != -1) { + return -1 - ordinal; + } + + if (size >= grow) { + grow(); + } + + return insert(key); + } + + /** + * Returns the key associated with the given ordinal. + * The result is undefined for an unused ordinal. + */ + public long get(final long ordinal) { + return keys.get(ordinal); + } + + /** + * Returns the ordinal associated with the given key, or -1 if the key doesn't exist. + * + *

+ * Using the 64-bit hash value, up to 32 least significant bits (LSB) are used to identify the + * home slot in the hash table, and an additional 16 bits are used to identify the fingerprint. + * The fingerprint further increases the entropy and reduces the number of false lookups in the + * keys' table during equality checks, which is expensive due to an uncorrelated memory lookup. + * + *

+ * Total entropy bits = 16 + log2(capacity) + * + *

+ * Linear probing starts from the home slot, until a match or an empty slot is found. + * Values are first checked using their fingerprint (to reduce false positives), then verified + * in the keys' table using an equality check. + */ + public long find(final long key) { + final long hash = hash(key); + final long fingerprint = hash & MASK_FINGERPRINT; + + for (long idx = hash & mask, value, ordinal;; idx = (idx + 1) & mask) { + if ((value = table.get(idx)) == -1) { + return -1; + } else if (((value & MASK_FINGERPRINT) == fingerprint) && (keys.get((ordinal = (value & MASK_ORDINAL))) == key)) { + return ordinal; + } + } + } + + /** + * Returns the number of mappings in this hash table. + */ + public long size() { + return size; + } + + /** + * Inserts the given key in the hash table and returns its ordinal. + * + *

+ * Inspired by Robin Hood Hashing (RHH): if the PSL for the existing value is less than the PSL + * for the value being inserted, swap the two values and keep going. Values that were inserted + * early and thus "lucked out" on their PSLs will gradually be moved away from their preferred + * slot as new values come in that could make better use of that place in the table. It evens out + * the PSLs across the board and reduces the longest PSL dramatically. + * + *

+ * A lower variance is better because, with modern CPU architectures, a PSL of 1 isn't much + * faster than a PSL of 3, because the main cost is fetching the cache line. The ideal hash + * table layout is the one where all values have equal PSLs, and that PSL fits within one cache line. + * + *

+ * The expected longest PSL for a full table: log(N) + * + *

+ * Our implementation has a slight variation on top of it: by loosening the guarantees provided + * by RHH, we can improve the performance on correlated lookups (such as aggregating on repeated + * timestamps) by moving the "recent" keys closer to their home slot, and eventually converging + * to the ideal hash table layout defined by RHH. + */ + private long insert(final long key) { + final long hash = hash(key); + final long fingerprint = hash & MASK_FINGERPRINT; + + // The ideal home slot for the given key. + long idx = hash & mask; + + // The value yet to find an empty slot (candidate). + long value = fingerprint | size; + + // The existing value at idx. + long existingValue; + + // Always set the newly inserted key at its ideal home slot, even if it doesn't conform + // to the RHH scheme (yet). This will ensure subsequent correlated lookups are fast due + // to no additional probing. When another insertion causes this value to be displaced, it + // will eventually be placed at an appropriate location defined by the RHH scheme. + if ((value = table.set(idx, value)) == -1) { + // The ideal home slot was already empty; append the key and return early. + return append(key); + } + + // Find an alternative slot for the displaced value such that the longest PSL is minimized. + do { + idx = (idx + 1) & mask; + value += INCR_PSL; + + if ((existingValue = table.get(idx)) == -1) { + // Empty slot; insert the candidate value here. + table.set(idx, value); + return append(key); + } else if ((existingValue & MASK_PSL) <= (value & MASK_PSL)) { + // Existing value is "richer" than the candidate value at this index; + // swap and find an alternative slot for the displaced value. + // In the case of a tie, the candidate value (i.e. the recent value) is chosen as + // the winner and kept closer to its ideal home slot in order to speed up + // correlated lookups. + value = table.set(idx, value); + } + } while (true); + } + + /** + * Appends the key in the keys' table. + */ + private long append(final long key) { + keys = bigArrays.grow(keys, size + 1); + keys.set(size, key); + return size++; + } + + /** + * Returns the hash for the given key. + * Visible for unit-tests. + */ + long hash(final long key) { + return BitMixer.mix64(key); + } + + /** + * Returns the underlying hash table. + * Visible for unit-tests. + */ + LongArray getTable() { + return table; + } + + /** + * Grows the hash table by doubling its capacity and reinserting the keys. + */ + private void grow() { + // Ensure that the hash table doesn't grow too large. + // This implicitly also ensures that the ordinals are no larger than 2^32, thus, + // preventing them from polluting other bits (PSL/fingerprint) in the hash table values. + assert capacity < MAX_CAPACITY : "hash table already at the max capacity"; + + final long oldSize = size; + capacity <<= 1; + mask = capacity - 1; + size = 0; + grow = (long) (capacity * loadFactor); + table = bigArrays.resize(table, capacity); + table.fill(0, capacity, -1); + + for (long ordinal = 0; ordinal < oldSize; ordinal++) { + insert(keys.get(ordinal)); + } + } + + @Override + public void close() { + table.close(); + keys.close(); + } + + private static long nextPowerOfTwo(final long value) { + return Math.max(1, Long.highestOneBit(value - 1) << 1); + } +} diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/LongKeyedBucketOrds.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/LongKeyedBucketOrds.java index 6b511587cc271..a55b7d6bc154e 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/LongKeyedBucketOrds.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/LongKeyedBucketOrds.java @@ -33,7 +33,7 @@ package org.opensearch.search.aggregations.bucket.terms; import org.opensearch.common.util.BigArrays; -import org.opensearch.common.util.LongHash; +import org.opensearch.common.util.ReorganizingLongHash; import org.opensearch.common.util.LongLongHash; import org.opensearch.common.lease.Releasable; import org.opensearch.search.aggregations.CardinalityUpperBound; @@ -148,10 +148,10 @@ public long value() { * @opensearch.internal */ public static class FromSingle extends LongKeyedBucketOrds { - private final LongHash ords; + private final ReorganizingLongHash ords; public FromSingle(BigArrays bigArrays) { - ords = new LongHash(1, bigArrays); + ords = new ReorganizingLongHash(bigArrays); } @Override diff --git a/server/src/test/java/org/opensearch/common/util/ReorganizingLongHashTests.java b/server/src/test/java/org/opensearch/common/util/ReorganizingLongHashTests.java new file mode 100644 index 0000000000000..259eab6c624bd --- /dev/null +++ b/server/src/test/java/org/opensearch/common/util/ReorganizingLongHashTests.java @@ -0,0 +1,146 @@ +/* + * 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.common.util; + +import org.opensearch.test.OpenSearchTestCase; + +import java.util.HashMap; +import java.util.Map; + +public class ReorganizingLongHashTests extends OpenSearchTestCase { + + public void testFuzzy() { + Map reference = new HashMap<>(); + + try ( + ReorganizingLongHash h = new ReorganizingLongHash( + randomIntBetween(1, 100), // random capacity + 0.6f + randomFloat() * 0.39f, // random load factor to verify collision resolution + BigArrays.NON_RECYCLING_INSTANCE + ) + ) { + // Verify the behaviour of "add" and "find". + for (int i = 0; i < (1 << 20); i++) { + long key = randomLong() % (1 << 12); // roughly ~4% unique keys + if (reference.containsKey(key)) { + long expectedOrdinal = reference.get(key); + assertEquals(-1 - expectedOrdinal, h.add(key)); + assertEquals(expectedOrdinal, h.find(key)); + } else { + assertEquals(-1, h.find(key)); + reference.put(key, (long) reference.size()); + assertEquals((long) reference.get(key), h.add(key)); + } + } + + // Verify the behaviour of "get". + for (Map.Entry entry : reference.entrySet()) { + assertEquals((long) entry.getKey(), h.get(entry.getValue())); + } + + // Verify the behaviour of "size". + assertEquals(reference.size(), h.size()); + + // Verify the calculation of PSLs. + final long capacity = h.getTable().size(); + final long mask = capacity - 1; + for (long idx = 0; idx < h.getTable().size(); idx++) { + final long value = h.getTable().get(idx); + if (value != -1) { + final long homeIdx = h.hash(h.get((int) value)) & mask; + assertEquals((capacity + idx - homeIdx) & mask, value >>> 48); + } + } + } + } + + public void testRearrangement() { + try (ReorganizingLongHash h = new ReorganizingLongHash(4, 0.6f, BigArrays.NON_RECYCLING_INSTANCE) { + /** + * Overriding with an "identity" hash function to make it easier to reason about the placement + * of values in the hash table. The backing array of the hash table will have a size (8), + * i.e. nextPowerOfTwo(initialCapacity/loadFactor), so the bitmask will be (7). + * The ideal home slot of a key can then be defined as: (hash(key) & mask) = (key & 7). + */ + @Override + long hash(long key) { + return key; + } + }) { + /* + * Add key=0, hash=0, home_slot=0 + * + * Before: empty slot. + * â–¼ + * [ _ _ _ _ _ _ _ _ ] + * + * After: inserted [ordinal=0, psl=0] at the empty slot. + * [ 0 _ _ _ _ _ _ _ ] + */ + h.add(0); + assertEquals(encodeValue(0, 0, 0), h.getTable().get(0)); + + /* + * Add key=8, hash=8, home_slot=0 + * + * Before: occupied slot. + * â–¼ + * [ 0 _ _ _ _ _ _ _ ] + * + * After: inserted [ordinal=1, psl=0] at the existing slot, displaced [ordinal=0, psl=0], + * and re-inserted it at the next empty slot as [ordinal=0, psl=1]. + * [ 1 0 _ _ _ _ _ _ ] + */ + h.add(8); + assertEquals(encodeValue(0, 0, 1), h.getTable().get(0)); + assertEquals(encodeValue(1, 0, 0), h.getTable().get(1)); + + /* + * Add key=1, hash=1, home_slot=1 + * + * Before: occupied slot. + * â–¼ + * [ 1 0 _ _ _ _ _ _ ] + * + * After: inserted [ordinal=2, psl=0] at the existing slot, displaced [ordinal=0, psl=1], + * and re-inserted it at the next empty slot as [ordinal=0, psl=2]. + * [ 1 2 0 _ _ _ _ _ ] + */ + h.add(1); + assertEquals(encodeValue(0, 0, 1), h.getTable().get(0)); + assertEquals(encodeValue(0, 0, 2), h.getTable().get(1)); + assertEquals(encodeValue(2, 0, 0), h.getTable().get(2)); + + /* + * Add key=16, hash=16, home_slot=0 + * + * Before: occupied slot. + * â–¼ + * [ 1 2 0 _ _ _ _ _ ] + * + * After: inserted [ordinal=3, psl=0] at the existing slot, displaced [ordinal=1, psl=0] + * and re-inserted it at the next best slot. Repeated this for other displaced values + * until everything found an empty slot. + * [ 3 1 0 2 _ _ _ _ ] + */ + h.add(16); + assertEquals(encodeValue(0, 0, 3), h.getTable().get(0)); + assertEquals(encodeValue(1, 0, 1), h.getTable().get(1)); + assertEquals(encodeValue(2, 0, 0), h.getTable().get(2)); + assertEquals(encodeValue(2, 0, 2), h.getTable().get(3)); + } + } + + private static long encodeValue(long psl, long fingerprint, long ordinal) { + assert psl < (1L << 15); + assert fingerprint < (1L << 16); + assert ordinal < (1L << 32); + return (psl << 48) | (fingerprint << 32) | ordinal; + } +} From 74788c3f45895a90d030b2434539beac7a1b1723 Mon Sep 17 00:00:00 2001 From: Thomas Farr Date: Thu, 29 Jun 2023 10:40:55 +1200 Subject: [PATCH 086/109] Bump com.google.api-client:google-api-client from 1.34.0 to 2.2.0 in /plugins/repository-gcs (#8276) * Bump com.google.api-client:google-api-client from 1.34.0 to 2.2.0 in /plugins/repository-gcs Signed-off-by: Thomas Farr * Bump google-http-client and google-oauth-client Signed-off-by: Thomas --------- Signed-off-by: Thomas Farr Signed-off-by: Thomas --- CHANGELOG.md | 1 + plugins/repository-gcs/build.gradle | 44 +++++++++++-------- .../google-api-client-1.34.0.jar.sha1 | 1 - .../licenses/google-api-client-2.2.0.jar.sha1 | 1 + ...ces-storage-v1-rev20220608-1.32.1.jar.sha1 | 1 - ...ices-storage-v1-rev20230617-2.0.0.jar.sha1 | 1 + .../google-http-client-gson-1.41.4.jar.sha1 | 1 - .../google-http-client-gson-1.43.2.jar.sha1 | 1 + ...oogle-http-client-jackson2-1.42.3.jar.sha1 | 1 - ...oogle-http-client-jackson2-1.43.2.jar.sha1 | 1 + .../google-oauth-client-1.33.3.jar.sha1 | 1 - .../google-oauth-client-1.34.1.jar.sha1 | 1 + 12 files changed, 32 insertions(+), 23 deletions(-) delete mode 100644 plugins/repository-gcs/licenses/google-api-client-1.34.0.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/google-api-client-2.2.0.jar.sha1 delete mode 100644 plugins/repository-gcs/licenses/google-api-services-storage-v1-rev20220608-1.32.1.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/google-api-services-storage-v1-rev20230617-2.0.0.jar.sha1 delete mode 100644 plugins/repository-gcs/licenses/google-http-client-gson-1.41.4.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/google-http-client-gson-1.43.2.jar.sha1 delete mode 100644 plugins/repository-gcs/licenses/google-http-client-jackson2-1.42.3.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/google-http-client-jackson2-1.43.2.jar.sha1 delete mode 100644 plugins/repository-gcs/licenses/google-oauth-client-1.33.3.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/google-oauth-client-1.34.1.jar.sha1 diff --git a/CHANGELOG.md b/CHANGELOG.md index 5b3b8e251f205..5ec159efafc9a 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -122,6 +122,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Bump `io.projectreactor.netty:reactor-netty-http` from 1.1.7 to 1.1.8 (#8256) - [Upgrade] Lucene 9.7.0 release (#8272) - Bump `org.jboss.resteasy:resteasy-jackson2-provider` from 3.0.26.Final to 6.2.4.Final in /qa/wildfly ([#8209](https://github.com/opensearch-project/OpenSearch/pull/8209)) +- Bump `com.google.api-client:google-api-client` from 1.34.0 to 2.2.0 ([#8276](https://github.com/opensearch-project/OpenSearch/pull/8276)) ### Changed - Replace jboss-annotations-api_1.2_spec with jakarta.annotation-api ([#7836](https://github.com/opensearch-project/OpenSearch/pull/7836)) diff --git a/plugins/repository-gcs/build.gradle b/plugins/repository-gcs/build.gradle index 5d7a4c64ae34a..41c36dffea296 100644 --- a/plugins/repository-gcs/build.gradle +++ b/plugins/repository-gcs/build.gradle @@ -52,33 +52,43 @@ versions << [ ] dependencies { - api 'com.google.cloud:google-cloud-storage:1.113.1' + api 'com.google.api:api-common:1.8.1' + api 'com.google.api:gax:2.27.0' + api 'com.google.api:gax-httpjson:0.103.1' + + api 'com.google.apis:google-api-services-storage:v1-rev20230617-2.0.0' + + api 'com.google.api-client:google-api-client:2.2.0' + + api 'com.google.api.grpc:proto-google-common-protos:2.10.0' + api 'com.google.api.grpc:proto-google-iam-v1:0.12.0' + + api "com.google.auth:google-auth-library-credentials:${versions.google_auth}" + api "com.google.auth:google-auth-library-oauth2-http:${versions.google_auth}" + api 'com.google.cloud:google-cloud-core:2.5.10' + api 'com.google.cloud:google-cloud-core-http:2.17.0' + api 'com.google.cloud:google-cloud-storage:1.113.1' + + api 'com.google.code.gson:gson:2.9.0' + runtimeOnly "com.google.guava:guava:${versions.guava}" api 'com.google.guava:failureaccess:1.0.1' + api 'com.google.http-client:google-http-client:1.43.2' + api 'com.google.http-client:google-http-client-appengine:1.43.2' + api 'com.google.http-client:google-http-client-gson:1.43.2' + api 'com.google.http-client:google-http-client-jackson2:1.43.2' + + api 'com.google.oauth-client:google-oauth-client:1.34.1' + api "commons-logging:commons-logging:${versions.commonslogging}" api "org.apache.logging.log4j:log4j-1.2-api:${versions.log4j}" api "commons-codec:commons-codec:${versions.commonscodec}" - api 'com.google.api:api-common:1.8.1' - api 'com.google.api:gax:2.27.0' api 'org.threeten:threetenbp:1.4.4' - api 'com.google.code.gson:gson:2.9.0' - api 'com.google.api.grpc:proto-google-common-protos:2.10.0' - api 'com.google.api.grpc:proto-google-iam-v1:0.12.0' - api 'com.google.cloud:google-cloud-core-http:2.17.0' - api "com.google.auth:google-auth-library-credentials:${versions.google_auth}" - api "com.google.auth:google-auth-library-oauth2-http:${versions.google_auth}" - api 'com.google.oauth-client:google-oauth-client:1.33.3' - api 'com.google.api-client:google-api-client:1.34.0' - api 'com.google.http-client:google-http-client-appengine:1.43.2' - api 'com.google.http-client:google-http-client-jackson2:1.42.3' - api 'com.google.http-client:google-http-client-gson:1.41.4' - api 'com.google.api:gax-httpjson:0.103.1' api 'io.grpc:grpc-context:1.46.0' api 'io.opencensus:opencensus-api:0.31.1' api 'io.opencensus:opencensus-contrib-http-util:0.31.1' - api 'com.google.apis:google-api-services-storage:v1-rev20220608-1.32.1' testImplementation project(':test:fixtures:gcs-fixture') } @@ -167,8 +177,6 @@ thirdPartyAudit { 'org.apache.http.client.methods.HttpRequestBase', 'org.apache.http.config.Registry', 'org.apache.http.config.RegistryBuilder', - 'org.apache.http.config.SocketConfig', - 'org.apache.http.config.SocketConfig$Builder', 'org.apache.http.conn.ClientConnectionManager', 'org.apache.http.conn.ConnectionKeepAliveStrategy', 'org.apache.http.conn.params.ConnManagerParams', diff --git a/plugins/repository-gcs/licenses/google-api-client-1.34.0.jar.sha1 b/plugins/repository-gcs/licenses/google-api-client-1.34.0.jar.sha1 deleted file mode 100644 index 9be9480435085..0000000000000 --- a/plugins/repository-gcs/licenses/google-api-client-1.34.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -af2586412cabeee49c9db6d736e75b745bc467f8 \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/google-api-client-2.2.0.jar.sha1 b/plugins/repository-gcs/licenses/google-api-client-2.2.0.jar.sha1 new file mode 100644 index 0000000000000..f9604d6837ca9 --- /dev/null +++ b/plugins/repository-gcs/licenses/google-api-client-2.2.0.jar.sha1 @@ -0,0 +1 @@ +10e53fd4d987e37190432e896bdaa62e8ea2c628 \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/google-api-services-storage-v1-rev20220608-1.32.1.jar.sha1 b/plugins/repository-gcs/licenses/google-api-services-storage-v1-rev20220608-1.32.1.jar.sha1 deleted file mode 100644 index 07aaadb2664b2..0000000000000 --- a/plugins/repository-gcs/licenses/google-api-services-storage-v1-rev20220608-1.32.1.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -74724addc6cecac408dad3a6a26423b7647b3724 \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/google-api-services-storage-v1-rev20230617-2.0.0.jar.sha1 b/plugins/repository-gcs/licenses/google-api-services-storage-v1-rev20230617-2.0.0.jar.sha1 new file mode 100644 index 0000000000000..1a1452f773b96 --- /dev/null +++ b/plugins/repository-gcs/licenses/google-api-services-storage-v1-rev20230617-2.0.0.jar.sha1 @@ -0,0 +1 @@ +fc3f225b405303fe7cb760d578348b6b07e7ea8b \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/google-http-client-gson-1.41.4.jar.sha1 b/plugins/repository-gcs/licenses/google-http-client-gson-1.41.4.jar.sha1 deleted file mode 100644 index 17960a99abea2..0000000000000 --- a/plugins/repository-gcs/licenses/google-http-client-gson-1.41.4.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -fa665c1c573765dd858bc34931ad747e4ed11efe \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/google-http-client-gson-1.43.2.jar.sha1 b/plugins/repository-gcs/licenses/google-http-client-gson-1.43.2.jar.sha1 new file mode 100644 index 0000000000000..df0374aa27c70 --- /dev/null +++ b/plugins/repository-gcs/licenses/google-http-client-gson-1.43.2.jar.sha1 @@ -0,0 +1 @@ +b1c2e3e89804c113dba7b342aa8e0fc2cf3d9378 \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/google-http-client-jackson2-1.42.3.jar.sha1 b/plugins/repository-gcs/licenses/google-http-client-jackson2-1.42.3.jar.sha1 deleted file mode 100644 index 34d7d49f7b147..0000000000000 --- a/plugins/repository-gcs/licenses/google-http-client-jackson2-1.42.3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -789cafde696403b429026bf19071caf46d8c8934 \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/google-http-client-jackson2-1.43.2.jar.sha1 b/plugins/repository-gcs/licenses/google-http-client-jackson2-1.43.2.jar.sha1 new file mode 100644 index 0000000000000..7b606a07651ed --- /dev/null +++ b/plugins/repository-gcs/licenses/google-http-client-jackson2-1.43.2.jar.sha1 @@ -0,0 +1 @@ +5e52a9967ebd8246fc4cca64df5f03608db5ac6e \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/google-oauth-client-1.33.3.jar.sha1 b/plugins/repository-gcs/licenses/google-oauth-client-1.33.3.jar.sha1 deleted file mode 100644 index f2afaa1bc2dba..0000000000000 --- a/plugins/repository-gcs/licenses/google-oauth-client-1.33.3.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -9d445a8649b0de731922b9a3ebf1552b5403611d \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/google-oauth-client-1.34.1.jar.sha1 b/plugins/repository-gcs/licenses/google-oauth-client-1.34.1.jar.sha1 new file mode 100644 index 0000000000000..a8434bd380761 --- /dev/null +++ b/plugins/repository-gcs/licenses/google-oauth-client-1.34.1.jar.sha1 @@ -0,0 +1 @@ +4a4f88c5e13143f882268c98239fb85c3b2c6cb2 \ No newline at end of file From 976048d5d630b7adc53f54f2b27558d3a38b1f65 Mon Sep 17 00:00:00 2001 From: Sarthak Aggarwal Date: Thu, 29 Jun 2023 06:23:14 +0530 Subject: [PATCH 087/109] Moving zstd out of sandbox (#7908) * Adding zstd module to source Signed-off-by: Sarthak Aggarwal * Removing zstd module from sandbox Signed-off-by: Sarthak Aggarwal * Added tests and refactoring Signed-off-by: Sarthak Aggarwal * Fixing gradle issues Signed-off-by: Sarthak Aggarwal * flaky test Signed-off-by: Sarthak Aggarwal * fixing precommit failure Signed-off-by: Sarthak Aggarwal * Incorporate review comments and fixed precommit failures Signed-off-by: Sarthak Aggarwal * Incorporating review comments Signed-off-by: Sarthak Aggarwal * Incorporating review comments Signed-off-by: Sarthak Aggarwal * Adding Integ tests Signed-off-by: Sarthak Aggarwal * Incorporating review comments Signed-off-by: Sarthak Aggarwal --------- Signed-off-by: Sarthak Aggarwal Signed-off-by: Daniel (dB.) Doubrovkine Co-authored-by: Daniel (dB.) Doubrovkine --- CHANGELOG.md | 1 + .../index/codec/MultiCodecReindexIT.java | 189 ++++++++++++++++++ sandbox/plugins/custom-codecs/build.gradle | 28 --- .../codec/customcodecs/CustomCodecPlugin.java | 26 --- .../plugin-metadata/plugin-security.policy | 11 - .../index/codec/MultiCodecMergeIT.java | 175 ++++++++++++++++ .../opensearch/index/codec/CodecService.java | 8 + .../customcodecs/Lucene95CustomCodec.java | 27 ++- .../Lucene95CustomStoredFieldsFormat.java | 13 +- .../index/codec/customcodecs/ZstdCodec.java | 7 + .../customcodecs/ZstdCompressionMode.java | 5 +- .../codec/customcodecs/ZstdNoDictCodec.java | 9 +- .../ZstdNoDictCompressionMode.java | 5 +- .../codec/customcodecs/package-info.java | 0 .../opensearch/index/engine/EngineConfig.java | 4 +- .../services/org.apache.lucene.codecs.Codec | 0 .../opensearch/index/codec/CodecTests.java | 81 ++++++-- .../customcodecs/AbstractCompressorTests.java | 10 +- ...Lucene95CustomStoredFieldsFormatTests.java | 27 +++ .../customcodecs/ZstdCompressorTests.java | 0 .../ZstdNoDictCompressorTests.java | 0 21 files changed, 524 insertions(+), 102 deletions(-) create mode 100644 modules/reindex/src/internalClusterTest/java/org/opensearch/index/codec/MultiCodecReindexIT.java delete mode 100644 sandbox/plugins/custom-codecs/build.gradle delete mode 100644 sandbox/plugins/custom-codecs/src/main/java/org/opensearch/index/codec/customcodecs/CustomCodecPlugin.java delete mode 100644 sandbox/plugins/custom-codecs/src/main/plugin-metadata/plugin-security.policy create mode 100644 server/src/internalClusterTest/java/org/opensearch/index/codec/MultiCodecMergeIT.java rename {sandbox/plugins/custom-codecs => server}/src/main/java/org/opensearch/index/codec/customcodecs/Lucene95CustomCodec.java (65%) rename {sandbox/plugins/custom-codecs => server}/src/main/java/org/opensearch/index/codec/customcodecs/Lucene95CustomStoredFieldsFormat.java (96%) rename {sandbox/plugins/custom-codecs => server}/src/main/java/org/opensearch/index/codec/customcodecs/ZstdCodec.java (79%) rename {sandbox/plugins/custom-codecs => server}/src/main/java/org/opensearch/index/codec/customcodecs/ZstdCompressionMode.java (99%) rename {sandbox/plugins/custom-codecs => server}/src/main/java/org/opensearch/index/codec/customcodecs/ZstdNoDictCodec.java (74%) rename {sandbox/plugins/custom-codecs => server}/src/main/java/org/opensearch/index/codec/customcodecs/ZstdNoDictCompressionMode.java (99%) rename {sandbox/plugins/custom-codecs => server}/src/main/java/org/opensearch/index/codec/customcodecs/package-info.java (100%) rename {sandbox/plugins/custom-codecs => server}/src/main/resources/META-INF/services/org.apache.lucene.codecs.Codec (100%) rename {sandbox/plugins/custom-codecs => server}/src/test/java/org/opensearch/index/codec/customcodecs/AbstractCompressorTests.java (100%) create mode 100644 server/src/test/java/org/opensearch/index/codec/customcodecs/Lucene95CustomStoredFieldsFormatTests.java rename {sandbox/plugins/custom-codecs => server}/src/test/java/org/opensearch/index/codec/customcodecs/ZstdCompressorTests.java (100%) rename {sandbox/plugins/custom-codecs => server}/src/test/java/org/opensearch/index/codec/customcodecs/ZstdNoDictCompressorTests.java (100%) diff --git a/CHANGELOG.md b/CHANGELOG.md index 5ec159efafc9a..139807f0bffd1 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -138,6 +138,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - [Snapshot Interop] Add Changes in Create Snapshot Flow for remote store interoperability. ([#7118](https://github.com/opensearch-project/OpenSearch/pull/7118)) - Allow insecure string settings to warn-log usage and advise to migration of a newer secure variant ([#5496](https://github.com/opensearch-project/OpenSearch/pull/5496)) - Add self-organizing hash table to improve the performance of bucket aggregations ([#7652](https://github.com/opensearch-project/OpenSearch/pull/7652)) +- Move ZSTD compression codecs out of the sandbox ([#7908](https://github.com/opensearch-project/OpenSearch/pull/7908)) ### Deprecated diff --git a/modules/reindex/src/internalClusterTest/java/org/opensearch/index/codec/MultiCodecReindexIT.java b/modules/reindex/src/internalClusterTest/java/org/opensearch/index/codec/MultiCodecReindexIT.java new file mode 100644 index 0000000000000..87f3c68d8af76 --- /dev/null +++ b/modules/reindex/src/internalClusterTest/java/org/opensearch/index/codec/MultiCodecReindexIT.java @@ -0,0 +1,189 @@ +/* + * 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.index.codec; + +import org.opensearch.action.admin.indices.flush.FlushResponse; +import org.opensearch.action.admin.indices.refresh.RefreshResponse; +import org.opensearch.action.admin.indices.segments.IndicesSegmentsRequest; +import org.opensearch.action.admin.indices.settings.put.UpdateSettingsRequest; +import org.opensearch.action.support.ActiveShardCount; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.settings.Settings; +import org.opensearch.index.engine.Segment; +import org.opensearch.index.reindex.BulkByScrollResponse; +import org.opensearch.index.reindex.ReindexAction; +import org.opensearch.index.reindex.ReindexRequestBuilder; +import org.opensearch.index.reindex.ReindexTestCase; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static java.util.stream.Collectors.toList; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_BLOCKS_METADATA; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_BLOCKS_READ; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_BLOCKS_WRITE; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_READ_ONLY; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_READ_ONLY_ALLOW_DELETE; +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertNoFailures; + +public class MultiCodecReindexIT extends ReindexTestCase { + + public void testReindexingMultipleCodecs() throws InterruptedException, ExecutionException { + internalCluster().ensureAtLeastNumDataNodes(1); + Map codecMap = Map.of( + "best_compression", + "BEST_COMPRESSION", + "zstd_no_dict", + "ZSTD_NO_DICT", + "zstd", + "ZSTD", + "default", + "BEST_SPEED" + ); + + for (Map.Entry codec : codecMap.entrySet()) { + assertReindexingWithMultipleCodecs(codec.getKey(), codec.getValue(), codecMap); + } + + } + + private void assertReindexingWithMultipleCodecs(String destCodec, String destCodecMode, Map codecMap) + throws ExecutionException, InterruptedException { + + final String index = "test-index" + destCodec; + final String destIndex = "dest-index" + destCodec; + + // creating source index + createIndex( + index, + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put("index.codec", "default") + .put("index.merge.policy.max_merged_segment", "1b") + .build() + ); + ensureGreen(index); + + final int nbDocs = randomIntBetween(2, 5); + + // indexing with all 4 codecs + for (Map.Entry codec : codecMap.entrySet()) { + useCodec(index, codec.getKey()); + ingestDocs(index, nbDocs); + } + + assertTrue( + getSegments(index).stream() + .flatMap(s -> s.getAttributes().values().stream()) + .collect(Collectors.toSet()) + .containsAll(codecMap.values()) + ); + + // creating destination index with destination codec + createIndex( + destIndex, + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put("index.codec", destCodec) + .build() + ); + + BulkByScrollResponse bulkResponse = new ReindexRequestBuilder(client(), ReindexAction.INSTANCE).source(index) + .destination(destIndex) + .refresh(true) + .waitForActiveShards(ActiveShardCount.ONE) + .get(); + + assertEquals(codecMap.size() * nbDocs, bulkResponse.getCreated()); + assertEquals(codecMap.size() * nbDocs, bulkResponse.getTotal()); + assertEquals(0, bulkResponse.getDeleted()); + assertEquals(0, bulkResponse.getNoops()); + assertEquals(0, bulkResponse.getVersionConflicts()); + assertEquals(1, bulkResponse.getBatches()); + assertTrue(bulkResponse.getTook().getMillis() > 0); + assertEquals(0, bulkResponse.getBulkFailures().size()); + assertEquals(0, bulkResponse.getSearchFailures().size()); + assertTrue(getSegments(destIndex).stream().allMatch(segment -> segment.attributes.containsValue(destCodecMode))); + } + + private void useCodec(String index, String codec) throws ExecutionException, InterruptedException { + assertAcked(client().admin().indices().prepareClose(index)); + + assertAcked( + client().admin() + .indices() + .updateSettings(new UpdateSettingsRequest(index).settings(Settings.builder().put("index.codec", codec))) + .get() + ); + + assertAcked(client().admin().indices().prepareOpen(index)); + } + + private void flushAndRefreshIndex(String index) { + + // Request is not blocked + for (String blockSetting : Arrays.asList( + SETTING_BLOCKS_READ, + SETTING_BLOCKS_WRITE, + SETTING_READ_ONLY, + SETTING_BLOCKS_METADATA, + SETTING_READ_ONLY_ALLOW_DELETE + )) { + try { + enableIndexBlock(index, blockSetting); + // flush + FlushResponse flushResponse = client().admin().indices().prepareFlush(index).setForce(true).execute().actionGet(); + assertNoFailures(flushResponse); + + // refresh + RefreshResponse refreshResponse = client().admin().indices().prepareRefresh(index).execute().actionGet(); + assertNoFailures(refreshResponse); + } finally { + disableIndexBlock(index, blockSetting); + } + } + } + + private void ingestDocs(String index, int nbDocs) throws InterruptedException { + + indexRandom( + randomBoolean(), + false, + randomBoolean(), + IntStream.range(0, nbDocs) + .mapToObj(i -> client().prepareIndex(index).setId(UUID.randomUUID().toString()).setSource("num", i)) + .collect(toList()) + ); + flushAndRefreshIndex(index); + } + + private ArrayList getSegments(String index) { + + return new ArrayList<>( + client().admin() + .indices() + .segments(new IndicesSegmentsRequest(index)) + .actionGet() + .getIndices() + .get(index) + .getShards() + .get(0) + .getShards()[0].getSegments() + ); + } + +} diff --git a/sandbox/plugins/custom-codecs/build.gradle b/sandbox/plugins/custom-codecs/build.gradle deleted file mode 100644 index 2183df25044a4..0000000000000 --- a/sandbox/plugins/custom-codecs/build.gradle +++ /dev/null @@ -1,28 +0,0 @@ -/* - * 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. - * - * Modifications Copyright OpenSearch Contributors. See - * GitHub history for details. - */ - -apply plugin: 'opensearch.opensearchplugin' -apply plugin: 'opensearch.yaml-rest-test' - -opensearchplugin { - name 'custom-codecs' - description 'A plugin that implements custom compression codecs.' - classname 'org.opensearch.index.codec.customcodecs.CustomCodecPlugin' - licenseFile rootProject.file('licenses/APACHE-LICENSE-2.0.txt') - noticeFile rootProject.file('NOTICE.txt') -} - -dependencies { - api "com.github.luben:zstd-jni:${versions.zstd}" -} - -yamlRestTest.enabled = false; -testingConventions.enabled = false; diff --git a/sandbox/plugins/custom-codecs/src/main/java/org/opensearch/index/codec/customcodecs/CustomCodecPlugin.java b/sandbox/plugins/custom-codecs/src/main/java/org/opensearch/index/codec/customcodecs/CustomCodecPlugin.java deleted file mode 100644 index 9d36184bf81af..0000000000000 --- a/sandbox/plugins/custom-codecs/src/main/java/org/opensearch/index/codec/customcodecs/CustomCodecPlugin.java +++ /dev/null @@ -1,26 +0,0 @@ -/* - * 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.index.codec.customcodecs; - -import org.opensearch.plugins.Plugin; -import org.opensearch.plugins.EnginePlugin; - -/** - * A plugin that implements custom codecs. Supports these codecs: - *

    - *
  • zstd - *
  • zstdnodict - *
- * - * @opensearch.internal - */ -public final class CustomCodecPlugin extends Plugin implements EnginePlugin { - /** Creates a new instance. */ - public CustomCodecPlugin() {} -} diff --git a/sandbox/plugins/custom-codecs/src/main/plugin-metadata/plugin-security.policy b/sandbox/plugins/custom-codecs/src/main/plugin-metadata/plugin-security.policy deleted file mode 100644 index 8161010cfa897..0000000000000 --- a/sandbox/plugins/custom-codecs/src/main/plugin-metadata/plugin-security.policy +++ /dev/null @@ -1,11 +0,0 @@ -/* - * 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. - */ - -grant codeBase "${codebase.zstd-jni}" { - permission java.lang.RuntimePermission "loadLibrary.*"; -}; diff --git a/server/src/internalClusterTest/java/org/opensearch/index/codec/MultiCodecMergeIT.java b/server/src/internalClusterTest/java/org/opensearch/index/codec/MultiCodecMergeIT.java new file mode 100644 index 0000000000000..2866292e5e2e0 --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/index/codec/MultiCodecMergeIT.java @@ -0,0 +1,175 @@ +/* + * 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.index.codec; + +import org.opensearch.action.admin.indices.flush.FlushResponse; +import org.opensearch.action.admin.indices.forcemerge.ForceMergeResponse; +import org.opensearch.action.admin.indices.refresh.RefreshResponse; +import org.opensearch.action.admin.indices.segments.IndicesSegmentsRequest; +import org.opensearch.action.admin.indices.settings.put.UpdateSettingsRequest; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.settings.Settings; +import org.opensearch.index.engine.Segment; +import org.opensearch.test.OpenSearchIntegTestCase; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static java.util.stream.Collectors.toList; +import static org.hamcrest.Matchers.is; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_BLOCKS_METADATA; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_BLOCKS_READ; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_BLOCKS_WRITE; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_READ_ONLY; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_READ_ONLY_ALLOW_DELETE; +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertNoFailures; + +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST) +public class MultiCodecMergeIT extends OpenSearchIntegTestCase { + + public void testForceMergeMultipleCodecs() throws ExecutionException, InterruptedException { + + Map codecMap = Map.of( + "best_compression", + "BEST_COMPRESSION", + "zstd_no_dict", + "ZSTD_NO_DICT", + "zstd", + "ZSTD", + "default", + "BEST_SPEED" + ); + + for (Map.Entry codec : codecMap.entrySet()) { + forceMergeMultipleCodecs(codec.getKey(), codec.getValue(), codecMap); + } + + } + + private void forceMergeMultipleCodecs(String finalCodec, String finalCodecMode, Map codecMap) throws ExecutionException, + InterruptedException { + + internalCluster().ensureAtLeastNumDataNodes(1); + final String index = "test-index" + finalCodec; + + // creating index + createIndex( + index, + Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put("index.codec", "default") + .put("index.merge.policy.max_merged_segment", "1b") + .build() + ); + ensureGreen(index); + // ingesting and asserting segment codec mode for all four codecs + for (Map.Entry codec : codecMap.entrySet()) { + useCodec(index, codec.getKey()); + ingestDocs(index); + } + + assertTrue( + getSegments(index).stream() + .flatMap(s -> s.getAttributes().values().stream()) + .collect(Collectors.toSet()) + .containsAll(codecMap.values()) + ); + + // force merge into final codec + useCodec(index, finalCodec); + flushAndRefreshIndex(index); + final ForceMergeResponse forceMergeResponse = client().admin().indices().prepareForceMerge(index).setMaxNumSegments(1).get(); + + assertThat(forceMergeResponse.getFailedShards(), is(0)); + assertThat(forceMergeResponse.getSuccessfulShards(), is(1)); + + flushAndRefreshIndex(index); + + List segments = getSegments(index).stream().filter(Segment::isSearch).collect(Collectors.toList()); + assertEquals(1, segments.size()); + assertTrue(segments.stream().findFirst().get().attributes.containsValue(finalCodecMode)); + } + + private void useCodec(String index, String codec) throws ExecutionException, InterruptedException { + assertAcked(client().admin().indices().prepareClose(index)); + + assertAcked( + client().admin() + .indices() + .updateSettings(new UpdateSettingsRequest(index).settings(Settings.builder().put("index.codec", codec))) + .get() + ); + + assertAcked(client().admin().indices().prepareOpen(index)); + } + + private void ingestDocs(String index) throws InterruptedException { + ingest(index); + flushAndRefreshIndex(index); + } + + private ArrayList getSegments(String index) { + + return new ArrayList<>( + client().admin() + .indices() + .segments(new IndicesSegmentsRequest(index)) + .actionGet() + .getIndices() + .get(index) + .getShards() + .get(0) + .getShards()[0].getSegments() + ); + } + + private void ingest(String index) throws InterruptedException { + + final int nbDocs = randomIntBetween(1, 5); + indexRandom( + randomBoolean(), + false, + randomBoolean(), + IntStream.range(0, nbDocs) + .mapToObj(i -> client().prepareIndex(index).setId(UUID.randomUUID().toString()).setSource("num", i)) + .collect(toList()) + ); + } + + private void flushAndRefreshIndex(String index) { + + // Request is not blocked + for (String blockSetting : Arrays.asList( + SETTING_BLOCKS_READ, + SETTING_BLOCKS_WRITE, + SETTING_READ_ONLY, + SETTING_BLOCKS_METADATA, + SETTING_READ_ONLY_ALLOW_DELETE + )) { + try { + enableIndexBlock(index, blockSetting); + FlushResponse flushResponse = client().admin().indices().prepareFlush(index).setForce(true).execute().actionGet(); + assertNoFailures(flushResponse); + RefreshResponse response = client().admin().indices().prepareRefresh(index).execute().actionGet(); + assertNoFailures(response); + } finally { + disableIndexBlock(index, blockSetting); + } + } + } + +} diff --git a/server/src/main/java/org/opensearch/index/codec/CodecService.java b/server/src/main/java/org/opensearch/index/codec/CodecService.java index e4899c02d37e8..b6dac7bd1596c 100644 --- a/server/src/main/java/org/opensearch/index/codec/CodecService.java +++ b/server/src/main/java/org/opensearch/index/codec/CodecService.java @@ -38,6 +38,8 @@ import org.apache.lucene.codecs.lucene95.Lucene95Codec.Mode; import org.opensearch.common.Nullable; import org.opensearch.common.collect.MapBuilder; +import org.opensearch.index.codec.customcodecs.ZstdCodec; +import org.opensearch.index.codec.customcodecs.ZstdNoDictCodec; import org.opensearch.index.mapper.MapperService; import java.util.Map; @@ -58,15 +60,21 @@ public class CodecService { public static final String BEST_COMPRESSION_CODEC = "best_compression"; /** the raw unfiltered lucene default. useful for testing */ public static final String LUCENE_DEFAULT_CODEC = "lucene_default"; + public static final String ZSTD_CODEC = "zstd"; + public static final String ZSTD_NO_DICT_CODEC = "zstd_no_dict"; public CodecService(@Nullable MapperService mapperService, Logger logger) { final MapBuilder codecs = MapBuilder.newMapBuilder(); if (mapperService == null) { codecs.put(DEFAULT_CODEC, new Lucene95Codec()); codecs.put(BEST_COMPRESSION_CODEC, new Lucene95Codec(Mode.BEST_COMPRESSION)); + codecs.put(ZSTD_CODEC, new ZstdCodec()); + codecs.put(ZSTD_NO_DICT_CODEC, new ZstdNoDictCodec()); } else { codecs.put(DEFAULT_CODEC, new PerFieldMappingPostingFormatCodec(Mode.BEST_SPEED, mapperService, logger)); codecs.put(BEST_COMPRESSION_CODEC, new PerFieldMappingPostingFormatCodec(Mode.BEST_COMPRESSION, mapperService, logger)); + codecs.put(ZSTD_CODEC, new ZstdCodec(mapperService, logger)); + codecs.put(ZSTD_NO_DICT_CODEC, new ZstdNoDictCodec(mapperService, logger)); } codecs.put(LUCENE_DEFAULT_CODEC, Codec.getDefault()); for (String codec : Codec.availableCodecs()) { diff --git a/sandbox/plugins/custom-codecs/src/main/java/org/opensearch/index/codec/customcodecs/Lucene95CustomCodec.java b/server/src/main/java/org/opensearch/index/codec/customcodecs/Lucene95CustomCodec.java similarity index 65% rename from sandbox/plugins/custom-codecs/src/main/java/org/opensearch/index/codec/customcodecs/Lucene95CustomCodec.java rename to server/src/main/java/org/opensearch/index/codec/customcodecs/Lucene95CustomCodec.java index ad9e5cd3374fa..3c570f9d0566c 100644 --- a/sandbox/plugins/custom-codecs/src/main/java/org/opensearch/index/codec/customcodecs/Lucene95CustomCodec.java +++ b/server/src/main/java/org/opensearch/index/codec/customcodecs/Lucene95CustomCodec.java @@ -8,19 +8,27 @@ package org.opensearch.index.codec.customcodecs; -import org.apache.lucene.codecs.StoredFieldsFormat; +import org.apache.logging.log4j.Logger; import org.apache.lucene.codecs.FilterCodec; +import org.apache.lucene.codecs.StoredFieldsFormat; import org.apache.lucene.codecs.lucene95.Lucene95Codec; +import org.opensearch.index.codec.PerFieldMappingPostingFormatCodec; +import org.opensearch.index.mapper.MapperService; -import java.util.Locale; - -abstract class Lucene95CustomCodec extends FilterCodec { +/** + * + * Extends {@link FilterCodec} to reuse the functionality of Lucene Codec. + * Supports two modes zstd and zstd_no_dict. + * + * @opensearch.internal + */ +public abstract class Lucene95CustomCodec extends FilterCodec { public static final int DEFAULT_COMPRESSION_LEVEL = 6; /** Each mode represents a compression algorithm. */ public enum Mode { ZSTD, - ZSTDNODICT + ZSTD_NO_DICT } private final StoredFieldsFormat storedFieldsFormat; @@ -40,10 +48,15 @@ public Lucene95CustomCodec(Mode mode) { * the other compression codecs: default, lucene_default, and best_compression. * * @param mode The compression codec (ZSTD or ZSTDNODICT). - * @parama compressionLevel The compression level. + * @param compressionLevel The compression level. */ public Lucene95CustomCodec(Mode mode, int compressionLevel) { - super(mode.name().toLowerCase(Locale.ROOT), new Lucene95Codec()); + super("Lucene95CustomCodec", new Lucene95Codec()); + this.storedFieldsFormat = new Lucene95CustomStoredFieldsFormat(mode, compressionLevel); + } + + public Lucene95CustomCodec(Mode mode, int compressionLevel, MapperService mapperService, Logger logger) { + super("Lucene95CustomCodec", new PerFieldMappingPostingFormatCodec(Lucene95Codec.Mode.BEST_SPEED, mapperService, logger)); this.storedFieldsFormat = new Lucene95CustomStoredFieldsFormat(mode, compressionLevel); } diff --git a/sandbox/plugins/custom-codecs/src/main/java/org/opensearch/index/codec/customcodecs/Lucene95CustomStoredFieldsFormat.java b/server/src/main/java/org/opensearch/index/codec/customcodecs/Lucene95CustomStoredFieldsFormat.java similarity index 96% rename from sandbox/plugins/custom-codecs/src/main/java/org/opensearch/index/codec/customcodecs/Lucene95CustomStoredFieldsFormat.java rename to server/src/main/java/org/opensearch/index/codec/customcodecs/Lucene95CustomStoredFieldsFormat.java index 2bfec2ef171d4..f70306afc8562 100644 --- a/sandbox/plugins/custom-codecs/src/main/java/org/opensearch/index/codec/customcodecs/Lucene95CustomStoredFieldsFormat.java +++ b/server/src/main/java/org/opensearch/index/codec/customcodecs/Lucene95CustomStoredFieldsFormat.java @@ -8,8 +8,6 @@ package org.opensearch.index.codec.customcodecs; -import java.io.IOException; -import java.util.Objects; import org.apache.lucene.codecs.StoredFieldsFormat; import org.apache.lucene.codecs.StoredFieldsReader; import org.apache.lucene.codecs.StoredFieldsWriter; @@ -20,6 +18,9 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; +import java.io.IOException; +import java.util.Objects; + /** Stored field format used by pluggable codec */ public class Lucene95CustomStoredFieldsFormat extends StoredFieldsFormat { @@ -95,7 +96,7 @@ public StoredFieldsWriter fieldsWriter(Directory directory, SegmentInfo si, IOCo return impl(mode).fieldsWriter(directory, si, context); } - private StoredFieldsFormat impl(Lucene95CustomCodec.Mode mode) { + StoredFieldsFormat impl(Lucene95CustomCodec.Mode mode) { switch (mode) { case ZSTD: return new Lucene90CompressingStoredFieldsFormat( @@ -105,7 +106,7 @@ private StoredFieldsFormat impl(Lucene95CustomCodec.Mode mode) { ZSTD_MAX_DOCS_PER_BLOCK, ZSTD_BLOCK_SHIFT ); - case ZSTDNODICT: + case ZSTD_NO_DICT: return new Lucene90CompressingStoredFieldsFormat( "CustomStoredFieldsZstdNoDict", zstdNoDictCompressionMode, @@ -117,4 +118,8 @@ private StoredFieldsFormat impl(Lucene95CustomCodec.Mode mode) { throw new AssertionError(); } } + + Lucene95CustomCodec.Mode getMode() { + return mode; + } } diff --git a/sandbox/plugins/custom-codecs/src/main/java/org/opensearch/index/codec/customcodecs/ZstdCodec.java b/server/src/main/java/org/opensearch/index/codec/customcodecs/ZstdCodec.java similarity index 79% rename from sandbox/plugins/custom-codecs/src/main/java/org/opensearch/index/codec/customcodecs/ZstdCodec.java rename to server/src/main/java/org/opensearch/index/codec/customcodecs/ZstdCodec.java index 2b09540d8037d..68da782421e6e 100644 --- a/sandbox/plugins/custom-codecs/src/main/java/org/opensearch/index/codec/customcodecs/ZstdCodec.java +++ b/server/src/main/java/org/opensearch/index/codec/customcodecs/ZstdCodec.java @@ -8,6 +8,9 @@ package org.opensearch.index.codec.customcodecs; +import org.apache.logging.log4j.Logger; +import org.opensearch.index.mapper.MapperService; + /** * ZstdCodec provides ZSTD compressor using the zstd-jni library. */ @@ -29,6 +32,10 @@ public ZstdCodec(int compressionLevel) { super(Mode.ZSTD, compressionLevel); } + public ZstdCodec(MapperService mapperService, Logger logger) { + super(Mode.ZSTD, DEFAULT_COMPRESSION_LEVEL, mapperService, logger); + } + /** The name for this codec. */ @Override public String toString() { diff --git a/sandbox/plugins/custom-codecs/src/main/java/org/opensearch/index/codec/customcodecs/ZstdCompressionMode.java b/server/src/main/java/org/opensearch/index/codec/customcodecs/ZstdCompressionMode.java similarity index 99% rename from sandbox/plugins/custom-codecs/src/main/java/org/opensearch/index/codec/customcodecs/ZstdCompressionMode.java rename to server/src/main/java/org/opensearch/index/codec/customcodecs/ZstdCompressionMode.java index 5b8f1ffcc9569..7057dac3d6bd2 100644 --- a/sandbox/plugins/custom-codecs/src/main/java/org/opensearch/index/codec/customcodecs/ZstdCompressionMode.java +++ b/server/src/main/java/org/opensearch/index/codec/customcodecs/ZstdCompressionMode.java @@ -13,16 +13,17 @@ import com.github.luben.zstd.ZstdDecompressCtx; import com.github.luben.zstd.ZstdDictCompress; import com.github.luben.zstd.ZstdDictDecompress; -import java.io.IOException; import org.apache.lucene.codecs.compressing.CompressionMode; import org.apache.lucene.codecs.compressing.Compressor; import org.apache.lucene.codecs.compressing.Decompressor; +import org.apache.lucene.store.ByteBuffersDataInput; import org.apache.lucene.store.DataInput; import org.apache.lucene.store.DataOutput; -import org.apache.lucene.store.ByteBuffersDataInput; import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.BytesRef; +import java.io.IOException; + /** Zstandard Compression Mode */ public class ZstdCompressionMode extends CompressionMode { diff --git a/sandbox/plugins/custom-codecs/src/main/java/org/opensearch/index/codec/customcodecs/ZstdNoDictCodec.java b/server/src/main/java/org/opensearch/index/codec/customcodecs/ZstdNoDictCodec.java similarity index 74% rename from sandbox/plugins/custom-codecs/src/main/java/org/opensearch/index/codec/customcodecs/ZstdNoDictCodec.java rename to server/src/main/java/org/opensearch/index/codec/customcodecs/ZstdNoDictCodec.java index 4ed6ba57545d0..26620473ec116 100644 --- a/sandbox/plugins/custom-codecs/src/main/java/org/opensearch/index/codec/customcodecs/ZstdNoDictCodec.java +++ b/server/src/main/java/org/opensearch/index/codec/customcodecs/ZstdNoDictCodec.java @@ -8,6 +8,9 @@ package org.opensearch.index.codec.customcodecs; +import org.apache.logging.log4j.Logger; +import org.opensearch.index.mapper.MapperService; + /** * ZstdNoDictCodec provides ZSTD compressor without a dictionary support. */ @@ -26,7 +29,11 @@ public ZstdNoDictCodec() { * @param compressionLevel The compression level. */ public ZstdNoDictCodec(int compressionLevel) { - super(Mode.ZSTDNODICT, compressionLevel); + super(Mode.ZSTD_NO_DICT, compressionLevel); + } + + public ZstdNoDictCodec(MapperService mapperService, Logger logger) { + super(Mode.ZSTD_NO_DICT, DEFAULT_COMPRESSION_LEVEL, mapperService, logger); } /** The name for this codec. */ diff --git a/sandbox/plugins/custom-codecs/src/main/java/org/opensearch/index/codec/customcodecs/ZstdNoDictCompressionMode.java b/server/src/main/java/org/opensearch/index/codec/customcodecs/ZstdNoDictCompressionMode.java similarity index 99% rename from sandbox/plugins/custom-codecs/src/main/java/org/opensearch/index/codec/customcodecs/ZstdNoDictCompressionMode.java rename to server/src/main/java/org/opensearch/index/codec/customcodecs/ZstdNoDictCompressionMode.java index 6cfd85b053190..7a1d661550768 100644 --- a/sandbox/plugins/custom-codecs/src/main/java/org/opensearch/index/codec/customcodecs/ZstdNoDictCompressionMode.java +++ b/server/src/main/java/org/opensearch/index/codec/customcodecs/ZstdNoDictCompressionMode.java @@ -9,16 +9,17 @@ package org.opensearch.index.codec.customcodecs; import com.github.luben.zstd.Zstd; -import java.io.IOException; import org.apache.lucene.codecs.compressing.CompressionMode; import org.apache.lucene.codecs.compressing.Compressor; import org.apache.lucene.codecs.compressing.Decompressor; +import org.apache.lucene.store.ByteBuffersDataInput; import org.apache.lucene.store.DataInput; import org.apache.lucene.store.DataOutput; -import org.apache.lucene.store.ByteBuffersDataInput; import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.BytesRef; +import java.io.IOException; + /** ZSTD Compression Mode (without a dictionary support). */ public class ZstdNoDictCompressionMode extends CompressionMode { diff --git a/sandbox/plugins/custom-codecs/src/main/java/org/opensearch/index/codec/customcodecs/package-info.java b/server/src/main/java/org/opensearch/index/codec/customcodecs/package-info.java similarity index 100% rename from sandbox/plugins/custom-codecs/src/main/java/org/opensearch/index/codec/customcodecs/package-info.java rename to server/src/main/java/org/opensearch/index/codec/customcodecs/package-info.java diff --git a/server/src/main/java/org/opensearch/index/engine/EngineConfig.java b/server/src/main/java/org/opensearch/index/engine/EngineConfig.java index 338a541af387a..7419cf1dadea6 100644 --- a/server/src/main/java/org/opensearch/index/engine/EngineConfig.java +++ b/server/src/main/java/org/opensearch/index/engine/EngineConfig.java @@ -129,12 +129,14 @@ public Supplier retentionLeasesSupplier() { switch (s) { case "default": case "best_compression": + case "zstd": + case "zstd_no_dict": case "lucene_default": return s; default: if (Codec.availableCodecs().contains(s) == false) { // we don't error message the not officially supported ones throw new IllegalArgumentException( - "unknown value for [index.codec] must be one of [default, best_compression] but was: " + s + "unknown value for [index.codec] must be one of [default, best_compression, zstd, zstd_no_dict] but was: " + s ); } return s; diff --git a/sandbox/plugins/custom-codecs/src/main/resources/META-INF/services/org.apache.lucene.codecs.Codec b/server/src/main/resources/META-INF/services/org.apache.lucene.codecs.Codec similarity index 100% rename from sandbox/plugins/custom-codecs/src/main/resources/META-INF/services/org.apache.lucene.codecs.Codec rename to server/src/main/resources/META-INF/services/org.apache.lucene.codecs.Codec diff --git a/server/src/test/java/org/opensearch/index/codec/CodecTests.java b/server/src/test/java/org/opensearch/index/codec/CodecTests.java index bc50525412954..016e785f8da13 100644 --- a/server/src/test/java/org/opensearch/index/codec/CodecTests.java +++ b/server/src/test/java/org/opensearch/index/codec/CodecTests.java @@ -47,6 +47,8 @@ import org.opensearch.env.Environment; import org.opensearch.index.IndexSettings; import org.opensearch.index.analysis.IndexAnalyzers; +import org.opensearch.index.codec.customcodecs.Lucene95CustomCodec; +import org.opensearch.index.codec.customcodecs.Lucene95CustomStoredFieldsFormat; import org.opensearch.index.mapper.MapperService; import org.opensearch.index.similarity.SimilarityService; import org.opensearch.indices.mapper.MapperRegistry; @@ -63,40 +65,75 @@ public class CodecTests extends OpenSearchTestCase { public void testResolveDefaultCodecs() throws Exception { - CodecService codecService = createCodecService(); + CodecService codecService = createCodecService(false); assertThat(codecService.codec("default"), instanceOf(PerFieldMappingPostingFormatCodec.class)); assertThat(codecService.codec("default"), instanceOf(Lucene95Codec.class)); } public void testDefault() throws Exception { - Codec codec = createCodecService().codec("default"); + Codec codec = createCodecService(false).codec("default"); assertStoredFieldsCompressionEquals(Lucene95Codec.Mode.BEST_SPEED, codec); } public void testBestCompression() throws Exception { - Codec codec = createCodecService().codec("best_compression"); + Codec codec = createCodecService(false).codec("best_compression"); assertStoredFieldsCompressionEquals(Lucene95Codec.Mode.BEST_COMPRESSION, codec); } + public void testZstd() throws Exception { + Codec codec = createCodecService(false).codec("zstd"); + assertStoredFieldsCompressionEquals(Lucene95CustomCodec.Mode.ZSTD, codec); + } + + public void testZstdNoDict() throws Exception { + Codec codec = createCodecService(false).codec("zstd_no_dict"); + assertStoredFieldsCompressionEquals(Lucene95CustomCodec.Mode.ZSTD_NO_DICT, codec); + } + + public void testDefaultMapperServiceNull() throws Exception { + Codec codec = createCodecService(true).codec("default"); + assertStoredFieldsCompressionEquals(Lucene95Codec.Mode.BEST_SPEED, codec); + } + + public void testBestCompressionMapperServiceNull() throws Exception { + Codec codec = createCodecService(true).codec("best_compression"); + assertStoredFieldsCompressionEquals(Lucene95Codec.Mode.BEST_COMPRESSION, codec); + } + + public void testZstdMapperServiceNull() throws Exception { + Codec codec = createCodecService(true).codec("zstd"); + assertStoredFieldsCompressionEquals(Lucene95CustomCodec.Mode.ZSTD, codec); + } + + public void testZstdNoDictMapperServiceNull() throws Exception { + Codec codec = createCodecService(true).codec("zstd_no_dict"); + assertStoredFieldsCompressionEquals(Lucene95CustomCodec.Mode.ZSTD_NO_DICT, codec); + } + + public void testExceptionCodecNull() { + assertThrows(IllegalArgumentException.class, () -> createCodecService(true).codec(null)); + } + // write some docs with it, inspect .si to see this was the used compression private void assertStoredFieldsCompressionEquals(Lucene95Codec.Mode expected, Codec actual) throws Exception { - Directory dir = newDirectory(); - IndexWriterConfig iwc = newIndexWriterConfig(null); - iwc.setCodec(actual); - IndexWriter iw = new IndexWriter(dir, iwc); - iw.addDocument(new Document()); - iw.commit(); - iw.close(); - DirectoryReader ir = DirectoryReader.open(dir); - SegmentReader sr = (SegmentReader) ir.leaves().get(0).reader(); + SegmentReader sr = getSegmentReader(actual); String v = sr.getSegmentInfo().info.getAttribute(Lucene90StoredFieldsFormat.MODE_KEY); assertNotNull(v); assertEquals(expected, Lucene95Codec.Mode.valueOf(v)); - ir.close(); - dir.close(); } - private CodecService createCodecService() throws IOException { + private void assertStoredFieldsCompressionEquals(Lucene95CustomCodec.Mode expected, Codec actual) throws Exception { + SegmentReader sr = getSegmentReader(actual); + String v = sr.getSegmentInfo().info.getAttribute(Lucene95CustomStoredFieldsFormat.MODE_KEY); + assertNotNull(v); + assertEquals(expected, Lucene95CustomCodec.Mode.valueOf(v)); + } + + private CodecService createCodecService(boolean isMapperServiceNull) throws IOException { + + if (isMapperServiceNull) { + return new CodecService(null, LogManager.getLogger("test")); + } Settings nodeSettings = Settings.builder().put(Environment.PATH_HOME_SETTING.getKey(), createTempDir()).build(); IndexSettings settings = IndexSettingsModule.newIndexSettings("_na", nodeSettings); SimilarityService similarityService = new SimilarityService(settings, null, Collections.emptyMap()); @@ -115,4 +152,18 @@ private CodecService createCodecService() throws IOException { return new CodecService(service, LogManager.getLogger("test")); } + private SegmentReader getSegmentReader(Codec codec) throws IOException { + Directory dir = newDirectory(); + IndexWriterConfig iwc = newIndexWriterConfig(null); + iwc.setCodec(codec); + IndexWriter iw = new IndexWriter(dir, iwc); + iw.addDocument(new Document()); + iw.commit(); + iw.close(); + DirectoryReader ir = DirectoryReader.open(dir); + SegmentReader sr = (SegmentReader) ir.leaves().get(0).reader(); + ir.close(); + dir.close(); + return sr; + } } diff --git a/sandbox/plugins/custom-codecs/src/test/java/org/opensearch/index/codec/customcodecs/AbstractCompressorTests.java b/server/src/test/java/org/opensearch/index/codec/customcodecs/AbstractCompressorTests.java similarity index 100% rename from sandbox/plugins/custom-codecs/src/test/java/org/opensearch/index/codec/customcodecs/AbstractCompressorTests.java rename to server/src/test/java/org/opensearch/index/codec/customcodecs/AbstractCompressorTests.java index fcfb06ca6b050..cc794eb2c48f1 100644 --- a/sandbox/plugins/custom-codecs/src/test/java/org/opensearch/index/codec/customcodecs/AbstractCompressorTests.java +++ b/server/src/test/java/org/opensearch/index/codec/customcodecs/AbstractCompressorTests.java @@ -8,21 +8,21 @@ package org.opensearch.index.codec.customcodecs; -import org.apache.lucene.tests.util.LineFileDocs; -import org.apache.lucene.tests.util.TestUtil; -import org.opensearch.test.OpenSearchTestCase; import org.apache.lucene.codecs.compressing.Compressor; import org.apache.lucene.codecs.compressing.Decompressor; import org.apache.lucene.store.ByteArrayDataInput; import org.apache.lucene.store.ByteBuffersDataInput; import org.apache.lucene.store.ByteBuffersDataOutput; +import org.apache.lucene.tests.util.LineFileDocs; +import org.apache.lucene.tests.util.TestUtil; import org.apache.lucene.util.BytesRef; +import org.opensearch.test.OpenSearchTestCase; -import java.util.List; -import java.nio.ByteBuffer; import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; +import java.util.List; import java.util.Random; /** diff --git a/server/src/test/java/org/opensearch/index/codec/customcodecs/Lucene95CustomStoredFieldsFormatTests.java b/server/src/test/java/org/opensearch/index/codec/customcodecs/Lucene95CustomStoredFieldsFormatTests.java new file mode 100644 index 0000000000000..4f23450ce0b39 --- /dev/null +++ b/server/src/test/java/org/opensearch/index/codec/customcodecs/Lucene95CustomStoredFieldsFormatTests.java @@ -0,0 +1,27 @@ +/* + * 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.index.codec.customcodecs; + +import org.opensearch.test.OpenSearchTestCase; + +public class Lucene95CustomStoredFieldsFormatTests extends OpenSearchTestCase { + + public void testDefaultLucene95CustomCodecMode() { + Lucene95CustomStoredFieldsFormat lucene95CustomStoredFieldsFormat = new Lucene95CustomStoredFieldsFormat(); + assertEquals(Lucene95CustomCodec.Mode.ZSTD, lucene95CustomStoredFieldsFormat.getMode()); + } + + public void testZstdNoDictLucene95CustomCodecMode() { + Lucene95CustomStoredFieldsFormat lucene95CustomStoredFieldsFormat = new Lucene95CustomStoredFieldsFormat( + Lucene95CustomCodec.Mode.ZSTD_NO_DICT + ); + assertEquals(Lucene95CustomCodec.Mode.ZSTD_NO_DICT, lucene95CustomStoredFieldsFormat.getMode()); + } + +} diff --git a/sandbox/plugins/custom-codecs/src/test/java/org/opensearch/index/codec/customcodecs/ZstdCompressorTests.java b/server/src/test/java/org/opensearch/index/codec/customcodecs/ZstdCompressorTests.java similarity index 100% rename from sandbox/plugins/custom-codecs/src/test/java/org/opensearch/index/codec/customcodecs/ZstdCompressorTests.java rename to server/src/test/java/org/opensearch/index/codec/customcodecs/ZstdCompressorTests.java diff --git a/sandbox/plugins/custom-codecs/src/test/java/org/opensearch/index/codec/customcodecs/ZstdNoDictCompressorTests.java b/server/src/test/java/org/opensearch/index/codec/customcodecs/ZstdNoDictCompressorTests.java similarity index 100% rename from sandbox/plugins/custom-codecs/src/test/java/org/opensearch/index/codec/customcodecs/ZstdNoDictCompressorTests.java rename to server/src/test/java/org/opensearch/index/codec/customcodecs/ZstdNoDictCompressorTests.java From 86f955c2cd45bf8c4fa179c6ff317899dcab925e Mon Sep 17 00:00:00 2001 From: Austin Lee Date: Wed, 28 Jun 2023 17:58:11 -0700 Subject: [PATCH 088/109] Implement buildEmptyAggregations for MultiTermsAggregator (#7089) (#7318) * Implement buildEmptyAggregations for MultiTermsAggregator (#7089) Signed-off-by: Austin Lee * Address Spotless check issue Signed-off-by: Austin Lee * Add a unit test for MultiTermsAggregator.buildEmptyAggregations (#7089) Signed-off-by: Austin Lee * Update changelog Update version check and reason Signed-off-by: Austin Lee --------- Signed-off-by: Austin Lee --- CHANGELOG.md | 1 + .../search.aggregation/370_multi_terms.yml | 48 +++++++++++++ .../bucket/terms/MultiTermsAggregator.java | 15 +++- .../terms/MultiTermsAggregatorTests.java | 69 +++++++++++++++++++ 4 files changed, 132 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 139807f0bffd1..faa64a162cbd8 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -151,6 +151,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Enforce 512 byte document ID limit in bulk updates ([#8039](https://github.com/opensearch-project/OpenSearch/pull/8039)) - With only GlobalAggregation in request causes unnecessary wrapping with MultiCollector ([#8125](https://github.com/opensearch-project/OpenSearch/pull/8125)) - Fix mapping char_filter when mapping a hashtag ([#7591](https://github.com/opensearch-project/OpenSearch/pull/7591)) +- Fix NPE in multiterms aggregations involving empty buckets ([#7318](https://github.com/opensearch-project/OpenSearch/pull/7318)) ### Security diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/370_multi_terms.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/370_multi_terms.yml index 0f897866fcb9d..eeab8e78bf830 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/370_multi_terms.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/370_multi_terms.yml @@ -712,3 +712,51 @@ setup: - match: { aggregations.m_terms.buckets.0.key: ["a", 1] } - match: { aggregations.m_terms.buckets.0.key_as_string: "a|1" } - match: { aggregations.m_terms.buckets.0.doc_count: 4 } + +--- +"aggregate over multi-terms test": + - skip: + version: "- 2.9.99" + reason: "multi_terms aggregation was introduced in 2.1.0, NPE bug checked by this test case will manifest in any version < 3.0" + + - do: + bulk: + index: test_1 + refresh: true + body: + - '{"index": {}}' + - '{"str": "a", "ip": "127.0.0.1", "date": "2022-03-23"}' + - '{"index": {}}' + - '{"str": "a", "ip": "127.0.0.1", "date": "2022-03-25"}' + - '{"index": {}}' + - '{"str": "b", "ip": "127.0.0.1", "date": "2022-03-23"}' + - '{"index": {}}' + - '{"str": "b", "ip": "127.0.0.1", "date": "2022-03-25"}' + + - do: + search: + index: test_1 + size: 0 + body: + aggs: + histo: + date_histogram: + field: date + calendar_interval: day + aggs: + m_terms: + multi_terms: + terms: + - field: str + - field: ip + + - match: { hits.total.value: 4 } + - length: { aggregations.histo.buckets: 3 } + - match: { aggregations.histo.buckets.0.key_as_string: "2022-03-23T00:00:00.000Z" } + - match: { aggregations.histo.buckets.0.m_terms.buckets.0.key: ["a", "127.0.0.1"] } + - match: { aggregations.histo.buckets.0.m_terms.buckets.1.key: ["b", "127.0.0.1"] } + - match: { aggregations.histo.buckets.1.key_as_string: "2022-03-24T00:00:00.000Z" } + - length: { aggregations.histo.buckets.1.m_terms.buckets: 0 } + - match: { aggregations.histo.buckets.2.key_as_string: "2022-03-25T00:00:00.000Z" } + - match: { aggregations.histo.buckets.2.m_terms.buckets.0.key: [ "a", "127.0.0.1" ] } + - match: { aggregations.histo.buckets.2.m_terms.buckets.1.key: [ "b", "127.0.0.1" ] } diff --git a/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/MultiTermsAggregator.java b/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/MultiTermsAggregator.java index c810ba8f38624..fccb9c3af5986 100644 --- a/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/MultiTermsAggregator.java +++ b/server/src/main/java/org/opensearch/search/aggregations/bucket/terms/MultiTermsAggregator.java @@ -196,7 +196,20 @@ InternalMultiTerms buildResult(long owningBucketOrd, long otherDocCount, Interna @Override public InternalAggregation buildEmptyAggregation() { - return null; + return new InternalMultiTerms( + name, + order, + order, + bucketCountThresholds.getRequiredSize(), + bucketCountThresholds.getMinDocCount(), + metadata(), + bucketCountThresholds.getShardSize(), + showTermDocCountError, + 0, + 0, + formats, + Collections.emptyList() + ); } @Override diff --git a/server/src/test/java/org/opensearch/search/aggregations/bucket/terms/MultiTermsAggregatorTests.java b/server/src/test/java/org/opensearch/search/aggregations/bucket/terms/MultiTermsAggregatorTests.java index 75ad9e12e0776..a2792114e9529 100644 --- a/server/src/test/java/org/opensearch/search/aggregations/bucket/terms/MultiTermsAggregatorTests.java +++ b/server/src/test/java/org/opensearch/search/aggregations/bucket/terms/MultiTermsAggregatorTests.java @@ -28,6 +28,10 @@ import org.opensearch.common.network.InetAddresses; import org.opensearch.common.settings.Settings; import org.opensearch.common.time.DateFormatter; +import org.opensearch.common.util.BigArrays; +import org.opensearch.common.util.MockPageCacheRecycler; +import org.opensearch.index.IndexService; +import org.opensearch.index.cache.IndexCache; import org.opensearch.index.mapper.BooleanFieldMapper; import org.opensearch.index.mapper.DateFieldMapper; import org.opensearch.index.mapper.GeoPointFieldMapper; @@ -35,22 +39,32 @@ import org.opensearch.index.mapper.KeywordFieldMapper; import org.opensearch.index.mapper.MappedFieldType; import org.opensearch.index.mapper.NumberFieldMapper; +import org.opensearch.index.query.QueryShardContext; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.indices.breaker.NoneCircuitBreakerService; import org.opensearch.script.MockScriptEngine; import org.opensearch.script.Script; import org.opensearch.script.ScriptEngine; import org.opensearch.script.ScriptModule; import org.opensearch.script.ScriptService; import org.opensearch.script.ScriptType; +import org.opensearch.search.DocValueFormat; import org.opensearch.search.aggregations.AggregationBuilder; +import org.opensearch.search.aggregations.Aggregator; +import org.opensearch.search.aggregations.AggregatorFactories; import org.opensearch.search.aggregations.AggregatorTestCase; import org.opensearch.search.aggregations.BucketOrder; +import org.opensearch.search.aggregations.CardinalityUpperBound; +import org.opensearch.search.aggregations.InternalAggregation; import org.opensearch.search.aggregations.metrics.InternalMax; import org.opensearch.search.aggregations.metrics.MaxAggregationBuilder; import org.opensearch.search.aggregations.support.CoreValuesSourceType; import org.opensearch.search.aggregations.support.MultiTermsValuesSourceConfig; import org.opensearch.search.aggregations.support.ValueType; import org.opensearch.search.aggregations.support.ValuesSourceType; +import org.opensearch.search.internal.SearchContext; import org.opensearch.search.lookup.LeafDocLookup; +import org.opensearch.test.TestSearchContext; import java.io.IOException; import java.util.ArrayList; @@ -58,6 +72,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.UUID; import java.util.function.Consumer; import java.util.function.Function; @@ -68,8 +83,12 @@ import static java.util.stream.Collectors.toList; import static org.hamcrest.Matchers.closeTo; import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.instanceOf; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; public class MultiTermsAggregatorTests extends AggregatorTestCase { private static final String FIELD_NAME = "field"; @@ -852,6 +871,56 @@ public void testIncludeExclude() throws IOException { ); } + public void testEmptyAggregations() throws IOException { + QueryShardContext queryShardContext = mock(QueryShardContext.class); + IndexShard indexShard = mock(IndexShard.class); + BigArrays bigArrays = new BigArrays(new MockPageCacheRecycler(Settings.EMPTY), new NoneCircuitBreakerService(), ""); + IndexService indexService = mock(IndexService.class); + when(indexService.getShardOrNull(0)).thenReturn(indexShard); + IndexCache cache = mock(IndexCache.class); + when(cache.bitsetFilterCache()).thenReturn(null); + when(indexService.cache()).thenReturn(cache); + SearchContext context = new TestSearchContext(bigArrays, indexService); + when(indexService.newQueryShardContext(0, null, () -> 0L, null)).thenReturn(queryShardContext); + AggregatorFactories factories = AggregatorFactories.EMPTY; + boolean showTermDocCountError = true; + MultiTermsAggregator.InternalValuesSource internalValuesSources = mock(MultiTermsAggregator.InternalValuesSource.class); + DocValueFormat format = mock(DocValueFormat.class); + BucketOrder order = mock(BucketOrder.class); + Aggregator.SubAggCollectionMode collectMode = Aggregator.SubAggCollectionMode.BREADTH_FIRST; + TermsAggregator.BucketCountThresholds bucketCountThresholds = mock(TermsAggregator.BucketCountThresholds.class); + Aggregator parent = mock(Aggregator.class); + CardinalityUpperBound cardinality = CardinalityUpperBound.ONE; + Map metadata = new HashMap<>(); + String k1 = UUID.randomUUID().toString(); + String v1 = UUID.randomUUID().toString(); + metadata.put(k1, v1); + + MultiTermsAggregator mAgg = new MultiTermsAggregator( + AGG_NAME, + factories, + showTermDocCountError, + List.of(internalValuesSources), + List.of(format), + order, + collectMode, + bucketCountThresholds, + context, + parent, + cardinality, + metadata + ); + InternalAggregation emptyAgg = mAgg.buildEmptyAggregation(); + + MatcherAssert.assertThat(emptyAgg.getName(), equalTo(AGG_NAME)); + MatcherAssert.assertThat(emptyAgg, instanceOf(InternalMultiTerms.class)); + + InternalMultiTerms mt = (InternalMultiTerms) emptyAgg; + MatcherAssert.assertThat(mt.getMetadata().keySet(), contains(k1)); + MatcherAssert.assertThat(mt.getMetadata().get(k1), equalTo(v1)); + MatcherAssert.assertThat(mt.getBuckets(), empty()); + } + private void testAggregation( Query query, List terms, From b33979a4d95f728ee3080eb6983d93e024fb7ba2 Mon Sep 17 00:00:00 2001 From: Navneet Verma Date: Wed, 28 Jun 2023 18:01:17 -0700 Subject: [PATCH 089/109] Adding the SearchPhaseResultsProcessor interface in Search Pipeline (#7283) * Initial code for adding the SearchPhaseInjectorProcessor interface in Search Pipeline Signed-off-by: Navneet Verma * Pass PipelinedRequest to SearchAsyncActions We should resolve a search pipeline once at the start of a search request and then propagate that pipeline through the async actions. When completing a search phase, we will then use that pipeline to inject behavior (if applicable). Signed-off-by: Michael Froh * Renamed SearchPhaseInjectorProcessor to SearchPhaseResultsProcessor and fixed the comments Signed-off-by: Navneet Verma * Make PipelinedSearchRequest extend SearchRequest Rather than wrapping a SearchRequest in a PipelinedSearchRequest, changes are less intrusive if we say that a PipelinedSearchRequest "is a" SearchRequest. Signed-off-by: Michael Froh * Revert code change from merge conflict Signed-off-by: Michael Froh * Updated the changelog with more appropiate wording for the change. Signed-off-by: Navneet Verma * Fixed Typos in the code Signed-off-by: Navneet Verma * Fixing comments relating to return of SearchPhaseResults from processor Signed-off-by: Navneet Verma * Moved SearchPhaseName enum in separate class and fixed comments. Signed-off-by: Navneet Verma * Resolve remaining merge conflict Signed-off-by: Michael Froh --------- Signed-off-by: Navneet Verma Signed-off-by: Michael Froh Co-authored-by: Michael Froh Co-authored-by: Andrew Ross --- CHANGELOG.md | 1 + .../search_pipeline/50_script_processor.yml | 2 +- .../search/AbstractSearchAsyncAction.java | 7 +- .../search/ArraySearchPhaseResults.java | 2 +- .../search/CanMatchPreFilterSearchPhase.java | 2 +- .../action/search/DfsQueryPhase.java | 2 +- .../action/search/ExpandSearchPhase.java | 2 +- .../action/search/FetchSearchPhase.java | 2 +- .../opensearch/action/search/SearchPhase.java | 10 + .../action/search/SearchPhaseContext.java | 2 +- .../action/search/SearchPhaseName.java | 31 +++ .../action/search/SearchPhaseResults.java | 10 +- .../search/SearchScrollAsyncAction.java | 2 +- ...SearchScrollQueryThenFetchAsyncAction.java | 2 +- .../action/search/TransportSearchAction.java | 7 +- .../plugins/SearchPipelinePlugin.java | 12 + .../opensearch/search/pipeline/Pipeline.java | 33 ++- .../search/pipeline/PipelineWithMetrics.java | 24 +- .../search/pipeline/PipelinedRequest.java | 19 +- .../pipeline/SearchPhaseResultsProcessor.java | 47 ++++ .../pipeline/SearchPipelineService.java | 10 +- .../pipeline/SearchPipelineServiceTests.java | 219 ++++++++++++++++-- 22 files changed, 402 insertions(+), 46 deletions(-) create mode 100644 server/src/main/java/org/opensearch/action/search/SearchPhaseName.java create mode 100644 server/src/main/java/org/opensearch/search/pipeline/SearchPhaseResultsProcessor.java diff --git a/CHANGELOG.md b/CHANGELOG.md index faa64a162cbd8..93b2296475fd1 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -80,6 +80,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), ## [Unreleased 2.x] ### Added +- [SearchPipeline] Add new search pipeline processor type, SearchPhaseResultsProcessor, that can modify the result of one search phase before starting the next phase.([#7283](https://github.com/opensearch-project/OpenSearch/pull/7283)) - Add task cancellation monitoring service ([#7642](https://github.com/opensearch-project/OpenSearch/pull/7642)) - Add TokenManager Interface ([#7452](https://github.com/opensearch-project/OpenSearch/pull/7452)) - Add Remote store as a segment replication source ([#7653](https://github.com/opensearch-project/OpenSearch/pull/7653)) diff --git a/modules/search-pipeline-common/src/yamlRestTest/resources/rest-api-spec/test/search_pipeline/50_script_processor.yml b/modules/search-pipeline-common/src/yamlRestTest/resources/rest-api-spec/test/search_pipeline/50_script_processor.yml index 9b2dc0c41ff31..9d855e8a1861a 100644 --- a/modules/search-pipeline-common/src/yamlRestTest/resources/rest-api-spec/test/search_pipeline/50_script_processor.yml +++ b/modules/search-pipeline-common/src/yamlRestTest/resources/rest-api-spec/test/search_pipeline/50_script_processor.yml @@ -39,7 +39,7 @@ teardown: { "script" : { "lang" : "painless", - "source" : "ctx._source['size'] += 10; ctx._source['from'] -= 1; ctx._source['explain'] = !ctx._source['explain']; ctx._source['version'] = !ctx._source['version']; ctx._source['seq_no_primary_term'] = !ctx._source['seq_no_primary_term']; ctx._source['track_scores'] = !ctx._source['track_scores']; ctx._source['track_total_hits'] = 1; ctx._source['min_score'] -= 0.9; ctx._source['terminate_after'] += 2; ctx._source['profile'] = !ctx._source['profile'];" + "source" : "ctx._source['size'] += 10; ctx._source['from'] = ctx._source['from'] <= 0 ? ctx._source['from'] : ctx._source['from'] - 1 ; ctx._source['explain'] = !ctx._source['explain']; ctx._source['version'] = !ctx._source['version']; ctx._source['seq_no_primary_term'] = !ctx._source['seq_no_primary_term']; ctx._source['track_scores'] = !ctx._source['track_scores']; ctx._source['track_total_hits'] = 1; ctx._source['min_score'] -= 0.9; ctx._source['terminate_after'] += 2; ctx._source['profile'] = !ctx._source['profile'];" } } ] 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 969e0edbbc9d6..48fac9e8c8d38 100644 --- a/server/src/main/java/org/opensearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/opensearch/action/search/AbstractSearchAsyncAction.java @@ -57,6 +57,7 @@ import org.opensearch.search.internal.InternalSearchResponse; import org.opensearch.search.internal.SearchContext; import org.opensearch.search.internal.ShardSearchRequest; +import org.opensearch.search.pipeline.PipelinedRequest; import org.opensearch.transport.Transport; import java.util.ArrayDeque; @@ -696,7 +697,11 @@ private void raisePhaseFailure(SearchPhaseExecutionException exception) { * @see #onShardResult(SearchPhaseResult, SearchShardIterator) */ final void onPhaseDone() { // as a tribute to @kimchy aka. finishHim() - executeNextPhase(this, getNextPhase(results, this)); + final SearchPhase nextPhase = getNextPhase(results, this); + if (request instanceof PipelinedRequest && nextPhase != null) { + ((PipelinedRequest) request).transformSearchPhaseResults(results, this, this.getName(), nextPhase.getName()); + } + executeNextPhase(this, nextPhase); } @Override diff --git a/server/src/main/java/org/opensearch/action/search/ArraySearchPhaseResults.java b/server/src/main/java/org/opensearch/action/search/ArraySearchPhaseResults.java index 61c81e6cda97a..653b0e8aedb9d 100644 --- a/server/src/main/java/org/opensearch/action/search/ArraySearchPhaseResults.java +++ b/server/src/main/java/org/opensearch/action/search/ArraySearchPhaseResults.java @@ -66,7 +66,7 @@ boolean hasResult(int shardIndex) { } @Override - AtomicArray getAtomicArray() { + public AtomicArray getAtomicArray() { return results; } } diff --git a/server/src/main/java/org/opensearch/action/search/CanMatchPreFilterSearchPhase.java b/server/src/main/java/org/opensearch/action/search/CanMatchPreFilterSearchPhase.java index ec4d45a0a7124..c026c72f77f00 100644 --- a/server/src/main/java/org/opensearch/action/search/CanMatchPreFilterSearchPhase.java +++ b/server/src/main/java/org/opensearch/action/search/CanMatchPreFilterSearchPhase.java @@ -94,7 +94,7 @@ final class CanMatchPreFilterSearchPhase extends AbstractSearchAsyncAction, SearchPhase> nextPhaseFactory, SearchPhaseContext context ) { - super("dfs_query"); + super(SearchPhaseName.DFS_QUERY.getName()); this.progressListener = context.getTask().getProgressListener(); this.queryResult = queryResult; this.searchResults = searchResults; diff --git a/server/src/main/java/org/opensearch/action/search/ExpandSearchPhase.java b/server/src/main/java/org/opensearch/action/search/ExpandSearchPhase.java index cdefe7c2c1712..618a5620ce093 100644 --- a/server/src/main/java/org/opensearch/action/search/ExpandSearchPhase.java +++ b/server/src/main/java/org/opensearch/action/search/ExpandSearchPhase.java @@ -62,7 +62,7 @@ final class ExpandSearchPhase extends SearchPhase { private final AtomicArray queryResults; ExpandSearchPhase(SearchPhaseContext context, InternalSearchResponse searchResponse, AtomicArray queryResults) { - super("expand"); + super(SearchPhaseName.EXPAND.getName()); this.context = context; this.searchResponse = searchResponse; this.queryResults = queryResults; diff --git a/server/src/main/java/org/opensearch/action/search/FetchSearchPhase.java b/server/src/main/java/org/opensearch/action/search/FetchSearchPhase.java index 31ec896856ce6..85a3d140977bb 100644 --- a/server/src/main/java/org/opensearch/action/search/FetchSearchPhase.java +++ b/server/src/main/java/org/opensearch/action/search/FetchSearchPhase.java @@ -92,7 +92,7 @@ final class FetchSearchPhase extends SearchPhase { SearchPhaseContext context, BiFunction, SearchPhase> nextPhaseFactory ) { - super("fetch"); + super(SearchPhaseName.FETCH.getName()); if (context.getNumShards() != resultConsumer.getNumShards()) { throw new IllegalStateException( "number of shards must match the length of the query results but doesn't:" 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 50f0940754078..50b0cd8e01c1d 100644 --- a/server/src/main/java/org/opensearch/action/search/SearchPhase.java +++ b/server/src/main/java/org/opensearch/action/search/SearchPhase.java @@ -34,6 +34,7 @@ import org.opensearch.common.CheckedRunnable; import java.io.IOException; +import java.util.Locale; import java.util.Objects; /** @@ -54,4 +55,13 @@ protected SearchPhase(String name) { public String getName() { return name; } + + /** + * Returns the SearchPhase name as {@link SearchPhaseName}. Exception will come if SearchPhase name is not defined + * in {@link SearchPhaseName} + * @return {@link SearchPhaseName} + */ + public SearchPhaseName getSearchPhaseName() { + return SearchPhaseName.valueOf(name.toUpperCase(Locale.ROOT)); + } } 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 04b481249520b..018035f21179b 100644 --- a/server/src/main/java/org/opensearch/action/search/SearchPhaseContext.java +++ b/server/src/main/java/org/opensearch/action/search/SearchPhaseContext.java @@ -50,7 +50,7 @@ * * @opensearch.internal */ -interface SearchPhaseContext extends Executor { +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/SearchPhaseName.java b/server/src/main/java/org/opensearch/action/search/SearchPhaseName.java new file mode 100644 index 0000000000000..b6f842cf2cce1 --- /dev/null +++ b/server/src/main/java/org/opensearch/action/search/SearchPhaseName.java @@ -0,0 +1,31 @@ +/* + * 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.action.search; + +/** + * Enum for different Search Phases in OpenSearch + * @opensearch.internal + */ +public enum SearchPhaseName { + QUERY("query"), + FETCH("fetch"), + DFS_QUERY("dfs_query"), + EXPAND("expand"), + CAN_MATCH("can_match"); + + private final String name; + + SearchPhaseName(final String name) { + this.name = name; + } + + public String getName() { + return name; + } +} diff --git a/server/src/main/java/org/opensearch/action/search/SearchPhaseResults.java b/server/src/main/java/org/opensearch/action/search/SearchPhaseResults.java index 1baea0e721c44..2e6068b1ecddc 100644 --- a/server/src/main/java/org/opensearch/action/search/SearchPhaseResults.java +++ b/server/src/main/java/org/opensearch/action/search/SearchPhaseResults.java @@ -42,7 +42,7 @@ * * @opensearch.internal */ -abstract class SearchPhaseResults { +public abstract class SearchPhaseResults { private final int numShards; SearchPhaseResults(int numShards) { @@ -75,7 +75,13 @@ final int getNumShards() { void consumeShardFailure(int shardIndex) {} - AtomicArray getAtomicArray() { + /** + * Returns an {@link AtomicArray} of {@link Result}, which are nothing but the SearchPhaseResults + * for shards. The {@link Result} are of type {@link SearchPhaseResult} + * + * @return an {@link AtomicArray} of {@link Result} + */ + public AtomicArray getAtomicArray() { throw new UnsupportedOperationException(); } diff --git a/server/src/main/java/org/opensearch/action/search/SearchScrollAsyncAction.java b/server/src/main/java/org/opensearch/action/search/SearchScrollAsyncAction.java index 0b477624b15cc..899c7a3c1dabd 100644 --- a/server/src/main/java/org/opensearch/action/search/SearchScrollAsyncAction.java +++ b/server/src/main/java/org/opensearch/action/search/SearchScrollAsyncAction.java @@ -266,7 +266,7 @@ protected SearchPhase sendResponsePhase( SearchPhaseController.ReducedQueryPhase queryPhase, final AtomicArray fetchResults ) { - return new SearchPhase("fetch") { + return new SearchPhase(SearchPhaseName.FETCH.getName()) { @Override public void run() throws IOException { sendResponse(queryPhase, fetchResults); diff --git a/server/src/main/java/org/opensearch/action/search/SearchScrollQueryThenFetchAsyncAction.java b/server/src/main/java/org/opensearch/action/search/SearchScrollQueryThenFetchAsyncAction.java index 4119cb1cf28a0..9c0721ef63ea6 100644 --- a/server/src/main/java/org/opensearch/action/search/SearchScrollQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/opensearch/action/search/SearchScrollQueryThenFetchAsyncAction.java @@ -92,7 +92,7 @@ protected void executeInitialPhase( @Override protected SearchPhase moveToNextPhase(BiFunction clusterNodeLookup) { - return new SearchPhase("fetch") { + return new SearchPhase(SearchPhaseName.FETCH.getName()) { @Override public void run() { final SearchPhaseController.ReducedQueryPhase reducedQueryPhase = searchPhaseController.reducedScrollQueryPhase( 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 69f529fe1d00c..df2170cbe2af1 100644 --- a/server/src/main/java/org/opensearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/opensearch/action/search/TransportSearchAction.java @@ -390,13 +390,12 @@ private void executeRequest( relativeStartNanos, System::nanoTime ); - SearchRequest searchRequest; + PipelinedRequest searchRequest; ActionListener listener; try { - PipelinedRequest pipelinedRequest = searchPipelineService.resolvePipeline(originalSearchRequest); - searchRequest = pipelinedRequest.transformedRequest(); + searchRequest = searchPipelineService.resolvePipeline(originalSearchRequest); listener = ActionListener.wrap( - r -> originalListener.onResponse(pipelinedRequest.transformResponse(r)), + r -> originalListener.onResponse(searchRequest.transformResponse(r)), originalListener::onFailure ); } catch (Exception e) { diff --git a/server/src/main/java/org/opensearch/plugins/SearchPipelinePlugin.java b/server/src/main/java/org/opensearch/plugins/SearchPipelinePlugin.java index b8ceddecd3d20..3d76bab93a60c 100644 --- a/server/src/main/java/org/opensearch/plugins/SearchPipelinePlugin.java +++ b/server/src/main/java/org/opensearch/plugins/SearchPipelinePlugin.java @@ -9,6 +9,7 @@ package org.opensearch.plugins; import org.opensearch.search.pipeline.Processor; +import org.opensearch.search.pipeline.SearchPhaseResultsProcessor; import org.opensearch.search.pipeline.SearchRequestProcessor; import org.opensearch.search.pipeline.SearchResponseProcessor; @@ -42,4 +43,15 @@ default Map> getRequestProcess default Map> getResponseProcessors(Processor.Parameters parameters) { return Collections.emptyMap(); } + + /** + * Returns additional search pipeline search phase results processor types added by this plugin. + * + * The key of the returned {@link Map} is the unique name for the processor which is specified + * in pipeline configurations, and the value is a {@link org.opensearch.search.pipeline.Processor.Factory} + * to create the processor from a given pipeline configuration. + */ + default Map> getSearchPhaseResultsProcessors(Processor.Parameters parameters) { + return Collections.emptyMap(); + } } diff --git a/server/src/main/java/org/opensearch/search/pipeline/Pipeline.java b/server/src/main/java/org/opensearch/search/pipeline/Pipeline.java index 6f44daf48ed21..92826eee5a4f4 100644 --- a/server/src/main/java/org/opensearch/search/pipeline/Pipeline.java +++ b/server/src/main/java/org/opensearch/search/pipeline/Pipeline.java @@ -8,6 +8,8 @@ package org.opensearch.search.pipeline; +import org.opensearch.action.search.SearchPhaseContext; +import org.opensearch.action.search.SearchPhaseResults; import org.opensearch.action.search.SearchRequest; import org.opensearch.action.search.SearchResponse; import org.opensearch.common.Nullable; @@ -15,6 +17,7 @@ import org.opensearch.common.io.stream.NamedWriteableAwareStreamInput; import org.opensearch.common.io.stream.NamedWriteableRegistry; import org.opensearch.common.io.stream.StreamInput; +import org.opensearch.search.SearchPhaseResult; import java.util.Collections; import java.util.List; @@ -28,6 +31,7 @@ class Pipeline { public static final String REQUEST_PROCESSORS_KEY = "request_processors"; public static final String RESPONSE_PROCESSORS_KEY = "response_processors"; + public static final String PHASE_PROCESSORS_KEY = "phase_results_processors"; private final String id; private final String description; private final Integer version; @@ -36,7 +40,7 @@ class Pipeline { // Then these can be CompoundProcessors instead of lists. private final List searchRequestProcessors; private final List searchResponseProcessors; - + private final List searchPhaseResultsProcessors; private final NamedWriteableRegistry namedWriteableRegistry; private final LongSupplier relativeTimeSupplier; @@ -46,6 +50,7 @@ class Pipeline { @Nullable Integer version, List requestProcessors, List responseProcessors, + List phaseResultsProcessors, NamedWriteableRegistry namedWriteableRegistry, LongSupplier relativeTimeSupplier ) { @@ -54,6 +59,7 @@ class Pipeline { this.version = version; this.searchRequestProcessors = Collections.unmodifiableList(requestProcessors); this.searchResponseProcessors = Collections.unmodifiableList(responseProcessors); + this.searchPhaseResultsProcessors = Collections.unmodifiableList(phaseResultsProcessors); this.namedWriteableRegistry = namedWriteableRegistry; this.relativeTimeSupplier = relativeTimeSupplier; } @@ -78,6 +84,10 @@ List getSearchResponseProcessors() { return searchResponseProcessors; } + List getSearchPhaseResultsProcessors() { + return searchPhaseResultsProcessors; + } + protected void beforeTransformRequest() {} protected void afterTransformRequest(long timeInNanos) {} @@ -168,14 +178,33 @@ SearchResponse transformResponse(SearchRequest request, SearchResponse response) return response; } + void runSearchPhaseResultsTransformer( + SearchPhaseResults searchPhaseResult, + SearchPhaseContext context, + String currentPhase, + String nextPhase + ) throws SearchPipelineProcessingException { + + try { + for (SearchPhaseResultsProcessor searchPhaseResultsProcessor : searchPhaseResultsProcessors) { + if (currentPhase.equals(searchPhaseResultsProcessor.getBeforePhase().getName()) + && nextPhase.equals(searchPhaseResultsProcessor.getAfterPhase().getName())) { + searchPhaseResultsProcessor.process(searchPhaseResult, context); + } + } + } catch (RuntimeException e) { + throw new SearchPipelineProcessingException(e); + } + } + static final Pipeline NO_OP_PIPELINE = new Pipeline( SearchPipelineService.NOOP_PIPELINE_ID, "Pipeline that does not transform anything", 0, Collections.emptyList(), Collections.emptyList(), + Collections.emptyList(), null, () -> 0L ); - } diff --git a/server/src/main/java/org/opensearch/search/pipeline/PipelineWithMetrics.java b/server/src/main/java/org/opensearch/search/pipeline/PipelineWithMetrics.java index 662473f190006..612e979e56070 100644 --- a/server/src/main/java/org/opensearch/search/pipeline/PipelineWithMetrics.java +++ b/server/src/main/java/org/opensearch/search/pipeline/PipelineWithMetrics.java @@ -43,12 +43,22 @@ class PipelineWithMetrics extends Pipeline { Integer version, List requestProcessors, List responseProcessors, + List phaseResultsProcessors, NamedWriteableRegistry namedWriteableRegistry, OperationMetrics totalRequestMetrics, OperationMetrics totalResponseMetrics, LongSupplier relativeTimeSupplier ) { - super(id, description, version, requestProcessors, responseProcessors, namedWriteableRegistry, relativeTimeSupplier); + super( + id, + description, + version, + requestProcessors, + responseProcessors, + phaseResultsProcessors, + namedWriteableRegistry, + relativeTimeSupplier + ); this.totalRequestMetrics = totalRequestMetrics; this.totalResponseMetrics = totalResponseMetrics; for (Processor requestProcessor : getSearchRequestProcessors()) { @@ -64,6 +74,7 @@ static PipelineWithMetrics create( Map config, Map> requestProcessorFactories, Map> responseProcessorFactories, + Map> phaseResultsProcessorFactories, NamedWriteableRegistry namedWriteableRegistry, OperationMetrics totalRequestProcessingMetrics, OperationMetrics totalResponseProcessingMetrics @@ -79,6 +90,16 @@ static PipelineWithMetrics create( RESPONSE_PROCESSORS_KEY ); List responseProcessors = readProcessors(responseProcessorFactories, responseProcessorConfigs); + List> phaseResultsProcessorConfigs = ConfigurationUtils.readOptionalList( + null, + null, + config, + PHASE_PROCESSORS_KEY + ); + List phaseResultsProcessors = readProcessors( + phaseResultsProcessorFactories, + phaseResultsProcessorConfigs + ); if (config.isEmpty() == false) { throw new OpenSearchParseException( "pipeline [" @@ -93,6 +114,7 @@ static PipelineWithMetrics create( version, requestProcessors, responseProcessors, + phaseResultsProcessors, namedWriteableRegistry, totalRequestProcessingMetrics, totalResponseProcessingMetrics, diff --git a/server/src/main/java/org/opensearch/search/pipeline/PipelinedRequest.java b/server/src/main/java/org/opensearch/search/pipeline/PipelinedRequest.java index 0cfff013f4021..5a7539808c127 100644 --- a/server/src/main/java/org/opensearch/search/pipeline/PipelinedRequest.java +++ b/server/src/main/java/org/opensearch/search/pipeline/PipelinedRequest.java @@ -8,29 +8,36 @@ package org.opensearch.search.pipeline; +import org.opensearch.action.search.SearchPhaseContext; +import org.opensearch.action.search.SearchPhaseResults; import org.opensearch.action.search.SearchRequest; import org.opensearch.action.search.SearchResponse; +import org.opensearch.search.SearchPhaseResult; /** * Groups a search pipeline based on a request and the request after being transformed by the pipeline. * * @opensearch.internal */ -public final class PipelinedRequest { +public final class PipelinedRequest extends SearchRequest { private final Pipeline pipeline; - private final SearchRequest transformedRequest; PipelinedRequest(Pipeline pipeline, SearchRequest transformedRequest) { + super(transformedRequest); this.pipeline = pipeline; - this.transformedRequest = transformedRequest; } public SearchResponse transformResponse(SearchResponse response) { - return pipeline.transformResponse(transformedRequest, response); + return pipeline.transformResponse(this, response); } - public SearchRequest transformedRequest() { - return transformedRequest; + public void transformSearchPhaseResults( + final SearchPhaseResults searchPhaseResult, + final SearchPhaseContext searchPhaseContext, + final String currentPhase, + final String nextPhase + ) { + pipeline.runSearchPhaseResultsTransformer(searchPhaseResult, searchPhaseContext, currentPhase, nextPhase); } // Visible for testing diff --git a/server/src/main/java/org/opensearch/search/pipeline/SearchPhaseResultsProcessor.java b/server/src/main/java/org/opensearch/search/pipeline/SearchPhaseResultsProcessor.java new file mode 100644 index 0000000000000..772dc8758bace --- /dev/null +++ b/server/src/main/java/org/opensearch/search/pipeline/SearchPhaseResultsProcessor.java @@ -0,0 +1,47 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.search.pipeline; + +import org.opensearch.action.search.SearchPhaseContext; +import org.opensearch.action.search.SearchPhaseName; +import org.opensearch.action.search.SearchPhaseResults; +import org.opensearch.search.SearchPhaseResult; +import org.opensearch.search.internal.SearchContext; + +/** + * Creates a processor that runs between Phases of the Search. + * @opensearch.api + */ +public interface SearchPhaseResultsProcessor extends Processor { + + /** + * Processes the {@link SearchPhaseResults} obtained from a SearchPhase which will be returned to next + * SearchPhase. + * @param searchPhaseResult {@link SearchPhaseResults} + * @param searchPhaseContext {@link SearchContext} + * @param {@link SearchPhaseResult} + */ + void process( + final SearchPhaseResults searchPhaseResult, + final SearchPhaseContext searchPhaseContext + ); + + /** + * The phase which should have run before, this processor can start executing. + * @return {@link SearchPhaseName} + */ + SearchPhaseName getBeforePhase(); + + /** + * The phase which should run after, this processor execution. + * @return {@link SearchPhaseName} + */ + SearchPhaseName getAfterPhase(); + +} diff --git a/server/src/main/java/org/opensearch/search/pipeline/SearchPipelineService.java b/server/src/main/java/org/opensearch/search/pipeline/SearchPipelineService.java index 434c8fbfacc74..70dc8546a077f 100644 --- a/server/src/main/java/org/opensearch/search/pipeline/SearchPipelineService.java +++ b/server/src/main/java/org/opensearch/search/pipeline/SearchPipelineService.java @@ -73,6 +73,7 @@ public class SearchPipelineService implements ClusterStateApplier, ReportingServ private final ScriptService scriptService; private final Map> requestProcessorFactories; private final Map> responseProcessorFactories; + private final Map> phaseInjectorProcessorFactories; private volatile Map pipelines = Collections.emptyMap(); private final ThreadPool threadPool; private final List> searchPipelineClusterStateListeners = new CopyOnWriteArrayList<>(); @@ -116,6 +117,10 @@ public SearchPipelineService( ); this.requestProcessorFactories = processorFactories(searchPipelinePlugins, p -> p.getRequestProcessors(parameters)); this.responseProcessorFactories = processorFactories(searchPipelinePlugins, p -> p.getResponseProcessors(parameters)); + this.phaseInjectorProcessorFactories = processorFactories( + searchPipelinePlugins, + p -> p.getSearchPhaseResultsProcessors(parameters) + ); putPipelineTaskKey = clusterService.registerClusterManagerTask(ClusterManagerTaskKeys.PUT_SEARCH_PIPELINE_KEY, true); deletePipelineTaskKey = clusterService.registerClusterManagerTask(ClusterManagerTaskKeys.DELETE_SEARCH_PIPELINE_KEY, true); this.isEnabled = isEnabled; @@ -181,6 +186,7 @@ void innerUpdatePipelines(SearchPipelineMetadata newSearchPipelineMetadata) { newConfiguration.getConfigAsMap(), requestProcessorFactories, responseProcessorFactories, + phaseInjectorProcessorFactories, namedWriteableRegistry, totalRequestProcessingMetrics, totalResponseProcessingMetrics @@ -280,6 +286,7 @@ void validatePipeline(Map searchPipelineInfos pipelineConfig, requestProcessorFactories, responseProcessorFactories, + phaseInjectorProcessorFactories, namedWriteableRegistry, new OperationMetrics(), // Use ephemeral metrics for validation new OperationMetrics() @@ -359,7 +366,7 @@ static ClusterState innerDelete(DeleteSearchPipelineRequest request, ClusterStat return newState.build(); } - public PipelinedRequest resolvePipeline(SearchRequest searchRequest) throws Exception { + public PipelinedRequest resolvePipeline(SearchRequest searchRequest) { Pipeline pipeline = Pipeline.NO_OP_PIPELINE; if (isEnabled == false) { @@ -378,6 +385,7 @@ public PipelinedRequest resolvePipeline(SearchRequest searchRequest) throws Exce searchRequest.source().searchPipelineSource(), requestProcessorFactories, responseProcessorFactories, + phaseInjectorProcessorFactories, namedWriteableRegistry, totalRequestProcessingMetrics, totalResponseProcessingMetrics diff --git a/server/src/test/java/org/opensearch/search/pipeline/SearchPipelineServiceTests.java b/server/src/test/java/org/opensearch/search/pipeline/SearchPipelineServiceTests.java index 219dddff40b35..2ac0b2136ddd9 100644 --- a/server/src/test/java/org/opensearch/search/pipeline/SearchPipelineServiceTests.java +++ b/server/src/test/java/org/opensearch/search/pipeline/SearchPipelineServiceTests.java @@ -10,13 +10,22 @@ import org.apache.logging.log4j.Level; import org.apache.logging.log4j.LogManager; +import org.apache.lucene.search.ScoreDoc; +import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.TotalHits; import org.junit.Before; import org.opensearch.OpenSearchParseException; import org.opensearch.ResourceNotFoundException; import org.opensearch.Version; import org.opensearch.action.search.DeleteSearchPipelineRequest; +import org.opensearch.action.search.MockSearchPhaseContext; import org.opensearch.action.search.PutSearchPipelineRequest; +import org.opensearch.action.search.QueryPhaseResultConsumer; +import org.opensearch.action.search.SearchPhaseContext; +import org.opensearch.action.search.SearchPhaseController; +import org.opensearch.action.search.SearchPhaseName; +import org.opensearch.action.search.SearchPhaseResults; +import org.opensearch.action.search.SearchProgressListener; import org.opensearch.action.search.SearchRequest; import org.opensearch.action.search.SearchResponse; import org.opensearch.action.search.SearchResponseSections; @@ -28,10 +37,14 @@ import org.opensearch.cluster.metadata.Metadata; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.breaker.CircuitBreaker; +import org.opensearch.common.breaker.NoopCircuitBreaker; import org.opensearch.common.bytes.BytesArray; import org.opensearch.common.io.stream.NamedWriteableRegistry; +import org.opensearch.common.lucene.search.TopDocsAndMaxScore; import org.opensearch.common.metrics.OperationStats; import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.concurrent.AtomicArray; import org.opensearch.common.util.concurrent.OpenSearchExecutors; import org.opensearch.common.xcontent.XContentType; import org.opensearch.index.IndexSettings; @@ -40,7 +53,10 @@ import org.opensearch.search.SearchHit; import org.opensearch.search.SearchHits; import org.opensearch.search.SearchModule; +import org.opensearch.search.SearchPhaseResult; import org.opensearch.search.builder.SearchSourceBuilder; +import org.opensearch.search.query.QuerySearchResult; +import org.opensearch.test.InternalAggregationTestCase; import org.opensearch.test.MockLogAppender; import org.opensearch.test.OpenSearchTestCase; import org.opensearch.threadpool.ThreadPool; @@ -68,6 +84,13 @@ public Map> getRequestProcesso public Map> getResponseProcessors(Processor.Parameters parameters) { return Map.of("bar", (factories, tag, description, config) -> null); } + + @Override + public Map> getSearchPhaseResultsProcessors( + Processor.Parameters parameters + ) { + return Map.of("zoe", (factories, tag, description, config) -> null); + } }; private ThreadPool threadPool; @@ -178,13 +201,13 @@ public void testResolveIndexDefaultPipeline() throws Exception { SearchRequest searchRequest = new SearchRequest("my_index").source(SearchSourceBuilder.searchSource().size(5)); PipelinedRequest pipelinedRequest = service.resolvePipeline(searchRequest); assertEquals("p1", pipelinedRequest.getPipeline().getId()); - assertEquals(10, pipelinedRequest.transformedRequest().source().size()); + assertEquals(10, pipelinedRequest.source().size()); // Bypass the default pipeline searchRequest.pipeline("_none"); pipelinedRequest = service.resolvePipeline(searchRequest); assertEquals("_none", pipelinedRequest.getPipeline().getId()); - assertEquals(5, pipelinedRequest.transformedRequest().source().size()); + assertEquals(5, pipelinedRequest.source().size()); } private static abstract class FakeProcessor implements Processor { @@ -244,6 +267,40 @@ public SearchResponse processResponse(SearchRequest request, SearchResponse resp } } + private static class FakeSearchPhaseResultsProcessor extends FakeProcessor implements SearchPhaseResultsProcessor { + private Consumer querySearchResultConsumer; + + public FakeSearchPhaseResultsProcessor( + String type, + String tag, + String description, + Consumer querySearchResultConsumer + ) { + super(type, tag, description); + this.querySearchResultConsumer = querySearchResultConsumer; + } + + @Override + public void process( + SearchPhaseResults searchPhaseResult, + SearchPhaseContext searchPhaseContext + ) { + List resultAtomicArray = searchPhaseResult.getAtomicArray().asList(); + // updating the maxScore + resultAtomicArray.forEach(querySearchResultConsumer); + } + + @Override + public SearchPhaseName getBeforePhase() { + return SearchPhaseName.QUERY; + } + + @Override + public SearchPhaseName getAfterPhase() { + return SearchPhaseName.FETCH; + } + } + private SearchPipelineService createWithProcessors() { Map> requestProcessors = new HashMap<>(); requestProcessors.put("scale_request_size", (processorFactories, tag, description, config) -> { @@ -260,7 +317,15 @@ private SearchPipelineService createWithProcessors() { float score = ((Number) config.remove("score")).floatValue(); return new FakeResponseProcessor("fixed_score", tag, description, rsp -> rsp.getHits().forEach(h -> h.score(score))); }); - return createWithProcessors(requestProcessors, responseProcessors); + + Map> searchPhaseProcessors = new HashMap<>(); + searchPhaseProcessors.put("max_score", (processorFactories, tag, description, config) -> { + final float finalScore = config.containsKey("score") ? ((Number) config.remove("score")).floatValue() : 100f; + final Consumer querySearchResultConsumer = (result) -> result.queryResult().topDocs().maxScore = finalScore; + return new FakeSearchPhaseResultsProcessor("max_score", tag, description, querySearchResultConsumer); + }); + + return createWithProcessors(requestProcessors, responseProcessors, searchPhaseProcessors); } @Override @@ -271,7 +336,8 @@ protected NamedWriteableRegistry writableRegistry() { private SearchPipelineService createWithProcessors( Map> requestProcessors, - Map> responseProcessors + Map> responseProcessors, + Map> phaseProcessors ) { Client client = mock(Client.class); ThreadPool threadPool = mock(ThreadPool.class); @@ -296,6 +362,14 @@ public Map> getRequestProcesso public Map> getResponseProcessors(Processor.Parameters parameters) { return responseProcessors; } + + @Override + public Map> getSearchPhaseResultsProcessors( + Processor.Parameters parameters + ) { + return phaseProcessors; + } + }), client, true @@ -314,7 +388,8 @@ public void testUpdatePipelines() { new BytesArray( "{ " + "\"request_processors\" : [ { \"scale_request_size\": { \"scale\" : 2 } } ], " - + "\"response_processors\" : [ { \"fixed_score\" : { \"score\" : 1.0 } } ]" + + "\"response_processors\" : [ { \"fixed_score\" : { \"score\" : 1.0 } } ]," + + "\"phase_results_processors\" : [ { \"max_score\" : { \"score\": 100 } } ]" + "}" ), XContentType.JSON @@ -332,6 +407,11 @@ public void testUpdatePipelines() { "scale_request_size", searchPipelineService.getPipelines().get("_id").pipeline.getSearchRequestProcessors().get(0).getType() ); + assertEquals(1, searchPipelineService.getPipelines().get("_id").pipeline.getSearchPhaseResultsProcessors().size()); + assertEquals( + "max_score", + searchPipelineService.getPipelines().get("_id").pipeline.getSearchPhaseResultsProcessors().get(0).getType() + ); assertEquals(1, searchPipelineService.getPipelines().get("_id").pipeline.getSearchResponseProcessors().size()); assertEquals( "fixed_score", @@ -369,6 +449,7 @@ public void testPutPipeline() { assertEquals("empty pipeline", pipeline.pipeline.getDescription()); assertEquals(0, pipeline.pipeline.getSearchRequestProcessors().size()); assertEquals(0, pipeline.pipeline.getSearchResponseProcessors().size()); + assertEquals(0, pipeline.pipeline.getSearchPhaseResultsProcessors().size()); } public void testPutInvalidPipeline() throws IllegalAccessException { @@ -506,17 +587,14 @@ public void testTransformRequest() throws Exception { SearchRequest request = new SearchRequest("_index").source(sourceBuilder).pipeline("p1"); PipelinedRequest pipelinedRequest = searchPipelineService.resolvePipeline(request); - SearchRequest transformedRequest = pipelinedRequest.transformedRequest(); - assertEquals(2 * size, transformedRequest.source().size()); + assertEquals(2 * size, pipelinedRequest.source().size()); assertEquals(size, request.source().size()); // This request doesn't specify a pipeline, it doesn't get transformed. request = new SearchRequest("_index").source(sourceBuilder); pipelinedRequest = searchPipelineService.resolvePipeline(request); - SearchRequest notTransformedRequest = pipelinedRequest.transformedRequest(); - assertEquals(size, notTransformedRequest.source().size()); - assertSame(request, notTransformedRequest); + assertEquals(size, pipelinedRequest.source().size()); } public void testTransformResponse() throws Exception { @@ -565,6 +643,89 @@ public void testTransformResponse() throws Exception { } } + public void testTransformSearchPhase() { + SearchPipelineService searchPipelineService = createWithProcessors(); + SearchPipelineMetadata metadata = new SearchPipelineMetadata( + Map.of( + "p1", + new PipelineConfiguration( + "p1", + new BytesArray("{\"phase_results_processors\" : [ { \"max_score\" : { } } ]}"), + XContentType.JSON + ) + ) + ); + ClusterState clusterState = ClusterState.builder(new ClusterName("_name")).build(); + ClusterState previousState = clusterState; + clusterState = ClusterState.builder(clusterState) + .metadata(Metadata.builder().putCustom(SearchPipelineMetadata.TYPE, metadata)) + .build(); + searchPipelineService.applyClusterState(new ClusterChangedEvent("", clusterState, previousState)); + SearchPhaseController controller = new SearchPhaseController( + writableRegistry(), + s -> InternalAggregationTestCase.emptyReduceContextBuilder() + ); + SearchPhaseContext searchPhaseContext = new MockSearchPhaseContext(10); + QueryPhaseResultConsumer searchPhaseResults = new QueryPhaseResultConsumer( + searchPhaseContext.getRequest(), + OpenSearchExecutors.newDirectExecutorService(), + new NoopCircuitBreaker(CircuitBreaker.REQUEST), + controller, + SearchProgressListener.NOOP, + writableRegistry(), + 2, + exc -> {} + ); + + final QuerySearchResult querySearchResult = new QuerySearchResult(); + querySearchResult.setShardIndex(1); + querySearchResult.topDocs(new TopDocsAndMaxScore(new TopDocs(null, new ScoreDoc[1]), 1f), null); + searchPhaseResults.consumeResult(querySearchResult, () -> {}); + + // First try without specifying a pipeline, which should be a no-op. + SearchRequest searchRequest = new SearchRequest(); + PipelinedRequest pipelinedRequest = searchPipelineService.resolvePipeline(searchRequest); + AtomicArray notTransformedSearchPhaseResults = searchPhaseResults.getAtomicArray(); + pipelinedRequest.transformSearchPhaseResults( + searchPhaseResults, + searchPhaseContext, + SearchPhaseName.QUERY.getName(), + SearchPhaseName.FETCH.getName() + ); + assertSame(searchPhaseResults.getAtomicArray(), notTransformedSearchPhaseResults); + + // Now set the pipeline as p1 + searchRequest = new SearchRequest().pipeline("p1"); + pipelinedRequest = searchPipelineService.resolvePipeline(searchRequest); + + pipelinedRequest.transformSearchPhaseResults( + searchPhaseResults, + searchPhaseContext, + SearchPhaseName.QUERY.getName(), + SearchPhaseName.FETCH.getName() + ); + + List resultAtomicArray = searchPhaseResults.getAtomicArray().asList(); + assertEquals(1, resultAtomicArray.size()); + // updating the maxScore + for (SearchPhaseResult result : resultAtomicArray) { + assertEquals(100f, result.queryResult().topDocs().maxScore, 0); + } + + // Check Processor doesn't run for between other phases + searchRequest = new SearchRequest().pipeline("p1"); + pipelinedRequest = searchPipelineService.resolvePipeline(searchRequest); + AtomicArray notTransformedSearchPhaseResult = searchPhaseResults.getAtomicArray(); + pipelinedRequest.transformSearchPhaseResults( + searchPhaseResults, + searchPhaseContext, + SearchPhaseName.DFS_QUERY.getName(), + SearchPhaseName.QUERY.getName() + ); + + assertSame(searchPhaseResults.getAtomicArray(), notTransformedSearchPhaseResult); + } + public void testGetPipelines() { // assertEquals(0, SearchPipelineService.innerGetPipelines(null, "p1").size()); @@ -582,16 +743,23 @@ public void testGetPipelines() { "p2", new BytesArray("{\"response_processors\" : [ { \"fixed_score\": { \"score\" : 2 } } ] }"), XContentType.JSON + ), + "p3", + new PipelineConfiguration( + "p3", + new BytesArray("{\"phase_results_processors\" : [ { \"max_score\" : { } } ]}"), + XContentType.JSON ) ) ); // Return all when no ids specified List pipelines = SearchPipelineService.innerGetPipelines(metadata); - assertEquals(2, pipelines.size()); + assertEquals(3, pipelines.size()); pipelines.sort(Comparator.comparing(PipelineConfiguration::getId)); assertEquals("p1", pipelines.get(0).getId()); assertEquals("p2", pipelines.get(1).getId()); + assertEquals("p3", pipelines.get(2).getId()); // Get specific pipeline pipelines = SearchPipelineService.innerGetPipelines(metadata, "p1"); @@ -607,17 +775,19 @@ public void testGetPipelines() { // Match all pipelines = SearchPipelineService.innerGetPipelines(metadata, "*"); - assertEquals(2, pipelines.size()); + assertEquals(3, pipelines.size()); pipelines.sort(Comparator.comparing(PipelineConfiguration::getId)); assertEquals("p1", pipelines.get(0).getId()); assertEquals("p2", pipelines.get(1).getId()); + assertEquals("p3", pipelines.get(2).getId()); // Match prefix pipelines = SearchPipelineService.innerGetPipelines(metadata, "p*"); - assertEquals(2, pipelines.size()); + assertEquals(3, pipelines.size()); pipelines.sort(Comparator.comparing(PipelineConfiguration::getId)); assertEquals("p1", pipelines.get(0).getId()); assertEquals("p2", pipelines.get(1).getId()); + assertEquals("p3", pipelines.get(2).getId()); } public void testValidatePipeline() throws Exception { @@ -625,6 +795,7 @@ public void testValidatePipeline() throws Exception { ProcessorInfo reqProcessor = new ProcessorInfo("scale_request_size"); ProcessorInfo rspProcessor = new ProcessorInfo("fixed_score"); + ProcessorInfo injProcessor = new ProcessorInfo("max_score"); DiscoveryNode n1 = new DiscoveryNode("n1", buildNewFakeTransportAddress(), Version.CURRENT); DiscoveryNode n2 = new DiscoveryNode("n2", buildNewFakeTransportAddress(), Version.CURRENT); PutSearchPipelineRequest putRequest = new PutSearchPipelineRequest( @@ -632,7 +803,8 @@ public void testValidatePipeline() throws Exception { new BytesArray( "{" + "\"request_processors\": [{ \"scale_request_size\": { \"scale\" : 2 } }]," - + "\"response_processors\": [{ \"fixed_score\": { \"score\" : 2 } }]" + + "\"response_processors\": [{ \"fixed_score\": { \"score\" : 2 } }]," + + "\"phase_results_processors\" : [ { \"max_score\" : { } } ]" + "}" ), XContentType.JSON @@ -699,8 +871,7 @@ public void testInlinePipeline() throws Exception { assertEquals(1, pipeline.getSearchResponseProcessors().size()); // Verify that pipeline transforms request - SearchRequest transformedRequest = pipelinedRequest.transformedRequest(); - assertEquals(200, transformedRequest.source().size()); + assertEquals(200, pipelinedRequest.source().size()); int size = 10; SearchHit[] hits = new SearchHit[size]; @@ -730,7 +901,7 @@ public void testExceptionOnPipelineCreation() { "bad_factory", (pf, t, f, c) -> { throw new RuntimeException(); } ); - SearchPipelineService searchPipelineService = createWithProcessors(badFactory, Collections.emptyMap()); + SearchPipelineService searchPipelineService = createWithProcessors(badFactory, Collections.emptyMap(), Collections.emptyMap()); Map pipelineSourceMap = new HashMap<>(); pipelineSourceMap.put(Pipeline.REQUEST_PROCESSORS_KEY, List.of(Map.of("bad_factory", Collections.emptyMap()))); @@ -752,7 +923,11 @@ public void testExceptionOnRequestProcessing() { (pf, t, f, c) -> throwingRequestProcessor ); - SearchPipelineService searchPipelineService = createWithProcessors(throwingRequestProcessorFactory, Collections.emptyMap()); + SearchPipelineService searchPipelineService = createWithProcessors( + throwingRequestProcessorFactory, + Collections.emptyMap(), + Collections.emptyMap() + ); Map pipelineSourceMap = new HashMap<>(); pipelineSourceMap.put(Pipeline.REQUEST_PROCESSORS_KEY, List.of(Map.of("throwing_request", Collections.emptyMap()))); @@ -773,7 +948,11 @@ public void testExceptionOnResponseProcessing() throws Exception { (pf, t, f, c) -> throwingResponseProcessor ); - SearchPipelineService searchPipelineService = createWithProcessors(Collections.emptyMap(), throwingResponseProcessorFactory); + SearchPipelineService searchPipelineService = createWithProcessors( + Collections.emptyMap(), + throwingResponseProcessorFactory, + Collections.emptyMap() + ); Map pipelineSourceMap = new HashMap<>(); pipelineSourceMap.put(Pipeline.RESPONSE_PROCESSORS_KEY, List.of(Map.of("throwing_response", Collections.emptyMap()))); @@ -807,7 +986,7 @@ public void testStats() throws Exception { "throwing_response", (pf, t, f, c) -> throwingResponseProcessor ); - SearchPipelineService searchPipelineService = createWithProcessors(requestProcessors, responseProcessors); + SearchPipelineService searchPipelineService = createWithProcessors(requestProcessors, responseProcessors, Collections.emptyMap()); SearchPipelineMetadata metadata = new SearchPipelineMetadata( Map.of( From d5482f77679da2a8f72a7164e04b1c3d3370c90a Mon Sep 17 00:00:00 2001 From: Gaurav Bafna <85113518+gbbafna@users.noreply.github.com> Date: Thu, 29 Jun 2023 10:04:11 +0530 Subject: [PATCH 090/109] Handle case where IndexShard is null due to failed creation of index (#8237) Signed-off-by: Gaurav Bafna --- server/src/main/java/org/opensearch/index/IndexService.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/opensearch/index/IndexService.java b/server/src/main/java/org/opensearch/index/IndexService.java index 73797106bb66f..e6930b41088e2 100644 --- a/server/src/main/java/org/opensearch/index/IndexService.java +++ b/server/src/main/java/org/opensearch/index/IndexService.java @@ -603,7 +603,10 @@ public synchronized void removeShard(int shardId, String reason) { private void closeShard(String reason, ShardId sId, IndexShard indexShard, Store store, IndexEventListener listener) { final int shardId = sId.id(); final Settings indexSettings = this.getIndexSettings().getSettings(); - Store remoteStore = indexShard.remoteStore(); + Store remoteStore = null; + if (indexShard != null) { + remoteStore = indexShard.remoteStore(); + } if (store != null) { store.beforeClose(); } From 544b1cac97a848479f2355e11e1a2c18b40e8201 Mon Sep 17 00:00:00 2001 From: Rishab Nahata Date: Thu, 29 Jun 2023 10:44:02 +0530 Subject: [PATCH 091/109] Check UTF16 string size before converting to String to avoid OOME (#7963) * Check UTF16 string size before converting to string to avoid OOME Signed-off-by: Rishab Nahata --- CHANGELOG.md | 2 + .../common/bytes/AbstractBytesReference.java | 14 ++++++- .../bytes/AbstractBytesReferenceTestCase.java | 39 +++++++++++++++++++ 3 files changed, 54 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 93b2296475fd1..240c016524752 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -139,8 +139,10 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - [Snapshot Interop] Add Changes in Create Snapshot Flow for remote store interoperability. ([#7118](https://github.com/opensearch-project/OpenSearch/pull/7118)) - Allow insecure string settings to warn-log usage and advise to migration of a newer secure variant ([#5496](https://github.com/opensearch-project/OpenSearch/pull/5496)) - Add self-organizing hash table to improve the performance of bucket aggregations ([#7652](https://github.com/opensearch-project/OpenSearch/pull/7652)) +- Check UTF16 string size before converting to String to avoid OOME ([#7963](https://github.com/opensearch-project/OpenSearch/pull/7963)) - Move ZSTD compression codecs out of the sandbox ([#7908](https://github.com/opensearch-project/OpenSearch/pull/7908)) + ### Deprecated ### Removed diff --git a/libs/core/src/main/java/org/opensearch/common/bytes/AbstractBytesReference.java b/libs/core/src/main/java/org/opensearch/common/bytes/AbstractBytesReference.java index 7b3c71321e4f0..043d45223498e 100644 --- a/libs/core/src/main/java/org/opensearch/common/bytes/AbstractBytesReference.java +++ b/libs/core/src/main/java/org/opensearch/common/bytes/AbstractBytesReference.java @@ -33,6 +33,7 @@ import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRefIterator; +import org.apache.lucene.util.UnicodeUtil; import org.opensearch.common.io.stream.StreamInput; import org.opensearch.core.xcontent.XContentBuilder; @@ -49,6 +50,7 @@ public abstract class AbstractBytesReference implements BytesReference { private Integer hash = null; // we cache the hash of this reference since it can be quite costly to re-calculated it + private static final int MAX_UTF16_LENGTH = Integer.MAX_VALUE >> 1; @Override public int getInt(int index) { @@ -80,9 +82,19 @@ public void writeTo(OutputStream os) throws IOException { } } + protected int getMaxUTF16Length() { + return MAX_UTF16_LENGTH; + } + @Override public String utf8ToString() { - return toBytesRef().utf8ToString(); + BytesRef bytesRef = toBytesRef(); + final char[] ref = new char[bytesRef.length]; + final int len = UnicodeUtil.UTF8toUTF16(bytesRef, ref); + if (len > getMaxUTF16Length()) { + throw new IllegalArgumentException("UTF16 String size is " + len + ", should be less than " + getMaxUTF16Length()); + } + return new String(ref, 0, len); } @Override diff --git a/test/framework/src/main/java/org/opensearch/common/bytes/AbstractBytesReferenceTestCase.java b/test/framework/src/main/java/org/opensearch/common/bytes/AbstractBytesReferenceTestCase.java index dca46c37ca7d2..dd71711f9154c 100644 --- a/test/framework/src/main/java/org/opensearch/common/bytes/AbstractBytesReferenceTestCase.java +++ b/test/framework/src/main/java/org/opensearch/common/bytes/AbstractBytesReferenceTestCase.java @@ -478,6 +478,45 @@ public void testToUtf8() throws IOException { // TODO: good way to test? } + public void testUTF8toString_ExceedsMaxLength() { + AbstractBytesReference abr = new TestAbstractBytesReference(); + IllegalArgumentException e = assertThrows(IllegalArgumentException.class, abr::utf8ToString); + assertTrue(e.getMessage().contains("UTF16 String size is")); + assertTrue(e.getMessage().contains("should be less than")); + } + + static class TestAbstractBytesReference extends AbstractBytesReference { + @Override + public byte get(int index) { + return 0; + } + + @Override + public int length() { + return 0; + } + + @Override + public BytesReference slice(int from, int length) { + return null; + } + + @Override + public long ramBytesUsed() { + return 0; + } + + @Override + public BytesRef toBytesRef() { + return new BytesRef("UTF16 length exceed test"); + } + + @Override + public int getMaxUTF16Length() { + return 1; + } + } + public void testToBytesRef() throws IOException { int length = randomIntBetween(0, PAGE_SIZE); BytesReference pbr = newBytesReference(length); From 87a833fbedfbcf9029d5e14e0597f8c5e8968086 Mon Sep 17 00:00:00 2001 From: Chaitanya Gohel <104654647+gashutos@users.noreply.github.com> Date: Thu, 29 Jun 2023 17:14:30 +0530 Subject: [PATCH 092/109] Enable Point based optimization for custom comparators (#8168) * Enable Point based optimization for custom comparators Signed-off-by: gashutos * Simplifying enableSkipping logic in IndexField Signed-off-by: gashutos * Empty commit Signed-off-by: gashutos * Update server/src/main/java/org/opensearch/index/fielddata/IndexNumericFieldData.java Co-authored-by: Andrew Ross Signed-off-by: Chaitanya Gohel <104654647+gashutos@users.noreply.github.com> --------- Signed-off-by: gashutos Signed-off-by: Chaitanya Gohel <104654647+gashutos@users.noreply.github.com> Co-authored-by: Andrew Ross --- CHANGELOG.md | 1 + .../index/fielddata/IndexFieldData.java | 6 +++++ .../fielddata/IndexNumericFieldData.java | 25 +++++++++++++------ .../DoubleValuesComparatorSource.java | 2 +- .../FloatValuesComparatorSource.java | 2 +- .../IntValuesComparatorSource.java | 2 +- .../LongValuesComparatorSource.java | 2 +- .../UnsignedLongValuesComparatorSource.java | 2 +- 8 files changed, 30 insertions(+), 12 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 240c016524752..e1859437d1ee3 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -92,6 +92,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Make remote cluster connection setup in async ([#8038](https://github.com/opensearch-project/OpenSearch/pull/8038)) - Add API to initialize extensions ([#8029]()https://github.com/opensearch-project/OpenSearch/pull/8029) - Add distributed tracing framework ([#7543](https://github.com/opensearch-project/OpenSearch/issues/7543)) +- Enable Point based optimization for custom comparators ([#8168](https://github.com/opensearch-project/OpenSearch/pull/8168)) ### Dependencies - Bump `com.azure:azure-storage-common` from 12.21.0 to 12.21.1 (#7566, #7814) diff --git a/server/src/main/java/org/opensearch/index/fielddata/IndexFieldData.java b/server/src/main/java/org/opensearch/index/fielddata/IndexFieldData.java index 529405c90c9a4..36e6a242ecdec 100644 --- a/server/src/main/java/org/opensearch/index/fielddata/IndexFieldData.java +++ b/server/src/main/java/org/opensearch/index/fielddata/IndexFieldData.java @@ -120,11 +120,13 @@ abstract class XFieldComparatorSource extends FieldComparatorSource { protected final MultiValueMode sortMode; protected final Object missingValue; protected final Nested nested; + protected boolean enableSkipping; public XFieldComparatorSource(Object missingValue, MultiValueMode sortMode, Nested nested) { this.sortMode = sortMode; this.missingValue = missingValue; this.nested = nested; + this.enableSkipping = true; // true by default } public MultiValueMode sortMode() { @@ -135,6 +137,10 @@ public Nested nested() { return this.nested; } + public void disableSkipping() { + this.enableSkipping = false; + } + /** * Simple wrapper class around a filter that matches parent documents * and a filter that matches child documents. For every root document R, diff --git a/server/src/main/java/org/opensearch/index/fielddata/IndexNumericFieldData.java b/server/src/main/java/org/opensearch/index/fielddata/IndexNumericFieldData.java index 052a679de9765..ae8ffd8fe6b97 100644 --- a/server/src/main/java/org/opensearch/index/fielddata/IndexNumericFieldData.java +++ b/server/src/main/java/org/opensearch/index/fielddata/IndexNumericFieldData.java @@ -198,24 +198,35 @@ private XFieldComparatorSource comparatorSource( MultiValueMode sortMode, Nested nested ) { + final XFieldComparatorSource source; switch (targetNumericType) { case HALF_FLOAT: case FLOAT: - return new FloatValuesComparatorSource(this, missingValue, sortMode, nested); + source = new FloatValuesComparatorSource(this, missingValue, sortMode, nested); + break; case DOUBLE: - return new DoubleValuesComparatorSource(this, missingValue, sortMode, nested); + source = new DoubleValuesComparatorSource(this, missingValue, sortMode, nested); + break; case UNSIGNED_LONG: - return new UnsignedLongValuesComparatorSource(this, missingValue, sortMode, nested); + source = new UnsignedLongValuesComparatorSource(this, missingValue, sortMode, nested); + break; case DATE: - return dateComparatorSource(missingValue, sortMode, nested); + source = dateComparatorSource(missingValue, sortMode, nested); + break; case DATE_NANOSECONDS: - return dateNanosComparatorSource(missingValue, sortMode, nested); + source = dateNanosComparatorSource(missingValue, sortMode, nested); + break; case LONG: - return new LongValuesComparatorSource(this, missingValue, sortMode, nested); + source = new LongValuesComparatorSource(this, missingValue, sortMode, nested); + break; default: assert !targetNumericType.isFloatingPoint(); - return new IntValuesComparatorSource(this, missingValue, sortMode, nested); + source = new IntValuesComparatorSource(this, missingValue, sortMode, nested); } + if (targetNumericType != getNumericType()) { + source.disableSkipping(); // disable skipping logic for caste of sort field + } + return source; } protected XFieldComparatorSource dateComparatorSource(@Nullable Object missingValue, MultiValueMode sortMode, Nested nested) { diff --git a/server/src/main/java/org/opensearch/index/fielddata/fieldcomparator/DoubleValuesComparatorSource.java b/server/src/main/java/org/opensearch/index/fielddata/fieldcomparator/DoubleValuesComparatorSource.java index f8af86c904f2c..34e86070054c9 100644 --- a/server/src/main/java/org/opensearch/index/fielddata/fieldcomparator/DoubleValuesComparatorSource.java +++ b/server/src/main/java/org/opensearch/index/fielddata/fieldcomparator/DoubleValuesComparatorSource.java @@ -104,7 +104,7 @@ public FieldComparator newComparator(String fieldname, int numHits, boolean e final double dMissingValue = (Double) missingObject(missingValue, reversed); // NOTE: it's important to pass null as a missing value in the constructor so that // the comparator doesn't check docsWithField since we replace missing values in select() - return new DoubleComparator(numHits, null, null, reversed, false) { + return new DoubleComparator(numHits, fieldname, null, reversed, enableSkipping && this.enableSkipping) { @Override public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException { return new DoubleLeafComparator(context) { diff --git a/server/src/main/java/org/opensearch/index/fielddata/fieldcomparator/FloatValuesComparatorSource.java b/server/src/main/java/org/opensearch/index/fielddata/fieldcomparator/FloatValuesComparatorSource.java index 686bef479c179..04a34cd418520 100644 --- a/server/src/main/java/org/opensearch/index/fielddata/fieldcomparator/FloatValuesComparatorSource.java +++ b/server/src/main/java/org/opensearch/index/fielddata/fieldcomparator/FloatValuesComparatorSource.java @@ -97,7 +97,7 @@ public FieldComparator newComparator(String fieldname, int numHits, boolean e final float fMissingValue = (Float) missingObject(missingValue, reversed); // NOTE: it's important to pass null as a missing value in the constructor so that // the comparator doesn't check docsWithField since we replace missing values in select() - return new FloatComparator(numHits, null, null, reversed, false) { + return new FloatComparator(numHits, fieldname, null, reversed, enableSkipping && this.enableSkipping) { @Override public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException { return new FloatLeafComparator(context) { diff --git a/server/src/main/java/org/opensearch/index/fielddata/fieldcomparator/IntValuesComparatorSource.java b/server/src/main/java/org/opensearch/index/fielddata/fieldcomparator/IntValuesComparatorSource.java index 90afa5fc64c29..d5ea1eaf7263d 100644 --- a/server/src/main/java/org/opensearch/index/fielddata/fieldcomparator/IntValuesComparatorSource.java +++ b/server/src/main/java/org/opensearch/index/fielddata/fieldcomparator/IntValuesComparatorSource.java @@ -76,7 +76,7 @@ public FieldComparator newComparator(String fieldname, int numHits, boolean e final int iMissingValue = (Integer) missingObject(missingValue, reversed); // NOTE: it's important to pass null as a missing value in the constructor so that // the comparator doesn't check docsWithField since we replace missing values in select() - return new IntComparator(numHits, null, null, reversed, false) { + return new IntComparator(numHits, fieldname, null, reversed, enableSkipping && this.enableSkipping) { @Override public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException { return new IntLeafComparator(context) { diff --git a/server/src/main/java/org/opensearch/index/fielddata/fieldcomparator/LongValuesComparatorSource.java b/server/src/main/java/org/opensearch/index/fielddata/fieldcomparator/LongValuesComparatorSource.java index 462092dca5110..43e033dd59716 100644 --- a/server/src/main/java/org/opensearch/index/fielddata/fieldcomparator/LongValuesComparatorSource.java +++ b/server/src/main/java/org/opensearch/index/fielddata/fieldcomparator/LongValuesComparatorSource.java @@ -120,7 +120,7 @@ public FieldComparator newComparator(String fieldname, int numHits, boolean e final long lMissingValue = (Long) missingObject(missingValue, reversed); // NOTE: it's important to pass null as a missing value in the constructor so that // the comparator doesn't check docsWithField since we replace missing values in select() - return new LongComparator(numHits, null, null, reversed, false) { + return new LongComparator(numHits, fieldname, null, reversed, enableSkipping && this.enableSkipping) { @Override public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException { return new LongLeafComparator(context) { diff --git a/server/src/main/java/org/opensearch/index/fielddata/fieldcomparator/UnsignedLongValuesComparatorSource.java b/server/src/main/java/org/opensearch/index/fielddata/fieldcomparator/UnsignedLongValuesComparatorSource.java index d8b2e9528d2cf..be56b50179114 100644 --- a/server/src/main/java/org/opensearch/index/fielddata/fieldcomparator/UnsignedLongValuesComparatorSource.java +++ b/server/src/main/java/org/opensearch/index/fielddata/fieldcomparator/UnsignedLongValuesComparatorSource.java @@ -92,7 +92,7 @@ public FieldComparator newComparator(String fieldname, int numHits, boolean e assert indexFieldData == null || fieldname.equals(indexFieldData.getFieldName()); final BigInteger ulMissingValue = (BigInteger) missingObject(missingValue, reversed); - return new UnsignedLongComparator(numHits, null, null, reversed, false) { + return new UnsignedLongComparator(numHits, fieldname, null, reversed, enableSkipping && this.enableSkipping) { @Override public LeafFieldComparator getLeafComparator(LeafReaderContext context) throws IOException { return new UnsignedLongLeafComparator(context) { From 73b7a85d0a41580870a392aac9d47192adefa37a Mon Sep 17 00:00:00 2001 From: Sandesh Kumar Date: Thu, 29 Jun 2023 08:32:07 -0700 Subject: [PATCH 093/109] [Fix] Replace cached time with system clock in MasterService debug logs (#7902) * Replace cached time with system clock in MasterService debug logs Signed-off-by: Sandesh Kumar * Supply System.nanaoTime via TimeSupplier Signed-off-by: Sandesh Kumar * Add absolute time fetch method in Threadpool Signed-off-by: Sandesh Kumar * Rename absoluteTimeInNanos to preciseRelativeTimeInNanos Signed-off-by: Sandesh Kumar --------- Signed-off-by: Sandesh Kumar --- CHANGELOG.md | 1 + .../cluster/service/MasterService.java | 12 ++++---- .../org/opensearch/threadpool/ThreadPool.java | 13 +++++++++ .../cluster/service/MasterServiceTests.java | 28 +++++++++---------- 4 files changed, 34 insertions(+), 20 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index e1859437d1ee3..756f8a4652065 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -156,6 +156,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - With only GlobalAggregation in request causes unnecessary wrapping with MultiCollector ([#8125](https://github.com/opensearch-project/OpenSearch/pull/8125)) - Fix mapping char_filter when mapping a hashtag ([#7591](https://github.com/opensearch-project/OpenSearch/pull/7591)) - Fix NPE in multiterms aggregations involving empty buckets ([#7318](https://github.com/opensearch-project/OpenSearch/pull/7318)) +- Precise system clock time in MasterService debug logs ([#7902](https://github.com/opensearch-project/OpenSearch/pull/7902)) ### Security diff --git a/server/src/main/java/org/opensearch/cluster/service/MasterService.java b/server/src/main/java/org/opensearch/cluster/service/MasterService.java index 9712fdbfbe8ec..ffc6f81490c27 100644 --- a/server/src/main/java/org/opensearch/cluster/service/MasterService.java +++ b/server/src/main/java/org/opensearch/cluster/service/MasterService.java @@ -291,14 +291,14 @@ private void runTasks(TaskInputs taskInputs) { return; } - final long computationStartTime = threadPool.relativeTimeInMillis(); + final long computationStartTime = threadPool.preciseRelativeTimeInNanos(); final TaskOutputs taskOutputs = calculateTaskOutputs(taskInputs, previousClusterState); taskOutputs.notifyFailedTasks(); final TimeValue computationTime = getTimeSince(computationStartTime); logExecutionTime(computationTime, "compute cluster state update", summary); if (taskOutputs.clusterStateUnchanged()) { - final long notificationStartTime = threadPool.relativeTimeInMillis(); + final long notificationStartTime = threadPool.preciseRelativeTimeInNanos(); taskOutputs.notifySuccessfulTasksOnUnchangedClusterState(); final TimeValue executionTime = getTimeSince(notificationStartTime); logExecutionTime(executionTime, "notify listeners on unchanged cluster state", summary); @@ -309,7 +309,7 @@ private void runTasks(TaskInputs taskInputs) { } else { logger.debug("cluster state updated, version [{}], source [{}]", newClusterState.version(), summary); } - final long publicationStartTime = threadPool.relativeTimeInMillis(); + final long publicationStartTime = threadPool.preciseRelativeTimeInNanos(); try { ClusterChangedEvent clusterChangedEvent = new ClusterChangedEvent(summary, newClusterState, previousClusterState); // new cluster state, notify all listeners @@ -335,8 +335,8 @@ private void runTasks(TaskInputs taskInputs) { } } - private TimeValue getTimeSince(long startTimeMillis) { - return TimeValue.timeValueMillis(Math.max(0, threadPool.relativeTimeInMillis() - startTimeMillis)); + private TimeValue getTimeSince(long startTimeNanos) { + return TimeValue.timeValueMillis(TimeValue.nsecToMSec(threadPool.preciseRelativeTimeInNanos() - startTimeNanos)); } protected void publish(ClusterChangedEvent clusterChangedEvent, TaskOutputs taskOutputs, long startTimeMillis) { @@ -358,7 +358,7 @@ protected boolean blockingAllowed() { } void onPublicationSuccess(ClusterChangedEvent clusterChangedEvent, TaskOutputs taskOutputs) { - final long notificationStartTime = threadPool.relativeTimeInMillis(); + final long notificationStartTime = threadPool.preciseRelativeTimeInNanos(); taskOutputs.processedDifferentClusterState(clusterChangedEvent.previousState(), clusterChangedEvent.state()); try { diff --git a/server/src/main/java/org/opensearch/threadpool/ThreadPool.java b/server/src/main/java/org/opensearch/threadpool/ThreadPool.java index 2c91d5aa33090..ebc68c288e25a 100644 --- a/server/src/main/java/org/opensearch/threadpool/ThreadPool.java +++ b/server/src/main/java/org/opensearch/threadpool/ThreadPool.java @@ -330,6 +330,19 @@ public long relativeTimeInNanos() { return cachedTimeThread.relativeTimeInNanos(); } + /** + * Returns a value of nanoseconds that may be used for relative time calculations + * that require the highest precision possible. Performance critical code must use + * either {@link #relativeTimeInNanos()} or {@link #relativeTimeInMillis()} which + * give better performance at the cost of lower precision. + * + * This method should only be used for calculating time deltas. For an epoch based + * timestamp, see {@link #absoluteTimeInMillis()}. + */ + public long preciseRelativeTimeInNanos() { + return System.nanoTime(); + } + /** * Returns the value of milliseconds since UNIX epoch. * diff --git a/server/src/test/java/org/opensearch/cluster/service/MasterServiceTests.java b/server/src/test/java/org/opensearch/cluster/service/MasterServiceTests.java index fb47cb8e2d65a..3c27748daa87d 100644 --- a/server/src/test/java/org/opensearch/cluster/service/MasterServiceTests.java +++ b/server/src/test/java/org/opensearch/cluster/service/MasterServiceTests.java @@ -97,14 +97,14 @@ public class MasterServiceTests extends OpenSearchTestCase { private static ThreadPool threadPool; - private static long relativeTimeInMillis; + private static long timeDiffInMillis; @BeforeClass public static void createThreadPool() { threadPool = new TestThreadPool(MasterServiceTests.class.getName()) { @Override - public long relativeTimeInMillis() { - return relativeTimeInMillis; + public long preciseRelativeTimeInNanos() { + return timeDiffInMillis * TimeValue.NSEC_PER_MSEC; } }; } @@ -119,7 +119,7 @@ public static void stopThreadPool() { @Before public void randomizeCurrentTime() { - relativeTimeInMillis = randomLongBetween(0L, 1L << 62); + timeDiffInMillis = randomLongBetween(0L, 1L << 50); } private ClusterManagerService createClusterManagerService(boolean makeClusterManager) { @@ -426,7 +426,7 @@ public void testClusterStateUpdateLogging() throws Exception { clusterManagerService.submitStateUpdateTask("test1", new ClusterStateUpdateTask() { @Override public ClusterState execute(ClusterState currentState) { - relativeTimeInMillis += TimeValue.timeValueSeconds(1).millis(); + timeDiffInMillis += TimeValue.timeValueSeconds(1).millis(); return currentState; } @@ -441,7 +441,7 @@ public void onFailure(String source, Exception e) { clusterManagerService.submitStateUpdateTask("test2", new ClusterStateUpdateTask() { @Override public ClusterState execute(ClusterState currentState) { - relativeTimeInMillis += TimeValue.timeValueSeconds(2).millis(); + timeDiffInMillis += TimeValue.timeValueSeconds(2).millis(); throw new IllegalArgumentException("Testing handling of exceptions in the cluster state task"); } @@ -456,13 +456,13 @@ public void onFailure(String source, Exception e) {} clusterManagerService.submitStateUpdateTask("test3", new ClusterStateUpdateTask() { @Override public ClusterState execute(ClusterState currentState) { - relativeTimeInMillis += TimeValue.timeValueSeconds(3).millis(); + timeDiffInMillis += TimeValue.timeValueSeconds(3).millis(); return ClusterState.builder(currentState).incrementVersion().build(); } @Override public void clusterStateProcessed(String source, ClusterState oldState, ClusterState newState) { - relativeTimeInMillis += TimeValue.timeValueSeconds(4).millis(); + timeDiffInMillis += TimeValue.timeValueSeconds(4).millis(); } @Override @@ -1080,12 +1080,12 @@ public void testLongClusterStateUpdateLogging() throws Exception { final AtomicReference clusterStateRef = new AtomicReference<>(initialClusterState); clusterManagerService.setClusterStatePublisher((event, publishListener, ackListener) -> { if (event.source().contains("test5")) { - relativeTimeInMillis += ClusterManagerService.CLUSTER_MANAGER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING.get( + timeDiffInMillis += ClusterManagerService.CLUSTER_MANAGER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING.get( Settings.EMPTY ).millis() + randomLongBetween(1, 1000000); } if (event.source().contains("test6")) { - relativeTimeInMillis += ClusterManagerService.CLUSTER_MANAGER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING.get( + timeDiffInMillis += ClusterManagerService.CLUSTER_MANAGER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING.get( Settings.EMPTY ).millis() + randomLongBetween(1, 1000000); throw new OpenSearchException("simulated error during slow publication which should trigger logging"); @@ -1101,7 +1101,7 @@ public void testLongClusterStateUpdateLogging() throws Exception { clusterManagerService.submitStateUpdateTask("test1", new ClusterStateUpdateTask() { @Override public ClusterState execute(ClusterState currentState) { - relativeTimeInMillis += randomLongBetween( + timeDiffInMillis += randomLongBetween( 0L, ClusterManagerService.CLUSTER_MANAGER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING.get(Settings.EMPTY).millis() ); @@ -1124,7 +1124,7 @@ public void onFailure(String source, Exception e) { clusterManagerService.submitStateUpdateTask("test2", new ClusterStateUpdateTask() { @Override public ClusterState execute(ClusterState currentState) { - relativeTimeInMillis += ClusterManagerService.CLUSTER_MANAGER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING.get( + timeDiffInMillis += ClusterManagerService.CLUSTER_MANAGER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING.get( Settings.EMPTY ).millis() + randomLongBetween(1, 1000000); throw new IllegalArgumentException("Testing handling of exceptions in the cluster state task"); @@ -1143,7 +1143,7 @@ public void onFailure(String source, Exception e) { clusterManagerService.submitStateUpdateTask("test3", new ClusterStateUpdateTask() { @Override public ClusterState execute(ClusterState currentState) { - relativeTimeInMillis += ClusterManagerService.CLUSTER_MANAGER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING.get( + timeDiffInMillis += ClusterManagerService.CLUSTER_MANAGER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING.get( Settings.EMPTY ).millis() + randomLongBetween(1, 1000000); return ClusterState.builder(currentState).incrementVersion().build(); @@ -1162,7 +1162,7 @@ public void onFailure(String source, Exception e) { clusterManagerService.submitStateUpdateTask("test4", new ClusterStateUpdateTask() { @Override public ClusterState execute(ClusterState currentState) { - relativeTimeInMillis += ClusterManagerService.CLUSTER_MANAGER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING.get( + timeDiffInMillis += ClusterManagerService.CLUSTER_MANAGER_SERVICE_SLOW_TASK_LOGGING_THRESHOLD_SETTING.get( Settings.EMPTY ).millis() + randomLongBetween(1, 1000000); return currentState; From 22cb650e11e77bc86183d418fc99b00232559677 Mon Sep 17 00:00:00 2001 From: Andrey Pleskach Date: Thu, 29 Jun 2023 20:22:22 +0200 Subject: [PATCH 094/109] Fix HDFS fixture by excluding BouncyCastle (#8359) BouncyCastle excluded from the HDFS test fixture Signed-off-by: Andrey Pleskach --- test/fixtures/hdfs-fixture/build.gradle | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/test/fixtures/hdfs-fixture/build.gradle b/test/fixtures/hdfs-fixture/build.gradle index 310b088674d5c..b1a87fe6c3112 100644 --- a/test/fixtures/hdfs-fixture/build.gradle +++ b/test/fixtures/hdfs-fixture/build.gradle @@ -44,6 +44,7 @@ dependencies { exclude module: 'guava' exclude module: 'protobuf-java' exclude group: 'org.codehaus.jackson' + exclude group: "org.bouncycastle" } api "org.codehaus.jettison:jettison:${versions.jettison}" api "org.apache.commons:commons-compress:1.23.0" @@ -51,7 +52,6 @@ dependencies { api "org.apache.logging.log4j:log4j-core:${versions.log4j}" api "io.netty:netty-all:${versions.netty}" api 'com.google.code.gson:gson:2.10.1' - api "org.bouncycastle:bcpkix-jdk15to18:${versions.bouncycastle}" api "com.fasterxml.jackson.jaxrs:jackson-jaxrs-json-provider:${versions.jackson}" api "com.fasterxml.jackson.core:jackson-databind:${versions.jackson_databind}" api "com.fasterxml.woodstox:woodstox-core:${versions.woodstox}" @@ -65,4 +65,5 @@ dependencies { api "org.apache.commons:commons-text:1.10.0" api "commons-net:commons-net:3.9.0" runtimeOnly "com.google.guava:guava:${versions.guava}" + } From 51da85c1f48934cd23e2ff7cedd60d25de8cece3 Mon Sep 17 00:00:00 2001 From: Mohit Godwani <81609427+mgodwan@users.noreply.github.com> Date: Fri, 30 Jun 2023 01:13:55 +0530 Subject: [PATCH 095/109] Enable Fast Double Parser in Jackson (#7909) * Enable Fast Double Parser in Jackson Signed-off-by: Mohit Godwani * Use Fast Double Parser in jackson Signed-off-by: Mohit Godwani --------- Signed-off-by: Mohit Godwani Signed-off-by: Mohit Godwani <81609427+mgodwan@users.noreply.github.com> --- CHANGELOG.md | 1 + .../java/org/opensearch/common/xcontent/cbor/CborXContent.java | 2 ++ .../java/org/opensearch/common/xcontent/json/JsonXContent.java | 3 ++- .../org/opensearch/common/xcontent/smile/SmileXContent.java | 2 ++ .../java/org/opensearch/common/xcontent/yaml/YamlXContent.java | 2 ++ 5 files changed, 9 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 756f8a4652065..f75cc6e508e7a 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -50,6 +50,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Change http code for DecommissioningFailedException from 500 to 400 ([#5283](https://github.com/opensearch-project/OpenSearch/pull/5283)) - Improve summary error message for invalid setting updates ([#4792](https://github.com/opensearch-project/OpenSearch/pull/4792)) - Pass localNode info to all plugins on node start ([#7919](https://github.com/opensearch-project/OpenSearch/pull/7919)) +- Improved performance of parsing floating point numbers ([#7909](https://github.com/opensearch-project/OpenSearch/pull/7909)) ### Deprecated diff --git a/libs/x-content/src/main/java/org/opensearch/common/xcontent/cbor/CborXContent.java b/libs/x-content/src/main/java/org/opensearch/common/xcontent/cbor/CborXContent.java index d7df53e7a0cf5..46891b279ba43 100644 --- a/libs/x-content/src/main/java/org/opensearch/common/xcontent/cbor/CborXContent.java +++ b/libs/x-content/src/main/java/org/opensearch/common/xcontent/cbor/CborXContent.java @@ -36,6 +36,7 @@ import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.core.StreamReadConstraints; +import com.fasterxml.jackson.core.StreamReadFeature; import com.fasterxml.jackson.dataformat.cbor.CBORFactory; import org.opensearch.core.xcontent.DeprecationHandler; import org.opensearch.core.xcontent.MediaType; @@ -75,6 +76,7 @@ public static XContentBuilder contentBuilder() throws IOException { cborFactory.configure(JsonGenerator.Feature.AUTO_CLOSE_JSON_CONTENT, false); cborFactory.configure(JsonParser.Feature.STRICT_DUPLICATE_DETECTION, true); cborFactory.setStreamReadConstraints(StreamReadConstraints.builder().maxStringLength(DEFAULT_MAX_STRING_LEN).build()); + cborFactory.configure(StreamReadFeature.USE_FAST_DOUBLE_PARSER.mappedFeature(), true); cborXContent = new CborXContent(); } diff --git a/libs/x-content/src/main/java/org/opensearch/common/xcontent/json/JsonXContent.java b/libs/x-content/src/main/java/org/opensearch/common/xcontent/json/JsonXContent.java index 8ff8e7730b189..e6c27e4cf3eef 100644 --- a/libs/x-content/src/main/java/org/opensearch/common/xcontent/json/JsonXContent.java +++ b/libs/x-content/src/main/java/org/opensearch/common/xcontent/json/JsonXContent.java @@ -37,7 +37,7 @@ import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.core.StreamReadConstraints; - +import com.fasterxml.jackson.core.StreamReadFeature; import org.opensearch.core.xcontent.DeprecationHandler; import org.opensearch.core.xcontent.MediaType; import org.opensearch.core.xcontent.NamedXContentRegistry; @@ -78,6 +78,7 @@ public static XContentBuilder contentBuilder() throws IOException { jsonFactory.configure(JsonGenerator.Feature.AUTO_CLOSE_JSON_CONTENT, false); jsonFactory.configure(JsonParser.Feature.STRICT_DUPLICATE_DETECTION, true); jsonFactory.setStreamReadConstraints(StreamReadConstraints.builder().maxStringLength(DEFAULT_MAX_STRING_LEN).build()); + jsonFactory.configure(StreamReadFeature.USE_FAST_DOUBLE_PARSER.mappedFeature(), true); jsonXContent = new JsonXContent(); } diff --git a/libs/x-content/src/main/java/org/opensearch/common/xcontent/smile/SmileXContent.java b/libs/x-content/src/main/java/org/opensearch/common/xcontent/smile/SmileXContent.java index e0a39df1589a2..eb968556de8c9 100644 --- a/libs/x-content/src/main/java/org/opensearch/common/xcontent/smile/SmileXContent.java +++ b/libs/x-content/src/main/java/org/opensearch/common/xcontent/smile/SmileXContent.java @@ -36,6 +36,7 @@ import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.core.StreamReadConstraints; +import com.fasterxml.jackson.core.StreamReadFeature; import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.fasterxml.jackson.dataformat.smile.SmileGenerator; import org.opensearch.core.xcontent.DeprecationHandler; @@ -77,6 +78,7 @@ public static XContentBuilder contentBuilder() throws IOException { smileFactory.configure(JsonGenerator.Feature.AUTO_CLOSE_JSON_CONTENT, false); smileFactory.configure(JsonParser.Feature.STRICT_DUPLICATE_DETECTION, true); smileFactory.setStreamReadConstraints(StreamReadConstraints.builder().maxStringLength(DEFAULT_MAX_STRING_LEN).build()); + smileFactory.configure(StreamReadFeature.USE_FAST_DOUBLE_PARSER.mappedFeature(), true); smileXContent = new SmileXContent(); } diff --git a/libs/x-content/src/main/java/org/opensearch/common/xcontent/yaml/YamlXContent.java b/libs/x-content/src/main/java/org/opensearch/common/xcontent/yaml/YamlXContent.java index 1e73cb2bd9c5e..bb4fa9a09d448 100644 --- a/libs/x-content/src/main/java/org/opensearch/common/xcontent/yaml/YamlXContent.java +++ b/libs/x-content/src/main/java/org/opensearch/common/xcontent/yaml/YamlXContent.java @@ -35,6 +35,7 @@ import com.fasterxml.jackson.core.JsonEncoding; import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.core.StreamReadConstraints; +import com.fasterxml.jackson.core.StreamReadFeature; import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; import org.opensearch.core.xcontent.DeprecationHandler; import org.opensearch.core.xcontent.NamedXContentRegistry; @@ -70,6 +71,7 @@ public static XContentBuilder contentBuilder() throws IOException { yamlFactory = new YAMLFactory(); yamlFactory.configure(JsonParser.Feature.STRICT_DUPLICATE_DETECTION, true); yamlFactory.setStreamReadConstraints(StreamReadConstraints.builder().maxStringLength(DEFAULT_MAX_STRING_LEN).build()); + yamlFactory.configure(StreamReadFeature.USE_FAST_DOUBLE_PARSER.mappedFeature(), true); yamlXContent = new YamlXContent(); } From 7b436fb5f969b5e4ddff385e1dc7bd43ad3cacd8 Mon Sep 17 00:00:00 2001 From: Andriy Redko Date: Thu, 29 Jun 2023 16:52:07 -0400 Subject: [PATCH 096/109] Update search.aggregation/370_multi_terms.yml after backport (#8358) Signed-off-by: Andriy Redko --- .../rest-api-spec/test/search.aggregation/370_multi_terms.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/370_multi_terms.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/370_multi_terms.yml index eeab8e78bf830..7db5f31d8e761 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/370_multi_terms.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/370_multi_terms.yml @@ -716,7 +716,7 @@ setup: --- "aggregate over multi-terms test": - skip: - version: "- 2.9.99" + version: "- 2.8.99" reason: "multi_terms aggregation was introduced in 2.1.0, NPE bug checked by this test case will manifest in any version < 3.0" - do: From ed3124bb2eeed17cf0e7da026caa4c040c6b4906 Mon Sep 17 00:00:00 2001 From: Kunal Kotwani Date: Thu, 29 Jun 2023 15:44:17 -0700 Subject: [PATCH 097/109] Fix primary balance flaky test (#8366) Signed-off-by: Kunal Kotwani --- .../indices/replication/SegmentReplicationAllocationIT.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationAllocationIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationAllocationIT.java index 9f382d6a2e9ef..cbaf70c325a57 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationAllocationIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationAllocationIT.java @@ -96,7 +96,6 @@ public void testGlobalPrimaryAllocation() throws Exception { * This test in general passes without primary shard balance as well due to nature of allocation algorithm which * assigns all primary shards first followed by replica copies. */ - @AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/7751") public void testPerIndexPrimaryAllocation() throws Exception { internalCluster().startClusterManagerOnlyNode(); final int maxReplicaCount = 2; @@ -234,7 +233,7 @@ private void verifyPerIndexPrimaryBalance() throws Exception { RoutingNodes nodes = currentState.getRoutingNodes(); for (final Map.Entry index : currentState.getRoutingTable().indicesRouting().entrySet()) { final int totalPrimaryShards = index.getValue().primaryShardsActive(); - final int avgPrimaryShardsPerNode = (int) Math.ceil(totalPrimaryShards * 1f / currentState.getRoutingNodes().size()); + final int avgPrimaryShardsPerNode = (int) Math.floor(totalPrimaryShards * 1f / currentState.getRoutingNodes().size()); for (RoutingNode node : nodes) { final int primaryCount = node.shardsWithState(index.getKey(), STARTED) .stream() @@ -250,7 +249,8 @@ private void verifyPerIndexPrimaryBalance() throws Exception { avgPrimaryShardsPerNode ); } - assertTrue(primaryCount <= avgPrimaryShardsPerNode); + // Asserts value is within the variance threshold (-1/+1 of the average value). + assertTrue(avgPrimaryShardsPerNode - 1 <= primaryCount && primaryCount <= avgPrimaryShardsPerNode + 1); } } }, 60, TimeUnit.SECONDS); From 17ee1ce11d7b4382418454b9319679b9a5e4cbe9 Mon Sep 17 00:00:00 2001 From: Andriy Redko Date: Thu, 29 Jun 2023 21:40:17 -0400 Subject: [PATCH 098/109] Fix search.aggregation/60_empty.yml test case (fails on BWC runs) (#8351) Signed-off-by: Andriy Redko --- .../rest-api-spec/test/search.aggregation/60_empty.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/60_empty.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/60_empty.yml index 7b374e3f6a409..54f61d46b6f6c 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/60_empty.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search.aggregation/60_empty.yml @@ -1,5 +1,8 @@ --- "Empty aggs Body": + - skip: + version: "- 2.8.99" + reason: "the fix was introduced in 2.9.0" - do: index: index: test From a68733af8af7747fd0cc6c0d2704f082f172a1d9 Mon Sep 17 00:00:00 2001 From: Kunal Kotwani Date: Fri, 30 Jun 2023 10:34:22 -0700 Subject: [PATCH 099/109] Add lower and upper bound support for appropriate variance thresholds (#8372) Signed-off-by: Kunal Kotwani --- .../SegmentReplicationAllocationIT.java | 28 +++++++++++-------- 1 file changed, 17 insertions(+), 11 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationAllocationIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationAllocationIT.java index cbaf70c325a57..b6ea3a094f496 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationAllocationIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationAllocationIT.java @@ -233,24 +233,30 @@ private void verifyPerIndexPrimaryBalance() throws Exception { RoutingNodes nodes = currentState.getRoutingNodes(); for (final Map.Entry index : currentState.getRoutingTable().indicesRouting().entrySet()) { final int totalPrimaryShards = index.getValue().primaryShardsActive(); - final int avgPrimaryShardsPerNode = (int) Math.floor(totalPrimaryShards * 1f / currentState.getRoutingNodes().size()); + final int lowerBoundPrimaryShardsPerNode = (int) Math.floor(totalPrimaryShards * 1f / currentState.getRoutingNodes().size()) + - 1; + final int upperBoundPrimaryShardsPerNode = (int) Math.ceil(totalPrimaryShards * 1f / currentState.getRoutingNodes().size()) + + 1; for (RoutingNode node : nodes) { final int primaryCount = node.shardsWithState(index.getKey(), STARTED) .stream() .filter(ShardRouting::primary) .collect(Collectors.toList()) .size(); - if (primaryCount > avgPrimaryShardsPerNode) { - logger.info( - "--> Primary shard balance assertion failure for index {} on node {} {} <= {}", - index.getKey(), - node.node().getName(), - primaryCount, - avgPrimaryShardsPerNode - ); - } // Asserts value is within the variance threshold (-1/+1 of the average value). - assertTrue(avgPrimaryShardsPerNode - 1 <= primaryCount && primaryCount <= avgPrimaryShardsPerNode + 1); + assertTrue( + "--> Primary balance assertion failure for index " + + index + + "on node " + + node.node().getName() + + " " + + lowerBoundPrimaryShardsPerNode + + " <= " + + primaryCount + + " (assigned) <= " + + upperBoundPrimaryShardsPerNode, + lowerBoundPrimaryShardsPerNode <= primaryCount && primaryCount <= upperBoundPrimaryShardsPerNode + ); } } }, 60, TimeUnit.SECONDS); From 665d2379e9981851d40bab62cbefa0f04ff3e372 Mon Sep 17 00:00:00 2001 From: Kunal Kotwani Date: Fri, 30 Jun 2023 15:56:47 -0700 Subject: [PATCH 100/109] Fix version scipt for higher patch numbers (#8385) Signed-off-by: Kunal Kotwani --- .github/workflows/version.yml | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/.github/workflows/version.yml b/.github/workflows/version.yml index eaf4d085c6946..fdf42a9a2731e 100644 --- a/.github/workflows/version.yml +++ b/.github/workflows/version.yml @@ -30,7 +30,11 @@ jobs: CURRENT_VERSION_ARRAY[2]=$((CURRENT_VERSION_ARRAY[2]+1)) NEXT_VERSION=$(IFS=. ; echo "${CURRENT_VERSION_ARRAY[*]:0:3}") NEXT_VERSION_UNDERSCORE=$(IFS=_ ; echo "V_${CURRENT_VERSION_ARRAY[*]:0:3}") - NEXT_VERSION_ID=$(IFS=0 ; echo "${CURRENT_VERSION_ARRAY[*]:0:3}99") + if [[ ${#CURRENT_VERSION_ARRAY[2]} -gt 1 ]]; then + NEXT_VERSION_ID="${CURRENT_VERSION_ARRAY[0]:0:3}0${CURRENT_VERSION_ARRAY[1]:0:3}${CURRENT_VERSION_ARRAY[2]:0:3}99" + else + NEXT_VERSION_ID=$(IFS=0 ; echo "${CURRENT_VERSION_ARRAY[*]:0:3}99") + fi echo "TAG=$TAG" >> $GITHUB_ENV echo "BASE=$BASE" >> $GITHUB_ENV echo "BASE_X=$BASE_X" >> $GITHUB_ENV From 240193a8b129aaffffb39a57403264b8c4893efb Mon Sep 17 00:00:00 2001 From: Chaitanya Gohel <104654647+gashutos@users.noreply.github.com> Date: Sun, 2 Jul 2023 07:33:21 +0530 Subject: [PATCH 101/109] Avoid search_after short cutting in case of missing is specified (#8391) Signed-off-by: gashutos --- .../test/search/90_search_after.yml | 44 +++++++++++++++++++ .../org/opensearch/search/SearchService.java | 4 +- .../opensearch/search/SearchServiceTests.java | 17 +++++++ 3 files changed, 64 insertions(+), 1 deletion(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search/90_search_after.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search/90_search_after.yml index 65c1527a68b96..55e1566656faf 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search/90_search_after.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search/90_search_after.yml @@ -160,6 +160,28 @@ - match: {hits.hits.0._source.timestamp: "2019-10-21 00:30:04.828" } - match: {hits.hits.0.sort: [1571617804828] } + # search_after with the sort with missing + - do: + bulk: + refresh: true + index: test + body: | + {"index":{}} + {"timestamp": null} + - do: + search: + index: test + rest_total_hits_as_int: true + body: + "size": 5 + "sort": [ { "timestamp": { "order": "asc", "missing": "_last" } } ] + search_after: [ "2021-10-21 08:30:04.828" ] # making it out of min/max so only missing value hit is qualified + + - match: { hits.total: 3 } + - length: { hits.hits: 1 } + - match: { hits.hits.0._index: test } + - match: { hits.hits.0._source.timestamp: null } + --- "date_nanos": - skip: @@ -276,3 +298,25 @@ - match: {hits.hits.0._index: test } - match: {hits.hits.0._source.population: 15223372036854775800 } - match: {hits.hits.0.sort: [15223372036854775800] } + + # search_after with the sort with missing + - do: + bulk: + refresh: true + index: test + body: | + {"index":{}} + {"population": null} + - do: + search: + index: test + rest_total_hits_as_int: true + body: + "size": 5 + "sort": [ { "population": { "order": "asc", "missing": "_last" } } ] + search_after: [15223372036854775801] # making it out of min/max so only missing value hit is qualified + + - match: { hits.total: 3 } + - length: { hits.hits: 1 } + - match: { hits.hits.0._index: test } + - match: { hits.hits.0._source.population: null } diff --git a/server/src/main/java/org/opensearch/search/SearchService.java b/server/src/main/java/org/opensearch/search/SearchService.java index 7d67c6c3b45f4..9daad9112e473 100644 --- a/server/src/main/java/org/opensearch/search/SearchService.java +++ b/server/src/main/java/org/opensearch/search/SearchService.java @@ -1550,7 +1550,9 @@ private CanMatchResponse canMatch(ShardSearchRequest request, boolean checkRefre } public static boolean canMatchSearchAfter(FieldDoc searchAfter, MinAndMax minMax, FieldSortBuilder primarySortField) { - if (searchAfter != null && minMax != null && primarySortField != null) { + // Check for sort.missing == null, since in case of missing values sort queries, if segment/shard's min/max + // is out of search_after range, it still should be printed and hence we should not skip segment/shard. + if (searchAfter != null && minMax != null && primarySortField != null && primarySortField.missing() == null) { final Object searchAfterPrimary = searchAfter.fields[0]; if (primarySortField.order() == SortOrder.DESC) { if (minMax.compareMin(searchAfterPrimary) > 0) { diff --git a/server/src/test/java/org/opensearch/search/SearchServiceTests.java b/server/src/test/java/org/opensearch/search/SearchServiceTests.java index 8f8789a3a0323..74ef289c4b75f 100644 --- a/server/src/test/java/org/opensearch/search/SearchServiceTests.java +++ b/server/src/test/java/org/opensearch/search/SearchServiceTests.java @@ -1748,4 +1748,21 @@ public void testCanMatchSearchAfterDescEqualMin() throws IOException { primarySort.order(SortOrder.DESC); assertEquals(SearchService.canMatchSearchAfter(searchAfter, minMax, primarySort), true); } + + /** + * Test canMatchSearchAfter with missing value, even if min/max is out of range + * Min = 0L, Max = 9L, search_after = -1L + * Expected result is canMatch = true + */ + public void testCanMatchSearchAfterWithMissing() throws IOException { + FieldDoc searchAfter = new FieldDoc(0, 0, new Long[] { -1L }); + MinAndMax minMax = new MinAndMax(0L, 9L); + FieldSortBuilder primarySort = new FieldSortBuilder("test"); + primarySort.order(SortOrder.DESC); + // Should be false without missing values + assertEquals(SearchService.canMatchSearchAfter(searchAfter, minMax, primarySort), false); + primarySort.missing("_last"); + // Should be true with missing values + assertEquals(SearchService.canMatchSearchAfter(searchAfter, minMax, primarySort), true); + } } From 1d2274045b42e129b8c06748f8185d2431aa5336 Mon Sep 17 00:00:00 2001 From: Dharmesh Date: Mon, 3 Jul 2023 14:22:31 +0530 Subject: [PATCH 102/109] [Remote Store] Moving stale commit deletion to async flow (#8201) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --------- Signed-off-by: Dharmesh 💤 --- .../RemoteStoreBaseIntegTestCase.java | 8 ++ .../opensearch/remotestore/RemoteStoreIT.java | 39 ++++++++++ .../opensearch/index/shard/IndexShard.java | 2 +- .../shard/RemoteStoreRefreshListener.java | 13 +--- .../opensearch/index/shard/StoreRecovery.java | 9 ++- .../store/RemoteSegmentStoreDirectory.java | 44 ++++++++++- .../RemoteSegmentStoreDirectoryFactory.java | 8 +- .../main/java/org/opensearch/node/Node.java | 3 +- .../opensearch/index/IndexModuleTests.java | 2 +- ...moteSegmentStoreDirectoryFactoryTests.java | 5 +- .../RemoteSegmentStoreDirectoryTests.java | 73 ++++++++++++++++--- .../snapshots/SnapshotResiliencyTests.java | 2 +- .../index/shard/IndexShardTestCase.java | 2 +- 13 files changed, 175 insertions(+), 35 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java index d226d0d757638..336646b35b5a6 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java @@ -15,6 +15,7 @@ import org.opensearch.common.util.FeatureFlags; import org.opensearch.index.IndexModule; import org.opensearch.index.IndexSettings; +import org.opensearch.index.mapper.MapperService; import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.test.OpenSearchIntegTestCase; @@ -74,6 +75,13 @@ protected Settings remoteStoreIndexSettings(int numberOfReplicas) { return remoteStoreIndexSettings(numberOfReplicas, 1); } + protected Settings remoteStoreIndexSettings(int numberOfReplicas, long totalFieldLimit) { + return Settings.builder() + .put(remoteStoreIndexSettings(numberOfReplicas)) + .put(MapperService.INDEX_MAPPING_TOTAL_FIELDS_LIMIT_SETTING.getKey(), totalFieldLimit) + .build(); + } + protected Settings remoteTranslogIndexSettings(int numberOfReplicas, int numberOfShards) { return Settings.builder() .put(remoteStoreIndexSettings(numberOfReplicas, numberOfShards)) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java index 70a41d74a57c5..f6ba8cfed00d0 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java @@ -17,6 +17,7 @@ import org.opensearch.cluster.routing.RecoverySource; import org.opensearch.common.UUIDs; import org.opensearch.common.settings.Settings; +import org.opensearch.index.shard.RemoteStoreRefreshListener; import org.opensearch.indices.recovery.RecoveryState; import org.opensearch.plugins.Plugin; import org.opensearch.test.InternalTestCluster; @@ -277,4 +278,42 @@ public void testRemoteSegmentCleanup() throws Exception { public void testRemoteTranslogCleanup() throws Exception { verifyRemoteStoreCleanup(true); } + + public void testStaleCommitDeletionWithInvokeFlush() throws Exception { + internalCluster().startDataOnlyNodes(3); + createIndex(INDEX_NAME, remoteStoreIndexSettings(1, 10000l)); + int numberOfIterations = randomIntBetween(5, 15); + indexData(numberOfIterations, true); + String indexUUID = client().admin() + .indices() + .prepareGetSettings(INDEX_NAME) + .get() + .getSetting(INDEX_NAME, IndexMetadata.SETTING_INDEX_UUID); + Path indexPath = Path.of(String.valueOf(absolutePath), indexUUID, "/0/segments/metadata"); + // Delete is async. + assertBusy(() -> { + int actualFileCount = getFileCount(indexPath); + if (numberOfIterations <= RemoteStoreRefreshListener.LAST_N_METADATA_FILES_TO_KEEP) { + assertEquals(numberOfIterations, actualFileCount); + } else { + // As delete is async its possible that the file gets created before the deletion or after + // deletion. + assertTrue(actualFileCount >= 10 || actualFileCount <= 11); + } + }, 30, TimeUnit.SECONDS); + } + + public void testStaleCommitDeletionWithoutInvokeFlush() throws Exception { + internalCluster().startDataOnlyNodes(3); + createIndex(INDEX_NAME, remoteStoreIndexSettings(1, 10000l)); + int numberOfIterations = randomIntBetween(5, 15); + indexData(numberOfIterations, false); + String indexUUID = client().admin() + .indices() + .prepareGetSettings(INDEX_NAME) + .get() + .getSetting(INDEX_NAME, IndexMetadata.SETTING_INDEX_UUID); + Path indexPath = Path.of(String.valueOf(absolutePath), indexUUID, "/0/segments/metadata"); + assertEquals(1, getFileCount(indexPath)); + } } diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index d89d51c713d70..01c0a12d463ea 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -2632,7 +2632,7 @@ public void restoreFromSnapshotAndRemoteStore( assert recoveryState.getRecoverySource().getType() == RecoverySource.Type.SNAPSHOT : "invalid recovery type: " + recoveryState.getRecoverySource(); StoreRecovery storeRecovery = new StoreRecovery(shardId, logger); - storeRecovery.recoverFromSnapshotAndRemoteStore(this, repository, repositoriesService, listener); + storeRecovery.recoverFromSnapshotAndRemoteStore(this, repository, repositoriesService, listener, threadPool); } catch (Exception e) { listener.onFailure(e); } diff --git a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java index 7cfaaafcadd39..ddca12d9283f3 100644 --- a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java +++ b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java @@ -85,7 +85,7 @@ public final class RemoteStoreRefreshListener implements ReferenceManager.Refres // Visible for testing static final Set EXCLUDE_FILES = Set.of("write.lock"); // Visible for testing - static final int LAST_N_METADATA_FILES_TO_KEEP = 10; + public static final int LAST_N_METADATA_FILES_TO_KEEP = 10; private final IndexShard indexShard; private final Directory storeDirectory; @@ -200,9 +200,8 @@ private synchronized void syncSegments(boolean isRetry) { // if a new segments_N file is present in local that is not uploaded to remote store yet, it // is considered as a first refresh post commit. A cleanup of stale commit files is triggered. // This is done to avoid delete post each refresh. - // Ideally, we want this to be done in async flow. (GitHub issue #4315) if (isRefreshAfterCommit()) { - deleteStaleCommits(); + remoteDirectory.deleteStaleSegmentsAsync(LAST_N_METADATA_FILES_TO_KEEP); } try (GatedCloseable segmentInfosGatedCloseable = indexShard.getSegmentInfosSnapshot()) { @@ -381,14 +380,6 @@ private String getChecksumOfLocalFile(String file) throws IOException { return localSegmentChecksumMap.get(file); } - private void deleteStaleCommits() { - try { - remoteDirectory.deleteStaleSegments(LAST_N_METADATA_FILES_TO_KEEP); - } catch (IOException e) { - logger.info("Exception while deleting stale commits from remote segment store, will retry delete post next commit", e); - } - } - /** * Updates the last refresh time and refresh seq no which is seen by local store. */ 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 119524e8caf8a..da4e9113143af 100644 --- a/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java +++ b/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java @@ -69,6 +69,7 @@ import org.opensearch.repositories.IndexId; import org.opensearch.repositories.RepositoriesService; import org.opensearch.repositories.Repository; +import org.opensearch.threadpool.ThreadPool; import java.io.IOException; import java.nio.channels.FileChannel; @@ -356,7 +357,8 @@ void recoverFromSnapshotAndRemoteStore( final IndexShard indexShard, Repository repository, RepositoriesService repositoriesService, - ActionListener listener + ActionListener listener, + ThreadPool threadPool ) { try { if (canRecover(indexShard)) { @@ -384,7 +386,10 @@ void recoverFromSnapshotAndRemoteStore( remoteStoreRepository = shallowCopyShardMetadata.getRemoteStoreRepository(); } - RemoteSegmentStoreDirectoryFactory directoryFactory = new RemoteSegmentStoreDirectoryFactory(() -> repositoriesService); + RemoteSegmentStoreDirectoryFactory directoryFactory = new RemoteSegmentStoreDirectoryFactory( + () -> repositoriesService, + threadPool + ); RemoteSegmentStoreDirectory sourceRemoteDirectory = (RemoteSegmentStoreDirectory) directoryFactory.newDirectory( remoteStoreRepository, indexUUID, diff --git a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java index addd8a24af9c5..ac129aca8baf7 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java @@ -27,6 +27,7 @@ import org.opensearch.index.store.lockmanager.RemoteStoreLockManager; import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadata; import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadataHandler; +import org.opensearch.threadpool.ThreadPool; import java.io.FileNotFoundException; import java.io.IOException; @@ -75,6 +76,8 @@ public final class RemoteSegmentStoreDirectory extends FilterDirectory implement private final RemoteStoreLockManager mdLockManager; + private final ThreadPool threadPool; + /** * To prevent explosion of refresh metadata files, we replace refresh files for the given primary term and generation * This is achieved by uploading refresh metadata file with the same UUID suffix. @@ -96,15 +99,23 @@ public final class RemoteSegmentStoreDirectory extends FilterDirectory implement private static final Logger logger = LogManager.getLogger(RemoteSegmentStoreDirectory.class); + /** + * AtomicBoolean that ensures only one staleCommitDeletion activity is scheduled at a time. + * Visible for testing + */ + protected final AtomicBoolean canDeleteStaleCommits = new AtomicBoolean(true); + public RemoteSegmentStoreDirectory( RemoteDirectory remoteDataDirectory, RemoteDirectory remoteMetadataDirectory, - RemoteStoreLockManager mdLockManager + RemoteStoreLockManager mdLockManager, + ThreadPool threadPool ) throws IOException { super(remoteDataDirectory); this.remoteDataDirectory = remoteDataDirectory; this.remoteMetadataDirectory = remoteMetadataDirectory; this.mdLockManager = mdLockManager; + this.threadPool = threadPool; init(); } @@ -574,7 +585,7 @@ public Map getSegmentsUploadedToRemoteStore(lon * @param lastNMetadataFilesToKeep number of metadata files to keep * @throws IOException in case of I/O error while reading from / writing to remote segment store */ - public void deleteStaleSegments(int lastNMetadataFilesToKeep) throws IOException { + private void deleteStaleSegments(int lastNMetadataFilesToKeep) throws IOException { Collection metadataFiles = remoteMetadataDirectory.listFilesByPrefix(MetadataFilenameUtils.METADATA_PREFIX); List sortedMetadataFileList = metadataFiles.stream().sorted(METADATA_FILENAME_COMPARATOR).collect(Collectors.toList()); if (sortedMetadataFileList.size() <= lastNMetadataFilesToKeep) { @@ -656,6 +667,33 @@ public void deleteStaleSegments(int lastNMetadataFilesToKeep) throws IOException } } + /** + * Delete stale segment and metadata files asynchronously. + * This method calls {@link RemoteSegmentStoreDirectory#deleteStaleSegments(int)} in an async manner. + * @param lastNMetadataFilesToKeep number of metadata files to keep + */ + public void deleteStaleSegmentsAsync(int lastNMetadataFilesToKeep) { + if (canDeleteStaleCommits.compareAndSet(true, false)) { + try { + threadPool.executor(ThreadPool.Names.REMOTE_PURGE).execute(() -> { + try { + deleteStaleSegments(lastNMetadataFilesToKeep); + } catch (Exception e) { + logger.info( + "Exception while deleting stale commits from remote segment store, will retry delete post next commit", + e + ); + } finally { + canDeleteStaleCommits.set(true); + } + }); + } catch (Exception e) { + logger.info("Exception occurred while scheduling deleteStaleCommits", e); + canDeleteStaleCommits.set(true); + } + } + } + /* Tries to delete shard level directory if it is empty Return true if it deleted it successfully @@ -680,7 +718,7 @@ private boolean deleteIfEmpty() throws IOException { } public void close() throws IOException { - deleteStaleSegments(0); + deleteStaleSegmentsAsync(0); deleteIfEmpty(); } } diff --git a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java index 03995d5913fb3..3bec84f287ce4 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactory.java @@ -20,6 +20,7 @@ import org.opensearch.repositories.Repository; import org.opensearch.repositories.RepositoryMissingException; import org.opensearch.repositories.blobstore.BlobStoreRepository; +import org.opensearch.threadpool.ThreadPool; import java.io.IOException; import java.util.function.Supplier; @@ -34,8 +35,11 @@ public class RemoteSegmentStoreDirectoryFactory implements IndexStorePlugin.Dire private final Supplier repositoriesService; - public RemoteSegmentStoreDirectoryFactory(Supplier repositoriesService) { + private final ThreadPool threadPool; + + public RemoteSegmentStoreDirectoryFactory(Supplier repositoriesService, ThreadPool threadPool) { this.repositoriesService = repositoriesService; + this.threadPool = threadPool; } @Override @@ -62,7 +66,7 @@ public Directory newDirectory(String repositoryName, String indexUUID, String sh shardId ); - return new RemoteSegmentStoreDirectory(dataDirectory, metadataDirectory, mdLockManager); + return new RemoteSegmentStoreDirectory(dataDirectory, metadataDirectory, mdLockManager, threadPool); } catch (RepositoryMissingException e) { throw new IllegalArgumentException("Repository should be created before creating index with remote_store enabled setting", e); } diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index 3742c817118da..d3655671b516d 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -717,7 +717,8 @@ protected Node( clusterService.setRerouteService(rerouteService); final IndexStorePlugin.DirectoryFactory remoteDirectoryFactory = new RemoteSegmentStoreDirectoryFactory( - repositoriesServiceReference::get + repositoriesServiceReference::get, + threadPool ); final IndicesService indicesService = new IndicesService( diff --git a/server/src/test/java/org/opensearch/index/IndexModuleTests.java b/server/src/test/java/org/opensearch/index/IndexModuleTests.java index d9d87196ca289..32b8fb5a4dc62 100644 --- a/server/src/test/java/org/opensearch/index/IndexModuleTests.java +++ b/server/src/test/java/org/opensearch/index/IndexModuleTests.java @@ -252,7 +252,7 @@ private IndexService newIndexService(IndexModule module) throws IOException { writableRegistry(), () -> false, null, - new RemoteSegmentStoreDirectoryFactory(() -> repositoriesService), + new RemoteSegmentStoreDirectoryFactory(() -> repositoriesService, threadPool), translogFactorySupplier ); } diff --git a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactoryTests.java b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactoryTests.java index 7a9cbc12d823b..324315505987b 100644 --- a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryFactoryTests.java @@ -24,6 +24,7 @@ import org.opensearch.repositories.blobstore.BlobStoreRepository; import org.opensearch.test.IndexSettingsModule; import org.opensearch.test.OpenSearchTestCase; +import org.opensearch.threadpool.ThreadPool; import java.io.IOException; import java.nio.file.Path; @@ -41,14 +42,16 @@ public class RemoteSegmentStoreDirectoryFactoryTests extends OpenSearchTestCase private Supplier repositoriesServiceSupplier; private RepositoriesService repositoriesService; + private ThreadPool threadPool; private RemoteSegmentStoreDirectoryFactory remoteSegmentStoreDirectoryFactory; @Before public void setup() { repositoriesServiceSupplier = mock(Supplier.class); repositoriesService = mock(RepositoriesService.class); + threadPool = mock(ThreadPool.class); when(repositoriesServiceSupplier.get()).thenReturn(repositoriesService); - remoteSegmentStoreDirectoryFactory = new RemoteSegmentStoreDirectoryFactory(repositoriesServiceSupplier); + remoteSegmentStoreDirectoryFactory = new RemoteSegmentStoreDirectoryFactory(repositoriesServiceSupplier, threadPool); } public void testNewDirectory() throws IOException { diff --git a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java index 3417e7b0aee04..66e4b9a357b85 100644 --- a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java @@ -30,12 +30,14 @@ import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.lucene.store.ByteArrayIndexInput; import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.concurrent.OpenSearchExecutors; import org.opensearch.index.engine.NRTReplicationEngineFactory; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.IndexShardTestCase; import org.opensearch.index.store.lockmanager.RemoteStoreMetadataLockManager; import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadata; import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadataHandler; +import org.opensearch.threadpool.ThreadPool; import java.io.IOException; import java.nio.file.NoSuchFileException; @@ -45,6 +47,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.ExecutorService; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -56,6 +59,7 @@ import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.doReturn; +import static org.hamcrest.CoreMatchers.is; public class RemoteSegmentStoreDirectoryTests extends IndexShardTestCase { private RemoteDirectory remoteDataDirectory; @@ -65,21 +69,31 @@ public class RemoteSegmentStoreDirectoryTests extends IndexShardTestCase { private RemoteSegmentStoreDirectory remoteSegmentStoreDirectory; private IndexShard indexShard; private SegmentInfos segmentInfos; + private ThreadPool threadPool; @Before public void setup() throws IOException { remoteDataDirectory = mock(RemoteDirectory.class); remoteMetadataDirectory = mock(RemoteDirectory.class); mdLockManager = mock(RemoteStoreMetadataLockManager.class); + threadPool = mock(ThreadPool.class); - remoteSegmentStoreDirectory = new RemoteSegmentStoreDirectory(remoteDataDirectory, remoteMetadataDirectory, mdLockManager); + remoteSegmentStoreDirectory = new RemoteSegmentStoreDirectory( + remoteDataDirectory, + remoteMetadataDirectory, + mdLockManager, + threadPool + ); Settings indexSettings = Settings.builder().put(IndexMetadata.SETTING_VERSION_CREATED, org.opensearch.Version.CURRENT).build(); + ExecutorService executorService = OpenSearchExecutors.newDirectExecutorService(); indexShard = newStartedShard(false, indexSettings, new NRTReplicationEngineFactory()); try (Store store = indexShard.store()) { segmentInfos = store.readLastCommittedSegmentsInfo(); } + + when(threadPool.executor(ThreadPool.Names.REMOTE_PURGE)).thenReturn(executorService); } @After @@ -766,41 +780,76 @@ public void testIncorrectChecksumCorruptIndexException() throws IOException { assertThrows(CorruptIndexException.class, () -> remoteSegmentStoreDirectory.init()); } - public void testDeleteStaleCommitsException() throws IOException { + public void testDeleteStaleCommitsException() throws Exception { + populateMetadata(); when(remoteMetadataDirectory.listFilesByPrefix(RemoteSegmentStoreDirectory.MetadataFilenameUtils.METADATA_PREFIX)).thenThrow( new IOException("Error reading") ); - assertThrows(IOException.class, () -> remoteSegmentStoreDirectory.deleteStaleSegments(5)); + // popluateMetadata() adds stub to return 3 metadata files + // We are passing lastNMetadataFilesToKeep=2 here to validate that in case of exception deleteFile is not + // invoked + remoteSegmentStoreDirectory.deleteStaleSegmentsAsync(2); + + assertBusy(() -> assertThat(remoteSegmentStoreDirectory.canDeleteStaleCommits.get(), is(true))); + verify(remoteMetadataDirectory, times(0)).deleteFile(any(String.class)); + } + + public void testDeleteStaleCommitsExceptionWhileScheduling() throws Exception { + populateMetadata(); + doThrow(new IllegalArgumentException()).when(threadPool).executor(any(String.class)); + + // popluateMetadata() adds stub to return 3 metadata files + // We are passing lastNMetadataFilesToKeep=2 here to validate that in case of exception deleteFile is not + // invoked + remoteSegmentStoreDirectory.deleteStaleSegmentsAsync(2); + + assertBusy(() -> assertThat(remoteSegmentStoreDirectory.canDeleteStaleCommits.get(), is(true))); + verify(remoteMetadataDirectory, times(0)).deleteFile(any(String.class)); + } + + public void testDeleteStaleCommitsWithDeletionAlreadyInProgress() throws Exception { + populateMetadata(); + remoteSegmentStoreDirectory.canDeleteStaleCommits.set(false); + + // popluateMetadata() adds stub to return 3 metadata files + // We are passing lastNMetadataFilesToKeep=2 here to validate that in case of exception deleteFile is not + // invoked + remoteSegmentStoreDirectory.deleteStaleSegmentsAsync(2); + + assertBusy(() -> assertThat(remoteSegmentStoreDirectory.canDeleteStaleCommits.get(), is(false))); + verify(remoteMetadataDirectory, times(0)).deleteFile(any(String.class)); } - public void testDeleteStaleCommitsWithinThreshold() throws IOException { + public void testDeleteStaleCommitsWithinThreshold() throws Exception { populateMetadata(); // popluateMetadata() adds stub to return 3 metadata files // We are passing lastNMetadataFilesToKeep=5 here so that none of the metadata files will be deleted - remoteSegmentStoreDirectory.deleteStaleSegments(5); + remoteSegmentStoreDirectory.deleteStaleSegmentsAsync(5); + assertBusy(() -> assertThat(remoteSegmentStoreDirectory.canDeleteStaleCommits.get(), is(true))); verify(remoteMetadataDirectory, times(0)).openInput(any(String.class), eq(IOContext.DEFAULT)); } - public void testDeleteStaleCommitsActualDelete() throws IOException { + public void testDeleteStaleCommitsActualDelete() throws Exception { Map> metadataFilenameContentMapping = populateMetadata(); remoteSegmentStoreDirectory.init(); // popluateMetadata() adds stub to return 3 metadata files // We are passing lastNMetadataFilesToKeep=2 here so that oldest 1 metadata file will be deleted - remoteSegmentStoreDirectory.deleteStaleSegments(2); + remoteSegmentStoreDirectory.deleteStaleSegmentsAsync(2); for (String metadata : metadataFilenameContentMapping.get("metadata__1__5__abc").values()) { String uploadedFilename = metadata.split(RemoteSegmentStoreDirectory.UploadedSegmentMetadata.SEPARATOR)[1]; verify(remoteDataDirectory).deleteFile(uploadedFilename); } ; + assertBusy(() -> assertThat(remoteSegmentStoreDirectory.canDeleteStaleCommits.get(), is(true))); verify(remoteMetadataDirectory).deleteFile("metadata__1__5__abc"); } - public void testDeleteStaleCommitsActualDeleteIOException() throws IOException { + public void testDeleteStaleCommitsActualDeleteIOException() throws Exception { Map> metadataFilenameContentMapping = populateMetadata(); remoteSegmentStoreDirectory.init(); @@ -813,17 +862,18 @@ public void testDeleteStaleCommitsActualDeleteIOException() throws IOException { doThrow(new IOException("Error")).when(remoteDataDirectory).deleteFile(segmentFileWithException); // popluateMetadata() adds stub to return 3 metadata files // We are passing lastNMetadataFilesToKeep=2 here so that oldest 1 metadata file will be deleted - remoteSegmentStoreDirectory.deleteStaleSegments(2); + remoteSegmentStoreDirectory.deleteStaleSegmentsAsync(2); for (String metadata : metadataFilenameContentMapping.get("metadata__1__5__abc").values()) { String uploadedFilename = metadata.split(RemoteSegmentStoreDirectory.UploadedSegmentMetadata.SEPARATOR)[1]; verify(remoteDataDirectory).deleteFile(uploadedFilename); } ; + assertBusy(() -> assertThat(remoteSegmentStoreDirectory.canDeleteStaleCommits.get(), is(true))); verify(remoteMetadataDirectory, times(0)).deleteFile("metadata__1__5__abc"); } - public void testDeleteStaleCommitsActualDeleteNoSuchFileException() throws IOException { + public void testDeleteStaleCommitsActualDeleteNoSuchFileException() throws Exception { Map> metadataFilenameContentMapping = populateMetadata(); remoteSegmentStoreDirectory.init(); @@ -836,13 +886,14 @@ public void testDeleteStaleCommitsActualDeleteNoSuchFileException() throws IOExc doThrow(new NoSuchFileException(segmentFileWithException)).when(remoteDataDirectory).deleteFile(segmentFileWithException); // popluateMetadata() adds stub to return 3 metadata files // We are passing lastNMetadataFilesToKeep=2 here so that oldest 1 metadata file will be deleted - remoteSegmentStoreDirectory.deleteStaleSegments(2); + remoteSegmentStoreDirectory.deleteStaleSegmentsAsync(2); for (String metadata : metadataFilenameContentMapping.get("metadata__1__5__abc").values()) { String uploadedFilename = metadata.split(RemoteSegmentStoreDirectory.UploadedSegmentMetadata.SEPARATOR)[1]; verify(remoteDataDirectory).deleteFile(uploadedFilename); } ; + assertBusy(() -> assertThat(remoteSegmentStoreDirectory.canDeleteStaleCommits.get(), is(true))); verify(remoteMetadataDirectory).deleteFile("metadata__1__5__abc"); } diff --git a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java index 0bb2b604e8f1a..88899a1b282af 100644 --- a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java @@ -1835,7 +1835,7 @@ public void onFailure(final Exception e) { emptyMap(), null, emptyMap(), - new RemoteSegmentStoreDirectoryFactory(() -> repositoriesService), + new RemoteSegmentStoreDirectoryFactory(() -> repositoriesService, threadPool), repositoriesServiceReference::get, fileCacheCleaner ); diff --git a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java index ea9e9342673db..7f3819563dcbd 100644 --- a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java @@ -666,7 +666,7 @@ protected RemoteSegmentStoreDirectory createRemoteSegmentStoreDirectory(ShardId RemoteStoreLockManager remoteStoreLockManager = new RemoteStoreMetadataLockManager( new RemoteBufferedOutputDirectory(getBlobContainer(remoteShardPath.resolveIndex())) ); - return new RemoteSegmentStoreDirectory(dataDirectory, metadataDirectory, remoteStoreLockManager); + return new RemoteSegmentStoreDirectory(dataDirectory, metadataDirectory, remoteStoreLockManager, threadPool); } private RemoteDirectory newRemoteDirectory(Path f) throws IOException { From 2c5a22d167460a0a5d2ad61f16914ca309efe427 Mon Sep 17 00:00:00 2001 From: Sachin Kale Date: Mon, 3 Jul 2023 21:55:48 +0530 Subject: [PATCH 103/109] [Remote Store] Add method in BlobContainer to fetch the keys in sorted order (#8314) --------- Signed-off-by: Sachin Kale Co-authored-by: Sachin Kale --- .../repositories/s3/S3BlobContainer.java | 51 +++++- .../s3/S3BlobStoreContainerTests.java | 160 ++++++++++++++++-- .../org/opensearch/backwards/IndexingIT.java | 4 - .../common/blobstore/BlobContainer.java | 47 +++++ .../blobstore/fs/FsBlobContainerTests.java | 80 +++++++++ 5 files changed, 320 insertions(+), 22 deletions(-) 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 cf749eeffd903..49ebce77a59ad 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 @@ -36,6 +36,7 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.opensearch.ExceptionsHelper; +import org.opensearch.action.ActionListener; import org.opensearch.common.Nullable; import org.opensearch.common.SetOnce; import org.opensearch.common.blobstore.BlobContainer; @@ -296,6 +297,35 @@ private static DeleteObjectsRequest bulkDelete(String bucket, List blobs .build(); } + @Override + public void listBlobsByPrefixInSortedOrder( + String blobNamePrefix, + int limit, + BlobNameSortOrder blobNameSortOrder, + ActionListener> listener + ) { + // As AWS S3 returns list of keys in Lexicographic order, we don't have to fetch all the keys in order to sort them + // We fetch only keys as per the given limit to optimize the fetch. If provided sort order is not Lexicographic, + // we fall-back to default implementation of fetching all the keys and sorting them. + if (blobNameSortOrder != BlobNameSortOrder.LEXICOGRAPHIC) { + super.listBlobsByPrefixInSortedOrder(blobNamePrefix, limit, blobNameSortOrder, listener); + } else { + if (limit < 0) { + throw new IllegalArgumentException("limit should not be a negative value"); + } + String prefix = blobNamePrefix == null ? keyPath : buildKey(blobNamePrefix); + try (AmazonS3Reference clientReference = blobStore.clientReference()) { + List blobs = executeListing(clientReference, listObjectsRequest(prefix, limit), limit).stream() + .flatMap(listing -> listing.contents().stream()) + .map(s3Object -> new PlainBlobMetadata(s3Object.key().substring(keyPath.length()), s3Object.size())) + .collect(Collectors.toList()); + listener.onResponse(blobs.subList(0, Math.min(limit, blobs.size()))); + } catch (final Exception e) { + listener.onFailure(new IOException("Exception when listing blobs by prefix [" + prefix + "]", e)); + } + } + } + @Override public Map listBlobsByPrefix(@Nullable String blobNamePrefix) throws IOException { String prefix = blobNamePrefix == null ? keyPath : buildKey(blobNamePrefix); @@ -339,10 +369,25 @@ public Map children() throws IOException { } private static List executeListing(AmazonS3Reference clientReference, ListObjectsV2Request listObjectsRequest) { + return executeListing(clientReference, listObjectsRequest, -1); + } + + private static List executeListing( + AmazonS3Reference clientReference, + ListObjectsV2Request listObjectsRequest, + int limit + ) { return SocketAccess.doPrivileged(() -> { final List results = new ArrayList<>(); + int totalObjects = 0; ListObjectsV2Iterable listObjectsIterable = clientReference.get().listObjectsV2Paginator(listObjectsRequest); - listObjectsIterable.forEach(results::add); + for (ListObjectsV2Response listObjectsV2Response : listObjectsIterable) { + results.add(listObjectsV2Response); + totalObjects += listObjectsV2Response.contents().size(); + if (limit != -1 && totalObjects > limit) { + break; + } + } return results; }); } @@ -356,6 +401,10 @@ private ListObjectsV2Request listObjectsRequest(String keyPath) { .build(); } + private ListObjectsV2Request listObjectsRequest(String keyPath, int limit) { + return listObjectsRequest(keyPath).toBuilder().maxKeys(Math.min(limit, 1000)).build(); + } + private String buildKey(String blobName) { return keyPath + blobName; } diff --git a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobStoreContainerTests.java b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobStoreContainerTests.java index ec16f216f1777..a2a7ca8d8bdd5 100644 --- a/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobStoreContainerTests.java +++ b/plugins/repository-s3/src/test/java/org/opensearch/repositories/s3/S3BlobStoreContainerTests.java @@ -33,6 +33,9 @@ package org.opensearch.repositories.s3; import org.mockito.ArgumentCaptor; +import org.opensearch.action.ActionListener; +import org.opensearch.common.blobstore.BlobContainer; +import org.opensearch.common.blobstore.BlobMetadata; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.blobstore.BlobStoreException; import org.opensearch.common.blobstore.DeleteResult; @@ -74,9 +77,13 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.Comparator; +import java.util.HashSet; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.NoSuchElementException; +import java.util.Set; import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; @@ -84,12 +91,12 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.any; import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; +import static org.mockito.Mockito.times; public class S3BlobStoreContainerTests extends OpenSearchTestCase { @@ -187,26 +194,34 @@ private static class MockListObjectsV2ResponseIterator implements Iterator keysListed = new ArrayList<>(); + private final List keysListed; private final boolean throwExceptionOnNextInvocation; public MockListObjectsV2ResponseIterator(int totalPageCount, int s3ObjectsPerPage, long s3ObjectSize) { - this.totalPageCount = totalPageCount; - this.s3ObjectsPerPage = s3ObjectsPerPage; - this.s3ObjectSize = s3ObjectSize; - this.throwExceptionOnNextInvocation = false; + this(totalPageCount, s3ObjectsPerPage, s3ObjectSize, ""); + } + + public MockListObjectsV2ResponseIterator(int totalPageCount, int s3ObjectsPerPage, long s3ObjectSize, String blobPath) { + this(totalPageCount, s3ObjectsPerPage, s3ObjectSize, blobPath, false); } public MockListObjectsV2ResponseIterator( int totalPageCount, int s3ObjectsPerPage, long s3ObjectSize, + String blobPath, boolean throwExceptionOnNextInvocation ) { this.totalPageCount = totalPageCount; this.s3ObjectsPerPage = s3ObjectsPerPage; this.s3ObjectSize = s3ObjectSize; this.throwExceptionOnNextInvocation = throwExceptionOnNextInvocation; + keysListed = new ArrayList<>(); + for (int i = 0; i < totalPageCount * s3ObjectsPerPage; i++) { + keysListed.add(blobPath + UUID.randomUUID().toString()); + } + // S3 lists keys in lexicographic order + keysListed.sort(String::compareTo); } @Override @@ -220,11 +235,12 @@ public ListObjectsV2Response next() { throw SdkException.builder().build(); } if (currInvocationCount.getAndIncrement() < totalPageCount) { - String s3ObjectKey = UUID.randomUUID().toString(); - keysListed.add(s3ObjectKey); - return ListObjectsV2Response.builder() - .contents(Collections.nCopies(s3ObjectsPerPage, S3Object.builder().key(s3ObjectKey).size(s3ObjectSize).build())) - .build(); + List s3Objects = new ArrayList<>(); + for (int i = 0; i < s3ObjectsPerPage; i++) { + String s3ObjectKey = keysListed.get((currInvocationCount.get() - 1) * s3ObjectsPerPage + i); + s3Objects.add(S3Object.builder().key(s3ObjectKey).size(s3ObjectSize).build()); + } + return ListObjectsV2Response.builder().contents(s3Objects).build(); } throw new NoSuchElementException(); } @@ -232,6 +248,10 @@ public ListObjectsV2Response next() { public List getKeysListed() { return keysListed; } + + public int numberOfPagesFetched() { + return currInvocationCount.get(); + } } public void testDelete() throws IOException { @@ -273,10 +293,8 @@ public void testDelete() throws IOException { // keysDeleted will have blobPath also assertEquals(listObjectsV2ResponseIterator.getKeysListed().size(), keysDeleted.size() - 1); assertTrue(keysDeleted.contains(blobPath.buildAsString())); - assertArrayEquals( - listObjectsV2ResponseIterator.getKeysListed().toArray(String[]::new), - keysDeleted.stream().filter(key -> !blobPath.buildAsString().equals(key)).toArray(String[]::new) - ); + keysDeleted.remove(blobPath.buildAsString()); + assertEquals(new HashSet<>(listObjectsV2ResponseIterator.getKeysListed()), new HashSet<>(keysDeleted)); } public void testDeleteItemLevelErrorsDuringDelete() { @@ -772,4 +790,112 @@ private static void assertNumberOfMultiparts(final int expectedParts, final long assertEquals("Expected number of parts [" + expectedParts + "] but got [" + result.v1() + "]", expectedParts, (long) result.v1()); assertEquals("Expected remaining [" + expectedRemaining + "] but got [" + result.v2() + "]", expectedRemaining, (long) result.v2()); } + + public void testListBlobsByPrefix() throws IOException { + final S3BlobStore blobStore = mock(S3BlobStore.class); + when(blobStore.getStatsMetricPublisher()).thenReturn(new StatsMetricPublisher()); + + final S3Client client = mock(S3Client.class); + final AmazonS3Reference clientReference = new AmazonS3Reference(client); + when(blobStore.clientReference()).thenReturn(clientReference); + + BlobPath blobPath = mock(BlobPath.class); + when(blobPath.buildAsString()).thenReturn("/dummy/path"); + final S3BlobContainer blobContainer = new S3BlobContainer(blobPath, blobStore); + + final ListObjectsV2Iterable listObjectsV2Iterable = mock(ListObjectsV2Iterable.class); + when(client.listObjectsV2Paginator(any(ListObjectsV2Request.class))).thenReturn(listObjectsV2Iterable); + + MockListObjectsV2ResponseIterator iterator = new MockListObjectsV2ResponseIterator(2, 5, 100); + when(listObjectsV2Iterable.iterator()).thenReturn(iterator); + + Map listOfBlobs = blobContainer.listBlobsByPrefix(null); + assertEquals(10, listOfBlobs.size()); + + Set keys = iterator.keysListed.stream() + .map(s -> s.substring(blobPath.buildAsString().length())) + .collect(Collectors.toSet()); + assertEquals(keys, listOfBlobs.keySet()); + } + + private void testListBlobsByPrefixInLexicographicOrder( + int limit, + int expectedNumberofPagesFetched, + BlobContainer.BlobNameSortOrder blobNameSortOrder + ) throws IOException { + final S3BlobStore blobStore = mock(S3BlobStore.class); + when(blobStore.getStatsMetricPublisher()).thenReturn(new StatsMetricPublisher()); + + final S3Client client = mock(S3Client.class); + final AmazonS3Reference clientReference = new AmazonS3Reference(client); + when(blobStore.clientReference()).thenReturn(clientReference); + + BlobPath blobPath = mock(BlobPath.class); + when(blobPath.buildAsString()).thenReturn("/dummy/path"); + final S3BlobContainer blobContainer = new S3BlobContainer(blobPath, blobStore); + + final ListObjectsV2Iterable listObjectsV2Iterable = mock(ListObjectsV2Iterable.class); + when(client.listObjectsV2Paginator(any(ListObjectsV2Request.class))).thenReturn(listObjectsV2Iterable); + + final MockListObjectsV2ResponseIterator iterator = new MockListObjectsV2ResponseIterator(2, 5, 100, blobPath.buildAsString()); + when(listObjectsV2Iterable.iterator()).thenReturn(iterator); + + if (limit >= 0) { + blobContainer.listBlobsByPrefixInSortedOrder(null, limit, blobNameSortOrder, new ActionListener<>() { + @Override + public void onResponse(List blobMetadata) { + int actualLimit = Math.max(0, Math.min(limit, 10)); + assertEquals(actualLimit, blobMetadata.size()); + + List keys = iterator.keysListed.stream() + .map(s -> s.substring(blobPath.buildAsString().length())) + .collect(Collectors.toList()); + Comparator keysComparator = String::compareTo; + if (blobNameSortOrder != BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC) { + keysComparator = Collections.reverseOrder(String::compareTo); + } + keys.sort(keysComparator); + List sortedKeys = keys.subList(0, actualLimit); + assertEquals(sortedKeys, blobMetadata.stream().map(BlobMetadata::name).collect(Collectors.toList())); + assertEquals(expectedNumberofPagesFetched, iterator.numberOfPagesFetched()); + } + + @Override + public void onFailure(Exception e) { + fail("blobContainer.listBlobsByPrefixInLexicographicOrder failed with exception: " + e.getMessage()); + } + }); + } else { + assertThrows( + IllegalArgumentException.class, + () -> blobContainer.listBlobsByPrefixInSortedOrder(null, limit, blobNameSortOrder, new ActionListener<>() { + @Override + public void onResponse(List blobMetadata) {} + + @Override + public void onFailure(Exception e) {} + }) + ); + } + } + + public void testListBlobsByPrefixInLexicographicOrderWithNegativeLimit() throws IOException { + testListBlobsByPrefixInLexicographicOrder(-5, 0, BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC); + } + + public void testListBlobsByPrefixInLexicographicOrderWithZeroLimit() throws IOException { + testListBlobsByPrefixInLexicographicOrder(0, 1, BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC); + } + + public void testListBlobsByPrefixInLexicographicOrderWithLimitLessThanPageSize() throws IOException { + testListBlobsByPrefixInLexicographicOrder(2, 1, BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC); + } + + public void testListBlobsByPrefixInLexicographicOrderWithLimitGreaterThanPageSize() throws IOException { + testListBlobsByPrefixInLexicographicOrder(8, 2, BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC); + } + + public void testListBlobsByPrefixInLexicographicOrderWithLimitGreaterThanNumberOfRecords() throws IOException { + testListBlobsByPrefixInLexicographicOrder(12, 2, BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC); + } } diff --git a/qa/mixed-cluster/src/test/java/org/opensearch/backwards/IndexingIT.java b/qa/mixed-cluster/src/test/java/org/opensearch/backwards/IndexingIT.java index b867b90af333c..f1f469544f634 100644 --- a/qa/mixed-cluster/src/test/java/org/opensearch/backwards/IndexingIT.java +++ b/qa/mixed-cluster/src/test/java/org/opensearch/backwards/IndexingIT.java @@ -113,8 +113,6 @@ private void printClusterRouting() throws IOException, ParseException { /** * This test verifies that segment replication does not break when primary shards are on lower OS version. It does this * by verifying replica shards contains same number of documents as primary's. - * - * @throws Exception */ public void testIndexingWithPrimaryOnBwcNodes() throws Exception { if (UPGRADE_FROM_VERSION.before(Version.V_2_4_0)) { @@ -164,8 +162,6 @@ public void testIndexingWithPrimaryOnBwcNodes() throws Exception { * This test creates a cluster with primary on higher version but due to {@link org.opensearch.cluster.routing.allocation.decider.NodeVersionAllocationDecider}; * replica shard allocation on lower OpenSearch version is prevented. Thus, this test though cover the use case where * primary shard containing nodes are running on higher OS version while replicas are unassigned. - * - * @throws Exception */ public void testIndexingWithReplicaOnBwcNodes() throws Exception { if (UPGRADE_FROM_VERSION.before(Version.V_2_4_0)) { diff --git a/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java b/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java index ac38768c9f3d3..e626824e7e271 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java +++ b/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java @@ -32,10 +32,14 @@ package org.opensearch.common.blobstore; +import org.opensearch.action.ActionListener; + import java.io.IOException; import java.io.InputStream; import java.nio.file.FileAlreadyExistsException; import java.nio.file.NoSuchFileException; +import java.util.ArrayList; +import java.util.Comparator; import java.util.List; import java.util.Map; @@ -191,4 +195,47 @@ default long readBlobPreferredLength() { * @throws IOException if there were any failures in reading from the blob container. */ Map listBlobsByPrefix(String blobNamePrefix) throws IOException; + + /** + * The type representing sort order of blob names + */ + enum BlobNameSortOrder { + + LEXICOGRAPHIC(Comparator.comparing(BlobMetadata::name)); + + final Comparator comparator; + + public Comparator comparator() { + return comparator; + } + + BlobNameSortOrder(final Comparator comparator) { + this.comparator = comparator; + } + } + + /** + * Lists all blobs in the container that match the specified prefix in lexicographic order + * @param blobNamePrefix The prefix to match against blob names in the container. + * @param limit Limits the result size to min(limit, number of keys) + * @param blobNameSortOrder Comparator to sort keys with + * @param listener the listener to be notified upon request completion + */ + default void listBlobsByPrefixInSortedOrder( + String blobNamePrefix, + int limit, + BlobNameSortOrder blobNameSortOrder, + ActionListener> listener + ) { + if (limit < 0) { + throw new IllegalArgumentException("limit should not be a negative value"); + } + try { + List blobNames = new ArrayList<>(listBlobsByPrefix(blobNamePrefix).values()); + blobNames.sort(blobNameSortOrder.comparator()); + listener.onResponse(blobNames.subList(0, Math.min(blobNames.size(), limit))); + } catch (Exception e) { + listener.onFailure(e); + } + } } diff --git a/server/src/test/java/org/opensearch/common/blobstore/fs/FsBlobContainerTests.java b/server/src/test/java/org/opensearch/common/blobstore/fs/FsBlobContainerTests.java index 6c36368bfe446..f139a5d4e3bb1 100644 --- a/server/src/test/java/org/opensearch/common/blobstore/fs/FsBlobContainerTests.java +++ b/server/src/test/java/org/opensearch/common/blobstore/fs/FsBlobContainerTests.java @@ -34,6 +34,9 @@ import org.apache.lucene.tests.mockfile.FilterFileSystemProvider; import org.apache.lucene.tests.mockfile.FilterSeekableByteChannel; import org.apache.lucene.tests.util.LuceneTestCase; +import org.opensearch.action.ActionListener; +import org.opensearch.common.blobstore.BlobContainer; +import org.opensearch.common.blobstore.BlobMetadata; import org.opensearch.common.blobstore.BlobPath; import org.opensearch.common.io.PathUtils; import org.opensearch.common.io.PathUtilsForTesting; @@ -54,10 +57,14 @@ import java.nio.file.Path; import java.nio.file.attribute.FileAttribute; import java.nio.file.spi.FileSystemProvider; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; import java.util.Locale; import java.util.Set; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; +import java.util.stream.Collectors; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; @@ -118,6 +125,79 @@ public void testIsTempBlobName() { assertThat(FsBlobContainer.isTempBlobName(tempBlobName), is(true)); } + private void testListBlobsByPrefixInSortedOrder(int limit, BlobContainer.BlobNameSortOrder blobNameSortOrder) throws IOException { + + final Path path = PathUtils.get(createTempDir().toString()); + + List blobsInFileSystem = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + final String blobName = randomAlphaOfLengthBetween(1, 20).toLowerCase(Locale.ROOT); + final byte[] blobData = randomByteArrayOfLength(randomIntBetween(1, frequently() ? 512 : 1 << 20)); // rarely up to 1mb + Files.write(path.resolve(blobName), blobData); + blobsInFileSystem.add(blobName); + } + + final FsBlobContainer container = new FsBlobContainer( + new FsBlobStore(randomIntBetween(1, 8) * 1024, path, false), + BlobPath.cleanPath(), + path + ); + + if (limit >= 0) { + container.listBlobsByPrefixInSortedOrder(null, limit, blobNameSortOrder, new ActionListener<>() { + @Override + public void onResponse(List blobMetadata) { + int actualLimit = Math.min(limit, 10); + assertEquals(actualLimit, blobMetadata.size()); + + if (blobNameSortOrder == BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC) { + blobsInFileSystem.sort(String::compareTo); + } else { + blobsInFileSystem.sort(Collections.reverseOrder(String::compareTo)); + } + List keys = blobsInFileSystem.subList(0, actualLimit); + assertEquals(keys, blobMetadata.stream().map(BlobMetadata::name).collect(Collectors.toList())); + } + + @Override + public void onFailure(Exception e) { + fail("blobContainer.listBlobsByPrefixInLexicographicOrder failed with exception: " + e.getMessage()); + } + }); + } else { + assertThrows( + IllegalArgumentException.class, + () -> container.listBlobsByPrefixInSortedOrder(null, limit, blobNameSortOrder, new ActionListener<>() { + @Override + public void onResponse(List blobMetadata) {} + + @Override + public void onFailure(Exception e) {} + }) + ); + } + } + + public void testListBlobsByPrefixInLexicographicOrderWithNegativeLimit() throws IOException { + testListBlobsByPrefixInSortedOrder(-5, BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC); + } + + public void testListBlobsByPrefixInLexicographicOrderWithZeroLimit() throws IOException { + testListBlobsByPrefixInSortedOrder(0, BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC); + } + + public void testListBlobsByPrefixInLexicographicOrderWithLimitLessThanNumberOfRecords() throws IOException { + testListBlobsByPrefixInSortedOrder(8, BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC); + } + + public void testListBlobsByPrefixInLexicographicOrderWithLimitNumberOfRecords() throws IOException { + testListBlobsByPrefixInSortedOrder(10, BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC); + } + + public void testListBlobsByPrefixInLexicographicOrderWithLimitGreaterThanNumberOfRecords() throws IOException { + testListBlobsByPrefixInSortedOrder(12, BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC); + } + static class MockFileSystemProvider extends FilterFileSystemProvider { final Consumer onRead; From a3b515aaa7e14a18f98aef80658a630bb62bb4cc Mon Sep 17 00:00:00 2001 From: Sachin Kale Date: Tue, 4 Jul 2023 09:29:47 +0530 Subject: [PATCH 104/109] [Remote Store] Add remote store utils useful to name metadata files (#8412) Signed-off-by: Sachin Kale --- .../index/remote/RemoteStoreUtils.java | 51 +++++++++++++++++++ .../index/remote/RemoteStoreUtilsTests.java | 41 +++++++++++++++ 2 files changed, 92 insertions(+) create mode 100644 server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java create mode 100644 server/src/test/java/org/opensearch/index/remote/RemoteStoreUtilsTests.java diff --git a/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java b/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java new file mode 100644 index 0000000000000..1eeadfe228a45 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/remote/RemoteStoreUtils.java @@ -0,0 +1,51 @@ +/* + * 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.index.remote; + +import java.util.Arrays; + +/** + * Utils for remote store + * + * @opensearch.internal + */ +public class RemoteStoreUtils { + public static final int LONG_MAX_LENGTH = String.valueOf(Long.MAX_VALUE).length(); + + /** + * This method subtracts given numbers from Long.MAX_VALUE and returns a string representation of the result. + * The resultant string is guaranteed to be of the same length that of Long.MAX_VALUE. If shorter, we add left padding + * of 0s to the string. + * @param num number to get the inverted long string for + * @return String value of Long.MAX_VALUE - num + */ + public static String invertLong(long num) { + if (num < 0) { + throw new IllegalArgumentException("Negative long values are not allowed"); + } + String invertedLong = String.valueOf(Long.MAX_VALUE - num); + char[] characterArray = new char[LONG_MAX_LENGTH - invertedLong.length()]; + Arrays.fill(characterArray, '0'); + + return new String(characterArray) + invertedLong; + } + + /** + * This method converts the given string into long and subtracts it from Long.MAX_VALUE + * @param str long in string format to be inverted + * @return long value of the invert result + */ + public static long invertLong(String str) { + long num = Long.parseLong(str); + if (num < 0) { + throw new IllegalArgumentException("Strings representing negative long values are not allowed"); + } + return Long.MAX_VALUE - num; + } +} diff --git a/server/src/test/java/org/opensearch/index/remote/RemoteStoreUtilsTests.java b/server/src/test/java/org/opensearch/index/remote/RemoteStoreUtilsTests.java new file mode 100644 index 0000000000000..5b9135afb66f3 --- /dev/null +++ b/server/src/test/java/org/opensearch/index/remote/RemoteStoreUtilsTests.java @@ -0,0 +1,41 @@ +/* + * 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.index.remote; + +import org.opensearch.test.OpenSearchTestCase; + +public class RemoteStoreUtilsTests extends OpenSearchTestCase { + + public void testInvertToStrInvalid() { + assertThrows(IllegalArgumentException.class, () -> RemoteStoreUtils.invertLong(-1)); + } + + public void testInvertToStrValid() { + assertEquals("9223372036854774573", RemoteStoreUtils.invertLong(1234)); + assertEquals("0000000000000001234", RemoteStoreUtils.invertLong(9223372036854774573L)); + } + + public void testInvertToLongInvalid() { + assertThrows(IllegalArgumentException.class, () -> RemoteStoreUtils.invertLong("-5")); + } + + public void testInvertToLongValid() { + assertEquals(1234, RemoteStoreUtils.invertLong("9223372036854774573")); + assertEquals(9223372036854774573L, RemoteStoreUtils.invertLong("0000000000000001234")); + } + + public void testinvert() { + assertEquals(0, RemoteStoreUtils.invertLong(RemoteStoreUtils.invertLong(0))); + assertEquals(Long.MAX_VALUE, RemoteStoreUtils.invertLong(RemoteStoreUtils.invertLong(Long.MAX_VALUE))); + for (int i = 0; i < 10; i++) { + long num = randomLongBetween(1, Long.MAX_VALUE); + assertEquals(num, RemoteStoreUtils.invertLong(RemoteStoreUtils.invertLong(num))); + } + } +} From 64c0871fb529b17bc2360cf8a700d969ba302806 Mon Sep 17 00:00:00 2001 From: Sachin Kale Date: Tue, 4 Jul 2023 12:30:33 +0530 Subject: [PATCH 105/109] Fix flaky tests in restore flow using snapshot interop (#8422) Signed-off-by: Sachin Kale --- .../snapshots/RestoreSnapshotIT.java | 46 +++++++++++-------- 1 file changed, 26 insertions(+), 20 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/snapshots/RestoreSnapshotIT.java b/server/src/internalClusterTest/java/org/opensearch/snapshots/RestoreSnapshotIT.java index 9f492bbaee01a..e362b7f61e8e6 100644 --- a/server/src/internalClusterTest/java/org/opensearch/snapshots/RestoreSnapshotIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/snapshots/RestoreSnapshotIT.java @@ -183,8 +183,8 @@ public void testRestoreRemoteStoreIndicesWithoutRemoteTranslog() throws IOExcept public void testRestoreOperationsShallowCopyEnabled(boolean remoteTranslogEnabled) throws IOException, ExecutionException, InterruptedException { - internalCluster().startClusterManagerOnlyNode(); - final String primaryNode = internalCluster().startNode(); + String clusterManagerNode = internalCluster().startClusterManagerOnlyNode(); + String primary = internalCluster().startDataOnlyNode(); String indexName1 = "testindex1"; String indexName2 = "testindex2"; String snapshotRepoName = "test-restore-snapshot-repo"; @@ -216,7 +216,7 @@ public void testRestoreOperationsShallowCopyEnabled(boolean remoteTranslogEnable indexDocuments(client, indexName2, numDocsInIndex2); ensureGreen(indexName1, indexName2); - final String secondNode = internalCluster().startNode(); + internalCluster().startDataOnlyNode(); logger.info("--> snapshot"); CreateSnapshotResponse createSnapshotResponse = client.admin() .cluster() @@ -273,10 +273,12 @@ public void testRestoreOperationsShallowCopyEnabled(boolean remoteTranslogEnable assertDocsPresentInIndex(client, restoredIndexName2, numDocsInIndex2); // deleting data for restoredIndexName1 and restoring from remote store. - internalCluster().stopRandomNode(InternalTestCluster.nameFilter(primaryNodeName(restoredIndexName1))); + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(primary)); ensureRed(restoredIndexName1); - assertAcked(client().admin().indices().prepareClose(restoredIndexName1)); - client().admin() + // Re-initialize client to make sure we are not using client from stopped node. + client = client(clusterManagerNode); + assertAcked(client.admin().indices().prepareClose(restoredIndexName1)); + client.admin() .cluster() .restoreRemoteStore(new RestoreRemoteStoreRequest().indices(restoredIndexName1), PlainActionFuture.newFuture()); ensureYellowAndNoInitializingShards(restoredIndexName1); @@ -300,7 +302,7 @@ public void testRestoreOperationsShallowCopyEnabled(boolean remoteTranslogEnable assertEquals(restoreSnapshotResponse3.status(), RestStatus.ACCEPTED); ensureGreen(restoredIndexName1Seg); - GetIndexResponse getIndexResponse = client().admin() + GetIndexResponse getIndexResponse = client.admin() .indices() .getIndex(new GetIndexRequest().indices(restoredIndexName1Seg).includeDefaults(true)) .get(); @@ -331,7 +333,7 @@ public void testRestoreOperationsShallowCopyEnabled(boolean remoteTranslogEnable assertEquals(restoreSnapshotResponse4.status(), RestStatus.ACCEPTED); ensureGreen(restoredIndexName1Doc); - getIndexResponse = client().admin() + getIndexResponse = client.admin() .indices() .getIndex(new GetIndexRequest().indices(restoredIndexName1Doc).includeDefaults(true)) .get(); @@ -347,8 +349,8 @@ public void testRestoreOperationsShallowCopyEnabled(boolean remoteTranslogEnable } public void testRestoreInSameRemoteStoreEnabledIndex() throws IOException { - internalCluster().startClusterManagerOnlyNode(); - internalCluster().startNode(); + String clusterManagerNode = internalCluster().startClusterManagerOnlyNode(); + String primary = internalCluster().startDataOnlyNode(); String indexName1 = "testindex1"; String indexName2 = "testindex2"; String snapshotRepoName = "test-restore-snapshot-repo"; @@ -378,7 +380,7 @@ public void testRestoreInSameRemoteStoreEnabledIndex() throws IOException { indexDocuments(client, indexName2, numDocsInIndex2); ensureGreen(indexName1, indexName2); - final String secondNode = internalCluster().startNode(); + internalCluster().startDataOnlyNode(); logger.info("--> snapshot"); CreateSnapshotResponse createSnapshotResponse = client.admin() .cluster() @@ -435,10 +437,12 @@ public void testRestoreInSameRemoteStoreEnabledIndex() throws IOException { assertDocsPresentInIndex(client, restoredIndexName2, numDocsInIndex2); // deleting data for restoredIndexName1 and restoring from remote store. - internalCluster().stopRandomNode(InternalTestCluster.nameFilter(primaryNodeName(indexName1))); + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(primary)); ensureRed(indexName1); - assertAcked(client().admin().indices().prepareClose(indexName1)); - client().admin().cluster().restoreRemoteStore(new RestoreRemoteStoreRequest().indices(indexName1), PlainActionFuture.newFuture()); + // Re-initialize client to make sure we are not using client from stopped node. + client = client(clusterManagerNode); + assertAcked(client.admin().indices().prepareClose(indexName1)); + client.admin().cluster().restoreRemoteStore(new RestoreRemoteStoreRequest().indices(indexName1), PlainActionFuture.newFuture()); ensureYellowAndNoInitializingShards(indexName1); ensureGreen(indexName1); assertDocsPresentInIndex(client(), indexName1, numDocsInIndex1); @@ -449,8 +453,8 @@ public void testRestoreInSameRemoteStoreEnabledIndex() throws IOException { } public void testRestoreShallowCopySnapshotWithDifferentRepo() throws IOException { - internalCluster().startClusterManagerOnlyNode(); - final String primaryNode = internalCluster().startNode(); + String clusterManagerNode = internalCluster().startClusterManagerOnlyNode(); + String primary = internalCluster().startDataOnlyNode(); String indexName1 = "testindex1"; String indexName2 = "testindex2"; String snapshotRepoName = "test-restore-snapshot-repo"; @@ -479,7 +483,7 @@ public void testRestoreShallowCopySnapshotWithDifferentRepo() throws IOException indexDocuments(client, indexName2, numDocsInIndex2); ensureGreen(indexName1, indexName2); - final String secondNode = internalCluster().startNode(); + internalCluster().startDataOnlyNode(); logger.info("--> snapshot"); CreateSnapshotResponse createSnapshotResponse = client.admin() @@ -513,9 +517,11 @@ public void testRestoreShallowCopySnapshotWithDifferentRepo() throws IOException assertDocsPresentInIndex(client(), restoredIndexName1, numDocsInIndex1); // deleting data for restoredIndexName1 and restoring from remote store. - internalCluster().stopRandomNode(InternalTestCluster.nameFilter(primaryNodeName(restoredIndexName1))); - assertAcked(client().admin().indices().prepareClose(restoredIndexName1)); - client().admin() + internalCluster().stopRandomNode(InternalTestCluster.nameFilter(primary)); + // Re-initialize client to make sure we are not using client from stopped node. + client = client(clusterManagerNode); + assertAcked(client.admin().indices().prepareClose(restoredIndexName1)); + client.admin() .cluster() .restoreRemoteStore(new RestoreRemoteStoreRequest().indices(restoredIndexName1), PlainActionFuture.newFuture()); ensureYellowAndNoInitializingShards(restoredIndexName1); From 7a5c810a238e0566f0c7bdb6fbc774dafb3674b0 Mon Sep 17 00:00:00 2001 From: Gagan Juneja Date: Tue, 4 Jul 2023 20:30:14 +0530 Subject: [PATCH 106/109] Adds log4j configuration for telemetry-otel plugin (#8393) * Adds log4j configuration for telemetry-otel plugin Signed-off-by: Gagan Juneja * Updates changelog Signed-off-by: Gagan Juneja * Fixes spotless Signed-off-by: Gagan Juneja --------- Signed-off-by: Gagan Juneja Co-authored-by: Gagan Juneja --- CHANGELOG.md | 1 + plugins/telemetry-otel/build.gradle | 22 +++++++++++++++ .../config/telemetry-otel/log4j2.properties | 27 +++++++++++++++++++ .../tracing/OTelResourceProvider.java | 2 +- 4 files changed, 51 insertions(+), 1 deletion(-) create mode 100644 plugins/telemetry-otel/config/telemetry-otel/log4j2.properties diff --git a/CHANGELOG.md b/CHANGELOG.md index f75cc6e508e7a..562e9da458cf5 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -76,6 +76,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Support OpenSSL Provider with default Netty allocator ([#5460](https://github.com/opensearch-project/OpenSearch/pull/5460)) - Replaces ZipInputStream with ZipFile to fix Zip Slip vulnerability ([#7230](https://github.com/opensearch-project/OpenSearch/pull/7230)) - Add missing validation/parsing of SearchBackpressureMode of SearchBackpressureSettings ([#7541](https://github.com/opensearch-project/OpenSearch/pull/7541)) +- Adds log4j configuration for telemetry LogSpanExporter ([#8393](https://github.com/opensearch-project/OpenSearch/pull/8393)) ### Security diff --git a/plugins/telemetry-otel/build.gradle b/plugins/telemetry-otel/build.gradle index 7a56621be5f1e..2c275388cce38 100644 --- a/plugins/telemetry-otel/build.gradle +++ b/plugins/telemetry-otel/build.gradle @@ -54,3 +54,25 @@ thirdPartyAudit { 'io.opentelemetry.sdk.autoconfigure.spi.traces.ConfigurableSpanExporterProvider' ) } + +tasks.named("bundlePlugin").configure { + from('config/telemetry-otel') { + into 'config' + } +} + +tasks.register("writeTestJavaPolicy") { + doLast { + final File tmp = file("${buildDir}/tmp") + if (tmp.exists() == false && tmp.mkdirs() == false) { + throw new GradleException("failed to create temporary directory [${tmp}]") + } + final File javaPolicy = file("${tmp}/java.policy") + javaPolicy.write( + [ + "grant {", + " permission java.io.FilePermission \"config\", \"read\";", + "};" + ].join("\n")) + } +} diff --git a/plugins/telemetry-otel/config/telemetry-otel/log4j2.properties b/plugins/telemetry-otel/config/telemetry-otel/log4j2.properties new file mode 100644 index 0000000000000..544f42bd5513b --- /dev/null +++ b/plugins/telemetry-otel/config/telemetry-otel/log4j2.properties @@ -0,0 +1,27 @@ +# +# 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. +# + + +appender.tracing.type = RollingFile +appender.tracing.name = tracing +appender.tracing.fileName = ${sys:opensearch.logs.base_path}${sys:file.separator}${sys:opensearch.logs.cluster_name}_otel_traces.log +appender.tracing.filePermissions = rw-r----- +appender.tracing.layout.type = PatternLayout +appender.tracing.layout.pattern = %m%n +appender.tracing.filePattern = ${sys:opensearch.logs.base_path}${sys:file.separator}${sys:opensearch.logs.cluster_name}_otel_traces-%i.log.gz +appender.tracing.policies.type = Policies +appender.tracing.policies.size.type = SizeBasedTriggeringPolicy +appender.tracing.policies.size.size = 1GB +appender.tracing.strategy.type = DefaultRolloverStrategy +appender.tracing.strategy.max = 4 + + +logger.exporter.name = io.opentelemetry.exporter.logging.LoggingSpanExporter +logger.exporter.level = INFO +logger.exporter.appenderRef.tracing.ref = tracing +logger.exporter.additivity = false diff --git a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelResourceProvider.java b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelResourceProvider.java index 04bade9ec942a..292165979c2f2 100644 --- a/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelResourceProvider.java +++ b/plugins/telemetry-otel/src/main/java/org/opensearch/telemetry/tracing/OTelResourceProvider.java @@ -42,7 +42,7 @@ private OTelResourceProvider() {} public static OpenTelemetry get(Settings settings) { return get( settings, - new LoggingSpanExporter(), + LoggingSpanExporter.create(), ContextPropagators.create(W3CTraceContextPropagator.getInstance()), Sampler.alwaysOn() ); From a0299fceda9900f8bfdbf8b0229ad1e8884a4fd4 Mon Sep 17 00:00:00 2001 From: Craig Perkins Date: Tue, 4 Jul 2023 11:55:39 -0400 Subject: [PATCH 107/109] Support extension additional settings with extension REST initialization (#8414) * Support extension additional settings with extension REST initialization Signed-off-by: Craig Perkins * Add CHANGELOG entry Signed-off-by: Craig Perkins * Add tests Signed-off-by: Craig Perkins * Add addition setting types to test Signed-off-by: Craig Perkins * Address code review feedback Signed-off-by: Craig Perkins * Check for missing values Signed-off-by: Craig Perkins * Use Version.CURRENT Signed-off-by: Craig Perkins * Switch minimum compat version back to 3.0.0 Signed-off-by: Craig Perkins * Remove hardcoded versions Signed-off-by: Craig Perkins --------- Signed-off-by: Craig Perkins --- CHANGELOG.md | 1 + .../extensions/ExtensionDependency.java | 37 ------ .../extensions/ExtensionsManager.java | 6 +- .../rest/RestInitializeExtensionAction.java | 111 +++++++++++----- .../extensions/ExtensionsManagerTests.java | 12 -- .../RestInitializeExtensionActionTests.java | 119 +++++++++++++++++- 6 files changed, 202 insertions(+), 84 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 562e9da458cf5..0a8deb28833e6 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -95,6 +95,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Add API to initialize extensions ([#8029]()https://github.com/opensearch-project/OpenSearch/pull/8029) - Add distributed tracing framework ([#7543](https://github.com/opensearch-project/OpenSearch/issues/7543)) - Enable Point based optimization for custom comparators ([#8168](https://github.com/opensearch-project/OpenSearch/pull/8168)) +- [Extensions] Support extension additional settings with extension REST initialization ([#8414](https://github.com/opensearch-project/OpenSearch/pull/8414)) ### Dependencies - Bump `com.azure:azure-storage-common` from 12.21.0 to 12.21.1 (#7566, #7814) diff --git a/server/src/main/java/org/opensearch/extensions/ExtensionDependency.java b/server/src/main/java/org/opensearch/extensions/ExtensionDependency.java index 1423a30bbe307..56c9f0387b13e 100644 --- a/server/src/main/java/org/opensearch/extensions/ExtensionDependency.java +++ b/server/src/main/java/org/opensearch/extensions/ExtensionDependency.java @@ -16,10 +16,6 @@ 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.XContentParser; - -import static org.opensearch.common.xcontent.XContentParserUtils.ensureExpectedToken; /** * This class handles the dependent extensions information @@ -60,39 +56,6 @@ public void writeTo(StreamOutput out) throws IOException { out.writeVersion(version); } - public static ExtensionDependency parse(XContentParser parser) throws IOException { - String uniqueId = null; - Version version = null; - ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.currentToken(), parser); - while (parser.nextToken() != XContentParser.Token.END_OBJECT) { - String fieldName = parser.currentName(); - parser.nextToken(); - - switch (fieldName) { - case UNIQUE_ID: - uniqueId = parser.text(); - break; - case VERSION: - try { - version = Version.fromString(parser.text()); - } catch (IllegalArgumentException e) { - throw e; - } - break; - default: - parser.skipChildren(); - break; - } - } - if (Strings.isNullOrEmpty(uniqueId)) { - throw new IOException("Required field [uniqueId] is missing in the request for the dependent extension"); - } else if (version == null) { - throw new IOException("Required field [version] is missing in the request for the dependent extension"); - } - return new ExtensionDependency(uniqueId, version); - - } - /** * The uniqueId of the dependency extension * diff --git a/server/src/main/java/org/opensearch/extensions/ExtensionsManager.java b/server/src/main/java/org/opensearch/extensions/ExtensionsManager.java index 9987497b5fac0..cb22c8d864b1b 100644 --- a/server/src/main/java/org/opensearch/extensions/ExtensionsManager.java +++ b/server/src/main/java/org/opensearch/extensions/ExtensionsManager.java @@ -105,7 +105,7 @@ public static enum OpenSearchRequestType { /** * Instantiate a new ExtensionsManager object to handle requests and responses from extensions. This is called during Node bootstrap. * - * @param additionalSettings Additional settings to read in from extensions.yml + * @param additionalSettings Additional settings to read in from extension initialization request * @throws IOException If the extensions discovery file is not properly retrieved. */ public ExtensionsManager(Set> additionalSettings) throws IOException { @@ -504,4 +504,8 @@ void setAddSettingsUpdateConsumerRequestHandler(AddSettingsUpdateConsumerRequest Settings getEnvironmentSettings() { return environmentSettings; } + + public Set> getAdditionalSettings() { + return this.additionalSettings; + } } 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 e0806f8172278..f47f342617732 100644 --- a/server/src/main/java/org/opensearch/extensions/rest/RestInitializeExtensionAction.java +++ b/server/src/main/java/org/opensearch/extensions/rest/RestInitializeExtensionAction.java @@ -8,9 +8,14 @@ package org.opensearch.extensions.rest; +import org.opensearch.Version; import org.opensearch.client.node.NodeClient; +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.core.xcontent.XContentParser; import org.opensearch.extensions.ExtensionDependency; import org.opensearch.extensions.ExtensionScopedSettings; import org.opensearch.extensions.ExtensionsManager; @@ -23,12 +28,16 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Collections; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; import java.util.List; +import java.util.Map; +import java.util.Set; import java.util.concurrent.CompletionException; import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; -import static org.opensearch.common.xcontent.XContentParserUtils.ensureExpectedToken; import static org.opensearch.rest.RestRequest.Method.POST; /** @@ -62,36 +71,79 @@ public RestChannelConsumer prepareRequest(RestRequest request, NodeClient client String openSearchVersion = null; String minimumCompatibleVersion = null; List dependencies = new ArrayList<>(); + Set additionalSettingsKeys = extensionsManager.getAdditionalSettings() + .stream() + .map(s -> s.getKey()) + .collect(Collectors.toSet()); - try (XContentParser parser = request.contentParser()) { - parser.nextToken(); - ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.currentToken(), parser); - while (parser.nextToken() != XContentParser.Token.END_OBJECT) { - String currentFieldName = parser.currentName(); - parser.nextToken(); - if ("name".equals(currentFieldName)) { - name = parser.text(); - } else if ("uniqueId".equals(currentFieldName)) { - uniqueId = parser.text(); - } else if ("hostAddress".equals(currentFieldName)) { - hostAddress = parser.text(); - } else if ("port".equals(currentFieldName)) { - port = parser.text(); - } else if ("version".equals(currentFieldName)) { - version = parser.text(); - } else if ("opensearchVersion".equals(currentFieldName)) { - openSearchVersion = parser.text(); - } else if ("minimumCompatibleVersion".equals(currentFieldName)) { - minimumCompatibleVersion = parser.text(); - } else if ("dependencies".equals(currentFieldName)) { - ensureExpectedToken(XContentParser.Token.START_ARRAY, parser.currentToken(), parser); - while (parser.nextToken() != XContentParser.Token.END_ARRAY) { - dependencies.add(ExtensionDependency.parse(parser)); + Tuple> unreadExtensionTuple = XContentHelper.convertToMap( + request.content(), + false, + request.getXContentType().xContent().mediaType() + ); + Map extensionMap = unreadExtensionTuple.v2(); + + ExtensionScopedSettings extAdditionalSettings = new ExtensionScopedSettings(extensionsManager.getAdditionalSettings()); + + try { + // checking to see whether any required fields are missing from extension initialization request or not + String[] requiredFields = { + "name", + "uniqueId", + "hostAddress", + "port", + "version", + "opensearchVersion", + "minimumCompatibleVersion" }; + List missingFields = Arrays.stream(requiredFields) + .filter(field -> !extensionMap.containsKey(field)) + .collect(Collectors.toList()); + if (!missingFields.isEmpty()) { + throw new IOException("Extension is missing these required fields : " + missingFields); + } + + // Parse extension dependencies + List extensionDependencyList = new ArrayList(); + if (extensionMap.get("dependencies") != null) { + List> extensionDependencies = new ArrayList<>( + (Collection>) extensionMap.get("dependencies") + ); + for (HashMap dependency : extensionDependencies) { + if (Strings.isNullOrEmpty((String) dependency.get("uniqueId"))) { + throw new IOException("Required field [uniqueId] is missing in the request for the dependent extension"); + } else if (dependency.get("version") == null) { + throw new IOException("Required field [version] is missing in the request for the dependent extension"); } + extensionDependencyList.add( + new ExtensionDependency( + dependency.get("uniqueId").toString(), + Version.fromString(dependency.get("version").toString()) + ) + ); } } + + Map additionalSettingsMap = extensionMap.entrySet() + .stream() + .filter(kv -> additionalSettingsKeys.contains(kv.getKey())) + .collect(Collectors.toMap(map -> map.getKey(), map -> map.getValue())); + + Settings.Builder output = Settings.builder(); + output.loadFromMap(additionalSettingsMap); + extAdditionalSettings.applySettings(output.build()); + + // Create extension read from initialization request + name = extensionMap.get("name").toString(); + uniqueId = extensionMap.get("uniqueId").toString(); + hostAddress = extensionMap.get("hostAddress").toString(); + port = extensionMap.get("port").toString(); + version = extensionMap.get("version").toString(); + openSearchVersion = extensionMap.get("opensearchVersion").toString(); + minimumCompatibleVersion = extensionMap.get("minimumCompatibleVersion").toString(); + dependencies = extensionDependencyList; } catch (IOException e) { - throw new IOException("Missing attribute", e); + logger.warn("loading extension has been failed because of exception : " + e.getMessage()); + return channel -> channel.sendResponse(new BytesRestResponse(RestStatus.INTERNAL_SERVER_ERROR, e.getMessage())); } Extension extension = new Extension( @@ -103,8 +155,7 @@ public RestChannelConsumer prepareRequest(RestRequest request, NodeClient client openSearchVersion, minimumCompatibleVersion, dependencies, - // TODO add this to the API (https://github.com/opensearch-project/OpenSearch/issues/8032) - new ExtensionScopedSettings(Collections.emptySet()) + extAdditionalSettings ); try { extensionsManager.loadExtension(extension); diff --git a/server/src/test/java/org/opensearch/extensions/ExtensionsManagerTests.java b/server/src/test/java/org/opensearch/extensions/ExtensionsManagerTests.java index f8ec138d8eff2..713a70c6a7d3e 100644 --- a/server/src/test/java/org/opensearch/extensions/ExtensionsManagerTests.java +++ b/server/src/test/java/org/opensearch/extensions/ExtensionsManagerTests.java @@ -45,8 +45,6 @@ import org.opensearch.client.node.NodeClient; import org.opensearch.cluster.ClusterSettingsResponse; import org.opensearch.common.util.FeatureFlags; -import org.opensearch.common.xcontent.json.JsonXContent; -import org.opensearch.core.xcontent.XContentParser; import org.opensearch.env.EnvironmentSettingsResponse; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.node.DiscoveryNode; @@ -398,16 +396,6 @@ public void testExtensionDependency() throws Exception { } } - public void testParseExtensionDependency() throws Exception { - XContentParser parser = createParser(JsonXContent.jsonXContent, "{\"uniqueId\": \"test1\", \"version\": \"2.0.0\"}"); - - assertEquals(XContentParser.Token.START_OBJECT, parser.nextToken()); - ExtensionDependency dependency = ExtensionDependency.parse(parser); - - assertEquals("test1", dependency.getUniqueId()); - assertEquals(Version.fromString("2.0.0"), dependency.getVersion()); - } - public void testInitialize() throws Exception { ExtensionsManager extensionsManager = new ExtensionsManager(Set.of()); diff --git a/server/src/test/java/org/opensearch/extensions/rest/RestInitializeExtensionActionTests.java b/server/src/test/java/org/opensearch/extensions/rest/RestInitializeExtensionActionTests.java index 8d027b7fca9c2..7dd616c678e74 100644 --- a/server/src/test/java/org/opensearch/extensions/rest/RestInitializeExtensionActionTests.java +++ b/server/src/test/java/org/opensearch/extensions/rest/RestInitializeExtensionActionTests.java @@ -9,24 +9,33 @@ package org.opensearch.extensions.rest; import java.util.Collections; +import java.util.List; +import java.util.Optional; import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.function.Function; import static java.util.Collections.emptyMap; import static java.util.Collections.emptySet; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; import org.junit.After; import org.junit.Before; +import org.mockito.Mockito; import org.opensearch.Version; import org.opensearch.cluster.node.DiscoveryNode; import org.opensearch.common.bytes.BytesArray; import org.opensearch.common.io.stream.NamedWriteableRegistry; import org.opensearch.common.network.NetworkService; +import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.PageCacheRecycler; import org.opensearch.common.xcontent.XContentType; import org.opensearch.extensions.ExtensionsManager; +import org.opensearch.extensions.ExtensionsSettings; import org.opensearch.indices.breaker.NoneCircuitBreakerService; import org.opensearch.rest.RestRequest; import org.opensearch.rest.RestStatus; @@ -88,8 +97,12 @@ public void testRestInitializeExtensionActionResponse() throws Exception { ExtensionsManager extensionsManager = mock(ExtensionsManager.class); RestInitializeExtensionAction restInitializeExtensionAction = new RestInitializeExtensionAction(extensionsManager); final String content = "{\"name\":\"ad-extension\",\"uniqueId\":\"ad-extension\",\"hostAddress\":\"127.0.0.1\"," - + "\"port\":\"4532\",\"version\":\"1.0\",\"opensearchVersion\":\"3.0.0\"," - + "\"minimumCompatibleVersion\":\"3.0.0\"}"; + + "\"port\":\"4532\",\"version\":\"1.0\",\"opensearchVersion\":\"" + + Version.CURRENT.toString() + + "\"," + + "\"minimumCompatibleVersion\":\"" + + Version.CURRENT.minimumCompatibilityVersion().toString() + + "\"}"; RestRequest request = new FakeRestRequest.Builder(xContentRegistry()).withContent(new BytesArray(content), XContentType.JSON) .withMethod(RestRequest.Method.POST) .build(); @@ -106,8 +119,12 @@ public void testRestInitializeExtensionActionFailure() throws Exception { RestInitializeExtensionAction restInitializeExtensionAction = new RestInitializeExtensionAction(extensionsManager); final String content = "{\"name\":\"ad-extension\",\"uniqueId\":\"\",\"hostAddress\":\"127.0.0.1\"," - + "\"port\":\"4532\",\"version\":\"1.0\",\"opensearchVersion\":\"3.0.0\"," - + "\"minimumCompatibleVersion\":\"3.0.0\"}"; + + "\"port\":\"4532\",\"version\":\"1.0\",\"opensearchVersion\":\"" + + Version.CURRENT.toString() + + "\"," + + "\"minimumCompatibleVersion\":\"" + + Version.CURRENT.minimumCompatibilityVersion().toString() + + "\"}"; RestRequest request = new FakeRestRequest.Builder(xContentRegistry()).withContent(new BytesArray(content), XContentType.JSON) .withMethod(RestRequest.Method.POST) .build(); @@ -121,4 +138,98 @@ public void testRestInitializeExtensionActionFailure() throws Exception { ); } + public void testRestInitializeExtensionActionResponseWithAdditionalSettings() throws Exception { + Setting boolSetting = Setting.boolSetting("boolSetting", false, Setting.Property.ExtensionScope); + Setting stringSetting = Setting.simpleString("stringSetting", "default", Setting.Property.ExtensionScope); + Setting intSetting = Setting.intSetting("intSetting", 0, Setting.Property.ExtensionScope); + Setting listSetting = Setting.listSetting( + "listSetting", + List.of("first", "second", "third"), + Function.identity(), + Setting.Property.ExtensionScope + ); + ExtensionsManager extensionsManager = new ExtensionsManager(Set.of(boolSetting, stringSetting, intSetting, listSetting)); + ExtensionsManager spy = spy(extensionsManager); + + // optionally, you can stub out some methods: + when(spy.getAdditionalSettings()).thenCallRealMethod(); + Mockito.doCallRealMethod().when(spy).loadExtension(any(ExtensionsSettings.Extension.class)); + Mockito.doNothing().when(spy).initialize(); + RestInitializeExtensionAction restInitializeExtensionAction = new RestInitializeExtensionAction(spy); + final String content = "{\"name\":\"ad-extension\",\"uniqueId\":\"ad-extension\",\"hostAddress\":\"127.0.0.1\"," + + "\"port\":\"4532\",\"version\":\"1.0\",\"opensearchVersion\":\"" + + Version.CURRENT.toString() + + "\"," + + "\"minimumCompatibleVersion\":\"" + + Version.CURRENT.minimumCompatibilityVersion().toString() + + "\",\"boolSetting\":true,\"stringSetting\":\"customSetting\",\"intSetting\":5,\"listSetting\":[\"one\",\"two\",\"three\"]}"; + RestRequest request = new FakeRestRequest.Builder(xContentRegistry()).withContent(new BytesArray(content), XContentType.JSON) + .withMethod(RestRequest.Method.POST) + .build(); + + FakeRestChannel channel = new FakeRestChannel(request, false, 0); + restInitializeExtensionAction.handleRequest(request, channel, null); + + assertEquals(channel.capturedResponse().status(), RestStatus.ACCEPTED); + assertTrue(channel.capturedResponse().content().utf8ToString().contains("A request to initialize an extension has been sent.")); + + Optional extension = spy.lookupExtensionSettingsById("ad-extension"); + assertTrue(extension.isPresent()); + assertEquals(true, extension.get().getAdditionalSettings().get(boolSetting)); + assertEquals("customSetting", extension.get().getAdditionalSettings().get(stringSetting)); + assertEquals(5, extension.get().getAdditionalSettings().get(intSetting)); + + List listSettingValue = (List) extension.get().getAdditionalSettings().get(listSetting); + assertTrue(listSettingValue.contains("one")); + assertTrue(listSettingValue.contains("two")); + assertTrue(listSettingValue.contains("three")); + } + + public void testRestInitializeExtensionActionResponseWithAdditionalSettingsUsingDefault() throws Exception { + Setting boolSetting = Setting.boolSetting("boolSetting", false, Setting.Property.ExtensionScope); + Setting stringSetting = Setting.simpleString("stringSetting", "default", Setting.Property.ExtensionScope); + Setting intSetting = Setting.intSetting("intSetting", 0, Setting.Property.ExtensionScope); + Setting listSetting = Setting.listSetting( + "listSetting", + List.of("first", "second", "third"), + Function.identity(), + Setting.Property.ExtensionScope + ); + ExtensionsManager extensionsManager = new ExtensionsManager(Set.of(boolSetting, stringSetting, intSetting, listSetting)); + ExtensionsManager spy = spy(extensionsManager); + + // optionally, you can stub out some methods: + when(spy.getAdditionalSettings()).thenCallRealMethod(); + Mockito.doCallRealMethod().when(spy).loadExtension(any(ExtensionsSettings.Extension.class)); + Mockito.doNothing().when(spy).initialize(); + RestInitializeExtensionAction restInitializeExtensionAction = new RestInitializeExtensionAction(spy); + final String content = "{\"name\":\"ad-extension\",\"uniqueId\":\"ad-extension\",\"hostAddress\":\"127.0.0.1\"," + + "\"port\":\"4532\",\"version\":\"1.0\",\"opensearchVersion\":\"" + + Version.CURRENT.toString() + + "\"," + + "\"minimumCompatibleVersion\":\"" + + Version.CURRENT.minimumCompatibilityVersion().toString() + + "\"}"; + RestRequest request = new FakeRestRequest.Builder(xContentRegistry()).withContent(new BytesArray(content), XContentType.JSON) + .withMethod(RestRequest.Method.POST) + .build(); + + FakeRestChannel channel = new FakeRestChannel(request, false, 0); + restInitializeExtensionAction.handleRequest(request, channel, null); + + assertEquals(channel.capturedResponse().status(), RestStatus.ACCEPTED); + assertTrue(channel.capturedResponse().content().utf8ToString().contains("A request to initialize an extension has been sent.")); + + Optional extension = spy.lookupExtensionSettingsById("ad-extension"); + assertTrue(extension.isPresent()); + assertEquals(false, extension.get().getAdditionalSettings().get(boolSetting)); + assertEquals("default", extension.get().getAdditionalSettings().get(stringSetting)); + assertEquals(0, extension.get().getAdditionalSettings().get(intSetting)); + + List listSettingValue = (List) extension.get().getAdditionalSettings().get(listSetting); + assertTrue(listSettingValue.contains("first")); + assertTrue(listSettingValue.contains("second")); + assertTrue(listSettingValue.contains("third")); + } + } From 1f3a7a87eb64f8280bf984eed8fa821f1351b583 Mon Sep 17 00:00:00 2001 From: Andriy Redko Date: Tue, 4 Jul 2023 12:19:59 -0400 Subject: [PATCH 108/109] Update Apache HttpCore/ HttpClient and Apache HttpCore5 / HttpClient5 dependencies (#8434) Signed-off-by: Andriy Redko --- CHANGELOG.md | 1 + buildSrc/version.properties | 8 ++++---- client/rest/licenses/httpclient5-5.1.4.jar.sha1 | 1 - client/rest/licenses/httpclient5-5.2.1.jar.sha1 | 1 + client/rest/licenses/httpcore5-5.1.5.jar.sha1 | 1 - client/rest/licenses/httpcore5-5.2.2.jar.sha1 | 1 + client/rest/licenses/httpcore5-h2-5.1.5.jar.sha1 | 1 - client/rest/licenses/httpcore5-h2-5.2.2.jar.sha1 | 1 + client/sniffer/licenses/httpclient5-5.1.4.jar.sha1 | 1 - client/sniffer/licenses/httpclient5-5.2.1.jar.sha1 | 1 + client/sniffer/licenses/httpcore5-5.1.5.jar.sha1 | 1 - client/sniffer/licenses/httpcore5-5.2.2.jar.sha1 | 1 + .../licenses/httpclient-4.5.13.jar.sha1 | 1 - .../licenses/httpclient-4.5.14.jar.sha1 | 1 + .../licenses/httpcore-4.4.15.jar.sha1 | 1 - .../licenses/httpcore-4.4.16.jar.sha1 | 1 + plugins/discovery-ec2/licenses/httpclient-4.5.13.jar.sha1 | 1 - plugins/discovery-ec2/licenses/httpclient-4.5.14.jar.sha1 | 1 + plugins/discovery-ec2/licenses/httpcore-4.4.15.jar.sha1 | 1 - plugins/discovery-ec2/licenses/httpcore-4.4.16.jar.sha1 | 1 + plugins/discovery-gce/licenses/httpclient-4.5.13.jar.sha1 | 1 - plugins/discovery-gce/licenses/httpclient-4.5.14.jar.sha1 | 1 + plugins/discovery-gce/licenses/httpcore-4.4.15.jar.sha1 | 1 - plugins/discovery-gce/licenses/httpcore-4.4.16.jar.sha1 | 1 + plugins/repository-s3/licenses/httpclient-4.5.13.jar.sha1 | 1 - plugins/repository-s3/licenses/httpclient-4.5.14.jar.sha1 | 1 + plugins/repository-s3/licenses/httpcore-4.4.15.jar.sha1 | 1 - plugins/repository-s3/licenses/httpcore-4.4.16.jar.sha1 | 1 + 28 files changed, 18 insertions(+), 17 deletions(-) delete mode 100644 client/rest/licenses/httpclient5-5.1.4.jar.sha1 create mode 100644 client/rest/licenses/httpclient5-5.2.1.jar.sha1 delete mode 100644 client/rest/licenses/httpcore5-5.1.5.jar.sha1 create mode 100644 client/rest/licenses/httpcore5-5.2.2.jar.sha1 delete mode 100644 client/rest/licenses/httpcore5-h2-5.1.5.jar.sha1 create mode 100644 client/rest/licenses/httpcore5-h2-5.2.2.jar.sha1 delete mode 100644 client/sniffer/licenses/httpclient5-5.1.4.jar.sha1 create mode 100644 client/sniffer/licenses/httpclient5-5.2.1.jar.sha1 delete mode 100644 client/sniffer/licenses/httpcore5-5.1.5.jar.sha1 create mode 100644 client/sniffer/licenses/httpcore5-5.2.2.jar.sha1 delete mode 100644 plugins/discovery-azure-classic/licenses/httpclient-4.5.13.jar.sha1 create mode 100644 plugins/discovery-azure-classic/licenses/httpclient-4.5.14.jar.sha1 delete mode 100644 plugins/discovery-azure-classic/licenses/httpcore-4.4.15.jar.sha1 create mode 100644 plugins/discovery-azure-classic/licenses/httpcore-4.4.16.jar.sha1 delete mode 100644 plugins/discovery-ec2/licenses/httpclient-4.5.13.jar.sha1 create mode 100644 plugins/discovery-ec2/licenses/httpclient-4.5.14.jar.sha1 delete mode 100644 plugins/discovery-ec2/licenses/httpcore-4.4.15.jar.sha1 create mode 100644 plugins/discovery-ec2/licenses/httpcore-4.4.16.jar.sha1 delete mode 100644 plugins/discovery-gce/licenses/httpclient-4.5.13.jar.sha1 create mode 100644 plugins/discovery-gce/licenses/httpclient-4.5.14.jar.sha1 delete mode 100644 plugins/discovery-gce/licenses/httpcore-4.4.15.jar.sha1 create mode 100644 plugins/discovery-gce/licenses/httpcore-4.4.16.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/httpclient-4.5.13.jar.sha1 create mode 100644 plugins/repository-s3/licenses/httpclient-4.5.14.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/httpcore-4.4.15.jar.sha1 create mode 100644 plugins/repository-s3/licenses/httpcore-4.4.16.jar.sha1 diff --git a/CHANGELOG.md b/CHANGELOG.md index 0a8deb28833e6..1d10c55fc8064 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -128,6 +128,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - [Upgrade] Lucene 9.7.0 release (#8272) - Bump `org.jboss.resteasy:resteasy-jackson2-provider` from 3.0.26.Final to 6.2.4.Final in /qa/wildfly ([#8209](https://github.com/opensearch-project/OpenSearch/pull/8209)) - Bump `com.google.api-client:google-api-client` from 1.34.0 to 2.2.0 ([#8276](https://github.com/opensearch-project/OpenSearch/pull/8276)) +- Update Apache HttpCore/ HttpClient and Apache HttpCore5 / HttpClient5 dependencies ([#8434](https://github.com/opensearch-project/OpenSearch/pull/8434)) ### Changed - Replace jboss-annotations-api_1.2_spec with jakarta.annotation-api ([#7836](https://github.com/opensearch-project/OpenSearch/pull/7836)) diff --git a/buildSrc/version.properties b/buildSrc/version.properties index f9eac9516cb18..408b03e60cc5d 100644 --- a/buildSrc/version.properties +++ b/buildSrc/version.properties @@ -33,10 +33,10 @@ netty = 4.1.94.Final joda = 2.12.2 # client dependencies -httpclient5 = 5.1.4 -httpcore5 = 5.1.5 -httpclient = 4.5.13 -httpcore = 4.4.15 +httpclient5 = 5.2.1 +httpcore5 = 5.2.2 +httpclient = 4.5.14 +httpcore = 4.4.16 httpasyncclient = 4.1.5 commonslogging = 1.2 commonscodec = 1.15 diff --git a/client/rest/licenses/httpclient5-5.1.4.jar.sha1 b/client/rest/licenses/httpclient5-5.1.4.jar.sha1 deleted file mode 100644 index 3c0cb1335fb88..0000000000000 --- a/client/rest/licenses/httpclient5-5.1.4.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -208f9eed6d6ab709e2ae7a75b457ef60c0baefa5 \ No newline at end of file diff --git a/client/rest/licenses/httpclient5-5.2.1.jar.sha1 b/client/rest/licenses/httpclient5-5.2.1.jar.sha1 new file mode 100644 index 0000000000000..3555fe22f8e12 --- /dev/null +++ b/client/rest/licenses/httpclient5-5.2.1.jar.sha1 @@ -0,0 +1 @@ +0c900514d3446d9ce5d9dbd90c21192048125440 \ No newline at end of file diff --git a/client/rest/licenses/httpcore5-5.1.5.jar.sha1 b/client/rest/licenses/httpcore5-5.1.5.jar.sha1 deleted file mode 100644 index 8da253152e970..0000000000000 --- a/client/rest/licenses/httpcore5-5.1.5.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -df9da3a1fa2351c4790245400ed28d78a8ddd3fc \ No newline at end of file diff --git a/client/rest/licenses/httpcore5-5.2.2.jar.sha1 b/client/rest/licenses/httpcore5-5.2.2.jar.sha1 new file mode 100644 index 0000000000000..b641256c7d4a4 --- /dev/null +++ b/client/rest/licenses/httpcore5-5.2.2.jar.sha1 @@ -0,0 +1 @@ +6da28f5aa6c2b129ef49632e041a5203ce7507b2 \ No newline at end of file diff --git a/client/rest/licenses/httpcore5-h2-5.1.5.jar.sha1 b/client/rest/licenses/httpcore5-h2-5.1.5.jar.sha1 deleted file mode 100644 index 097e6cc2a3be8..0000000000000 --- a/client/rest/licenses/httpcore5-h2-5.1.5.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -624660339afd5006d427457e6b10b10b32fd86f1 \ No newline at end of file diff --git a/client/rest/licenses/httpcore5-h2-5.2.2.jar.sha1 b/client/rest/licenses/httpcore5-h2-5.2.2.jar.sha1 new file mode 100644 index 0000000000000..94bc0fa49bdb0 --- /dev/null +++ b/client/rest/licenses/httpcore5-h2-5.2.2.jar.sha1 @@ -0,0 +1 @@ +54ee1ed58fe8ac40be1083ea9873a6c734939ab9 \ No newline at end of file diff --git a/client/sniffer/licenses/httpclient5-5.1.4.jar.sha1 b/client/sniffer/licenses/httpclient5-5.1.4.jar.sha1 deleted file mode 100644 index 3c0cb1335fb88..0000000000000 --- a/client/sniffer/licenses/httpclient5-5.1.4.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -208f9eed6d6ab709e2ae7a75b457ef60c0baefa5 \ No newline at end of file diff --git a/client/sniffer/licenses/httpclient5-5.2.1.jar.sha1 b/client/sniffer/licenses/httpclient5-5.2.1.jar.sha1 new file mode 100644 index 0000000000000..3555fe22f8e12 --- /dev/null +++ b/client/sniffer/licenses/httpclient5-5.2.1.jar.sha1 @@ -0,0 +1 @@ +0c900514d3446d9ce5d9dbd90c21192048125440 \ No newline at end of file diff --git a/client/sniffer/licenses/httpcore5-5.1.5.jar.sha1 b/client/sniffer/licenses/httpcore5-5.1.5.jar.sha1 deleted file mode 100644 index 8da253152e970..0000000000000 --- a/client/sniffer/licenses/httpcore5-5.1.5.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -df9da3a1fa2351c4790245400ed28d78a8ddd3fc \ No newline at end of file diff --git a/client/sniffer/licenses/httpcore5-5.2.2.jar.sha1 b/client/sniffer/licenses/httpcore5-5.2.2.jar.sha1 new file mode 100644 index 0000000000000..b641256c7d4a4 --- /dev/null +++ b/client/sniffer/licenses/httpcore5-5.2.2.jar.sha1 @@ -0,0 +1 @@ +6da28f5aa6c2b129ef49632e041a5203ce7507b2 \ No newline at end of file diff --git a/plugins/discovery-azure-classic/licenses/httpclient-4.5.13.jar.sha1 b/plugins/discovery-azure-classic/licenses/httpclient-4.5.13.jar.sha1 deleted file mode 100644 index 3281e21595b39..0000000000000 --- a/plugins/discovery-azure-classic/licenses/httpclient-4.5.13.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -e5f6cae5ca7ecaac1ec2827a9e2d65ae2869cada \ No newline at end of file diff --git a/plugins/discovery-azure-classic/licenses/httpclient-4.5.14.jar.sha1 b/plugins/discovery-azure-classic/licenses/httpclient-4.5.14.jar.sha1 new file mode 100644 index 0000000000000..66e05851c2e3c --- /dev/null +++ b/plugins/discovery-azure-classic/licenses/httpclient-4.5.14.jar.sha1 @@ -0,0 +1 @@ +1194890e6f56ec29177673f2f12d0b8e627dec98 \ No newline at end of file diff --git a/plugins/discovery-azure-classic/licenses/httpcore-4.4.15.jar.sha1 b/plugins/discovery-azure-classic/licenses/httpcore-4.4.15.jar.sha1 deleted file mode 100644 index 42a03b5d7a376..0000000000000 --- a/plugins/discovery-azure-classic/licenses/httpcore-4.4.15.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -7f2e0c573eaa7a74bac2e89b359e1f73d92a0a1d \ No newline at end of file diff --git a/plugins/discovery-azure-classic/licenses/httpcore-4.4.16.jar.sha1 b/plugins/discovery-azure-classic/licenses/httpcore-4.4.16.jar.sha1 new file mode 100644 index 0000000000000..172110694b5bd --- /dev/null +++ b/plugins/discovery-azure-classic/licenses/httpcore-4.4.16.jar.sha1 @@ -0,0 +1 @@ +51cf043c87253c9f58b539c9f7e44c8894223850 \ No newline at end of file diff --git a/plugins/discovery-ec2/licenses/httpclient-4.5.13.jar.sha1 b/plugins/discovery-ec2/licenses/httpclient-4.5.13.jar.sha1 deleted file mode 100644 index 3281e21595b39..0000000000000 --- a/plugins/discovery-ec2/licenses/httpclient-4.5.13.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -e5f6cae5ca7ecaac1ec2827a9e2d65ae2869cada \ No newline at end of file diff --git a/plugins/discovery-ec2/licenses/httpclient-4.5.14.jar.sha1 b/plugins/discovery-ec2/licenses/httpclient-4.5.14.jar.sha1 new file mode 100644 index 0000000000000..66e05851c2e3c --- /dev/null +++ b/plugins/discovery-ec2/licenses/httpclient-4.5.14.jar.sha1 @@ -0,0 +1 @@ +1194890e6f56ec29177673f2f12d0b8e627dec98 \ No newline at end of file diff --git a/plugins/discovery-ec2/licenses/httpcore-4.4.15.jar.sha1 b/plugins/discovery-ec2/licenses/httpcore-4.4.15.jar.sha1 deleted file mode 100644 index 42a03b5d7a376..0000000000000 --- a/plugins/discovery-ec2/licenses/httpcore-4.4.15.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -7f2e0c573eaa7a74bac2e89b359e1f73d92a0a1d \ No newline at end of file diff --git a/plugins/discovery-ec2/licenses/httpcore-4.4.16.jar.sha1 b/plugins/discovery-ec2/licenses/httpcore-4.4.16.jar.sha1 new file mode 100644 index 0000000000000..172110694b5bd --- /dev/null +++ b/plugins/discovery-ec2/licenses/httpcore-4.4.16.jar.sha1 @@ -0,0 +1 @@ +51cf043c87253c9f58b539c9f7e44c8894223850 \ No newline at end of file diff --git a/plugins/discovery-gce/licenses/httpclient-4.5.13.jar.sha1 b/plugins/discovery-gce/licenses/httpclient-4.5.13.jar.sha1 deleted file mode 100644 index 3281e21595b39..0000000000000 --- a/plugins/discovery-gce/licenses/httpclient-4.5.13.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -e5f6cae5ca7ecaac1ec2827a9e2d65ae2869cada \ No newline at end of file diff --git a/plugins/discovery-gce/licenses/httpclient-4.5.14.jar.sha1 b/plugins/discovery-gce/licenses/httpclient-4.5.14.jar.sha1 new file mode 100644 index 0000000000000..66e05851c2e3c --- /dev/null +++ b/plugins/discovery-gce/licenses/httpclient-4.5.14.jar.sha1 @@ -0,0 +1 @@ +1194890e6f56ec29177673f2f12d0b8e627dec98 \ No newline at end of file diff --git a/plugins/discovery-gce/licenses/httpcore-4.4.15.jar.sha1 b/plugins/discovery-gce/licenses/httpcore-4.4.15.jar.sha1 deleted file mode 100644 index 42a03b5d7a376..0000000000000 --- a/plugins/discovery-gce/licenses/httpcore-4.4.15.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -7f2e0c573eaa7a74bac2e89b359e1f73d92a0a1d \ No newline at end of file diff --git a/plugins/discovery-gce/licenses/httpcore-4.4.16.jar.sha1 b/plugins/discovery-gce/licenses/httpcore-4.4.16.jar.sha1 new file mode 100644 index 0000000000000..172110694b5bd --- /dev/null +++ b/plugins/discovery-gce/licenses/httpcore-4.4.16.jar.sha1 @@ -0,0 +1 @@ +51cf043c87253c9f58b539c9f7e44c8894223850 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/httpclient-4.5.13.jar.sha1 b/plugins/repository-s3/licenses/httpclient-4.5.13.jar.sha1 deleted file mode 100644 index 3281e21595b39..0000000000000 --- a/plugins/repository-s3/licenses/httpclient-4.5.13.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -e5f6cae5ca7ecaac1ec2827a9e2d65ae2869cada \ No newline at end of file diff --git a/plugins/repository-s3/licenses/httpclient-4.5.14.jar.sha1 b/plugins/repository-s3/licenses/httpclient-4.5.14.jar.sha1 new file mode 100644 index 0000000000000..66e05851c2e3c --- /dev/null +++ b/plugins/repository-s3/licenses/httpclient-4.5.14.jar.sha1 @@ -0,0 +1 @@ +1194890e6f56ec29177673f2f12d0b8e627dec98 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/httpcore-4.4.15.jar.sha1 b/plugins/repository-s3/licenses/httpcore-4.4.15.jar.sha1 deleted file mode 100644 index 42a03b5d7a376..0000000000000 --- a/plugins/repository-s3/licenses/httpcore-4.4.15.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -7f2e0c573eaa7a74bac2e89b359e1f73d92a0a1d \ No newline at end of file diff --git a/plugins/repository-s3/licenses/httpcore-4.4.16.jar.sha1 b/plugins/repository-s3/licenses/httpcore-4.4.16.jar.sha1 new file mode 100644 index 0000000000000..172110694b5bd --- /dev/null +++ b/plugins/repository-s3/licenses/httpcore-4.4.16.jar.sha1 @@ -0,0 +1 @@ +51cf043c87253c9f58b539c9f7e44c8894223850 \ No newline at end of file From 71c9302c022d7fb8d461b0ec7e7810e0e04d0799 Mon Sep 17 00:00:00 2001 From: Ashish Date: Wed, 5 Jul 2023 10:45:12 +0530 Subject: [PATCH 109/109] [Remote translog] Add integration tests for primary term validation (#8406) --------- Signed-off-by: Ashish Singh --- .../remotestore/PrimaryTermValidationIT.java | 166 ++++++++++++++++++ .../RemoteStoreBaseIntegTestCase.java | 4 +- .../opensearch/remotestore/RemoteStoreIT.java | 6 + .../remotestore/RemoteStoreStatsIT.java | 6 + .../ReplicaToPrimaryPromotionIT.java | 6 + 5 files changed, 185 insertions(+), 3 deletions(-) create mode 100644 server/src/internalClusterTest/java/org/opensearch/remotestore/PrimaryTermValidationIT.java diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/PrimaryTermValidationIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/PrimaryTermValidationIT.java new file mode 100644 index 0000000000000..6691da81f057d --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/PrimaryTermValidationIT.java @@ -0,0 +1,166 @@ +/* + * 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.remotestore; + +import org.opensearch.action.admin.cluster.health.ClusterHealthRequest; +import org.opensearch.action.admin.cluster.health.ClusterHealthResponse; +import org.opensearch.action.admin.cluster.node.info.NodesInfoResponse; +import org.opensearch.action.admin.indices.refresh.RefreshResponse; +import org.opensearch.action.index.IndexResponse; +import org.opensearch.action.support.IndicesOptions; +import org.opensearch.cluster.coordination.FollowersChecker; +import org.opensearch.cluster.coordination.LeaderChecker; +import org.opensearch.cluster.health.ClusterHealthStatus; +import org.opensearch.cluster.health.ClusterIndexHealth; +import org.opensearch.common.UUIDs; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.xcontent.XContentType; +import org.opensearch.index.shard.ShardNotFoundException; +import org.opensearch.plugins.Plugin; +import org.opensearch.test.OpenSearchIntegTestCase; +import org.opensearch.test.disruption.NetworkDisruption; +import org.opensearch.test.transport.MockTransportService; + +import java.util.Arrays; +import java.util.Collection; +import java.util.HashSet; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.hamcrest.Matchers.equalTo; +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertHitCount; +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertNoFailures; + +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0) + +public class PrimaryTermValidationIT extends RemoteStoreBaseIntegTestCase { + + private static final String INDEX_NAME = "remote-store-test-idx-1"; + + @Override + protected Collection> nodePlugins() { + return Arrays.asList(MockTransportService.TestPlugin.class); + } + + public void testPrimaryTermValidation() throws Exception { + // Follower checker interval is lower compared to leader checker so that the cluster manager can remove the node + // with network partition faster. The follower check retry count is also kept 1. + Settings clusterSettings = Settings.builder() + .put(LeaderChecker.LEADER_CHECK_TIMEOUT_SETTING.getKey(), "1s") + .put(LeaderChecker.LEADER_CHECK_INTERVAL_SETTING.getKey(), "20s") + .put(LeaderChecker.LEADER_CHECK_RETRY_COUNT_SETTING.getKey(), 4) + .put(FollowersChecker.FOLLOWER_CHECK_TIMEOUT_SETTING.getKey(), "1s") + .put(FollowersChecker.FOLLOWER_CHECK_INTERVAL_SETTING.getKey(), "1s") + .put(FollowersChecker.FOLLOWER_CHECK_RETRY_COUNT_SETTING.getKey(), 1) + .build(); + internalCluster().startClusterManagerOnlyNode(clusterSettings); + + // Create repository + absolutePath = randomRepoPath().toAbsolutePath(); + assertAcked( + clusterAdmin().preparePutRepository(REPOSITORY_NAME).setType("fs").setSettings(Settings.builder().put("location", absolutePath)) + ); + + // Start data nodes and create index + internalCluster().startDataOnlyNodes(2, clusterSettings); + createIndex(INDEX_NAME, remoteTranslogIndexSettings(1)); + ensureYellowAndNoInitializingShards(INDEX_NAME); + ensureGreen(INDEX_NAME); + + // Get the names of nodes to create network disruption + String primaryNode = primaryNodeName(INDEX_NAME); + String replicaNode = replicaNodeName(INDEX_NAME); + String clusterManagerNode = internalCluster().getClusterManagerName(); + logger.info("Node names : clusterManager={} primary={} replica={}", clusterManagerNode, primaryNode, replicaNode); + + // Index some docs and validate that both primary and replica node has it. Refresh is triggered to trigger segment replication + // to ensure replica is also upto date. + int numOfDocs = randomIntBetween(5, 10); + for (int i = 0; i < numOfDocs; i++) { + indexSameDoc(clusterManagerNode, INDEX_NAME); + } + refresh(INDEX_NAME); + assertBusy( + () -> assertHitCount(client(primaryNode).prepareSearch(INDEX_NAME).setSize(0).setPreference("_only_local").get(), numOfDocs) + ); + assertBusy( + () -> assertHitCount(client(replicaNode).prepareSearch(INDEX_NAME).setSize(0).setPreference("_only_local").get(), numOfDocs) + ); + + // Start network disruption - primary node will be isolated + Set nodesInOneSide = Stream.of(clusterManagerNode, replicaNode).collect(Collectors.toCollection(HashSet::new)); + Set nodesInOtherSide = Stream.of(primaryNode).collect(Collectors.toCollection(HashSet::new)); + NetworkDisruption networkDisruption = new NetworkDisruption( + new NetworkDisruption.TwoPartitions(nodesInOneSide, nodesInOtherSide), + NetworkDisruption.DISCONNECT + ); + internalCluster().setDisruptionScheme(networkDisruption); + logger.info("--> network disruption is started"); + networkDisruption.startDisrupting(); + + // Ensure the node which is partitioned is removed from the cluster + assertBusy(() -> { + NodesInfoResponse response = client(clusterManagerNode).admin().cluster().prepareNodesInfo().get(); + assertThat(response.getNodes().size(), equalTo(2)); + }); + + // Ensure that the cluster manager has latest information about the index + assertBusy(() -> { + ClusterHealthResponse clusterHealthResponse = client(clusterManagerNode).admin() + .cluster() + .health(new ClusterHealthRequest()) + .actionGet(TimeValue.timeValueSeconds(1)); + assertTrue(clusterHealthResponse.getIndices().containsKey(INDEX_NAME)); + ClusterIndexHealth clusterIndexHealth = clusterHealthResponse.getIndices().get(INDEX_NAME); + assertEquals(ClusterHealthStatus.YELLOW, clusterHealthResponse.getStatus()); + assertEquals(1, clusterIndexHealth.getNumberOfShards()); + assertEquals(1, clusterIndexHealth.getActiveShards()); + assertEquals(1, clusterIndexHealth.getUnassignedShards()); + assertEquals(1, clusterIndexHealth.getUnassignedShards()); + assertEquals(1, clusterIndexHealth.getActivePrimaryShards()); + assertEquals(ClusterHealthStatus.YELLOW, clusterIndexHealth.getStatus()); + }); + + // Index data to the newly promoted primary + indexSameDoc(clusterManagerNode, INDEX_NAME); + RefreshResponse refreshResponse = client(clusterManagerNode).admin() + .indices() + .prepareRefresh(INDEX_NAME) + .setIndicesOptions(IndicesOptions.STRICT_EXPAND_OPEN_HIDDEN_FORBID_CLOSED) + .execute() + .actionGet(); + assertNoFailures(refreshResponse); + assertEquals(1, refreshResponse.getSuccessfulShards()); + assertHitCount(client(replicaNode).prepareSearch(INDEX_NAME).setSize(0).setPreference("_only_local").get(), numOfDocs + 1); + + // At this point we stop the disruption. Since the follower checker has already failed and cluster manager has removed the node + // from cluster, failed node needs to start discovery process by leader checker call. We stop the disruption to allow the failed + // node to + // communicate with the other node which it assumes has replica. + networkDisruption.stopDisrupting(); + + // When the index call is made to the stale primary, it makes the primary term validation call to the other node (which + // it assumes has the replica node). At this moment, the stale primary realises that it is no more the primary and the caller + // received the following exception. + ShardNotFoundException exception = assertThrows(ShardNotFoundException.class, () -> indexSameDoc(primaryNode, INDEX_NAME)); + assertTrue(exception.getMessage().contains("no such shard")); + ensureStableCluster(3); + ensureGreen(INDEX_NAME); + } + + private IndexResponse indexSameDoc(String nodeName, String indexName) { + return client(nodeName).prepareIndex(indexName) + .setId(UUIDs.randomBase64UUID()) + .setSource("{\"foo\" : \"bar\"}", XContentType.JSON) + .get(); + } +} diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java index 336646b35b5a6..2b3fcadfc645e 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreBaseIntegTestCase.java @@ -9,7 +9,6 @@ package org.opensearch.remotestore; import org.junit.After; -import org.junit.Before; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.FeatureFlags; @@ -100,8 +99,7 @@ protected void putRepository(Path path) { ); } - @Before - public void setup() { + protected void setupRepo() { internalCluster().startClusterManagerOnlyNode(); absolutePath = randomRepoPath().toAbsolutePath(); assertAcked( diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java index f6ba8cfed00d0..77de601b53ec6 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreIT.java @@ -8,6 +8,7 @@ package org.opensearch.remotestore; +import org.junit.Before; import org.opensearch.action.admin.cluster.remotestore.restore.RestoreRemoteStoreRequest; import org.opensearch.action.admin.indices.delete.DeleteIndexRequest; import org.opensearch.action.admin.indices.recovery.RecoveryResponse; @@ -51,6 +52,11 @@ protected Collection> nodePlugins() { return Arrays.asList(MockTransportService.TestPlugin.class); } + @Before + public void setup() { + setupRepo(); + } + @Override public Settings indexSettings() { return remoteStoreIndexSettings(0); diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreStatsIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreStatsIT.java index 0ea87d106c14e..0e4774c1f3454 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreStatsIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreStatsIT.java @@ -8,6 +8,7 @@ package org.opensearch.remotestore; +import org.junit.Before; import org.opensearch.action.admin.cluster.remotestore.stats.RemoteStoreStats; import org.opensearch.action.admin.cluster.remotestore.stats.RemoteStoreStatsRequestBuilder; import org.opensearch.action.admin.cluster.remotestore.stats.RemoteStoreStatsResponse; @@ -28,6 +29,11 @@ public class RemoteStoreStatsIT extends RemoteStoreBaseIntegTestCase { private static final String INDEX_NAME = "remote-store-test-idx-1"; + @Before + public void setup() { + setupRepo(); + } + public void testStatsResponseFromAllNodes() { // Step 1 - We create cluster, create an index, and then index documents into. We also do multiple refreshes/flushes diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/ReplicaToPrimaryPromotionIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/ReplicaToPrimaryPromotionIT.java index 712747f7479ae..0f3e041dd429a 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/ReplicaToPrimaryPromotionIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/ReplicaToPrimaryPromotionIT.java @@ -9,6 +9,7 @@ package org.opensearch.remotestore; import com.carrotsearch.randomizedtesting.RandomizedTest; +import org.junit.Before; import org.opensearch.action.admin.indices.close.CloseIndexResponse; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexMetadata; @@ -31,6 +32,11 @@ public class ReplicaToPrimaryPromotionIT extends RemoteStoreBaseIntegTestCase { private int shard_count = 5; + @Before + public void setup() { + setupRepo(); + } + @Override public Settings indexSettings() { return Settings.builder()