Remove much remaining frozen indices code (#120539)

This removes the transport action and the index setting.

The deprecation check has to go because the setting it uses to detect
frozen indices has gone. We are issueing a critical deprecation
warning in 8.last telling users to unfreeze them.

A lot of integration tests under `x-pack/plugin/sql` also have to go,
because they rely on being able to freeze indices.

ES-9736 #comment Removed the transport action and the index setting and so on in https://github.com/elastic/elasticsearch/pull/120539
This commit is contained in:
Pete Gillin 2025-01-29 10:49:15 +00:00 committed by GitHub
parent d2814d5b09
commit f7bf727570
No known key found for this signature in database
GPG key ID: B5690EEEBB952194
47 changed files with 38 additions and 702 deletions

View file

@ -66,7 +66,6 @@ module org.elasticsearch.xcore {
exports org.elasticsearch.xpack.core.esql;
exports org.elasticsearch.xpack.core.esql.action;
exports org.elasticsearch.xpack.core.esql.action.internal; // TODO: qualify to esql when modularized
exports org.elasticsearch.xpack.core.frozen.action;
exports org.elasticsearch.xpack.core.frozen;
exports org.elasticsearch.xpack.core.graph.action;
exports org.elasticsearch.xpack.core.graph;

View file

@ -17,7 +17,6 @@ import org.apache.lucene.search.ReferenceManager;
import org.apache.lucene.store.Directory;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.core.IOUtils;
import org.elasticsearch.core.SuppressForbidden;
import org.elasticsearch.index.engine.Engine;
@ -55,12 +54,7 @@ import java.util.function.Function;
* stats in order to obtain the number of reopens.
*/
public final class FrozenEngine extends ReadOnlyEngine {
public static final Setting<Boolean> INDEX_FROZEN = Setting.boolSetting(
"index.frozen",
false,
Setting.Property.IndexScope,
Setting.Property.PrivateIndex
);
private final SegmentsStats segmentsStats;
private final DocsStats docsStats;
private volatile ElasticsearchDirectoryReader lastOpenedReader;

View file

@ -1,20 +0,0 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License
* 2.0; you may not use this file except in compliance with the Elastic License
* 2.0.
*/
package org.elasticsearch.xpack.core.frozen.action;
import org.elasticsearch.action.ActionType;
import org.elasticsearch.protocol.xpack.frozen.FreezeResponse;
public class FreezeIndexAction extends ActionType<FreezeResponse> {
public static final FreezeIndexAction INSTANCE = new FreezeIndexAction();
public static final String NAME = "indices:admin/freeze";
private FreezeIndexAction() {
super(NAME);
}
}

View file

@ -15,7 +15,6 @@ import org.elasticsearch.common.time.LegacyFormatNames;
import org.elasticsearch.index.IndexModule;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.IndexVersion;
import org.elasticsearch.index.engine.frozen.FrozenEngine;
import org.elasticsearch.xpack.core.deprecation.DeprecatedIndexPredicate;
import org.elasticsearch.xpack.core.deprecation.DeprecationIssue;
@ -71,7 +70,6 @@ public class IndexDeprecationChecker implements ResourceDeprecationChecker {
IndexDeprecationChecker::translogRetentionSettingCheck,
IndexDeprecationChecker::checkIndexDataPath,
IndexDeprecationChecker::storeTypeSettingCheck,
IndexDeprecationChecker::frozenIndexSettingCheck,
IndexDeprecationChecker::deprecatedCamelCasePattern,
IndexDeprecationChecker::legacyRoutingSettingCheck
);
@ -189,24 +187,6 @@ public class IndexDeprecationChecker implements ResourceDeprecationChecker {
return null;
}
private static DeprecationIssue frozenIndexSettingCheck(IndexMetadata indexMetadata, ClusterState clusterState) {
Boolean isIndexFrozen = FrozenEngine.INDEX_FROZEN.get(indexMetadata.getSettings());
if (Boolean.TRUE.equals(isIndexFrozen)) {
String indexName = indexMetadata.getIndex().getName();
return new DeprecationIssue(
DeprecationIssue.Level.WARNING,
"index ["
+ indexName
+ "] is a frozen index. The frozen indices feature is deprecated and will be removed in a future version",
"https://www.elastic.co/guide/en/elasticsearch/reference/master/frozen-indices.html",
"Frozen indices no longer offer any advantages. Consider cold or frozen tiers in place of frozen indices.",
false,
null
);
}
return null;
}
private static DeprecationIssue legacyRoutingSettingCheck(IndexMetadata indexMetadata, ClusterState clusterState) {
List<String> deprecatedSettings = LegacyTiersDetection.getDeprecatedFilteredAllocationSettings(indexMetadata.getSettings());
if (deprecatedSettings.isEmpty()) {

View file

@ -21,7 +21,6 @@ import org.elasticsearch.core.TimeValue;
import org.elasticsearch.index.IndexMode;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.IndexVersion;
import org.elasticsearch.index.engine.frozen.FrozenEngine;
import org.elasticsearch.indices.TestIndexNameExpressionResolver;
import org.elasticsearch.snapshots.SearchableSnapshotsSettings;
import org.elasticsearch.test.ESTestCase;
@ -328,30 +327,6 @@ public class IndexDeprecationCheckerTests extends ESTestCase {
);
}
public void testFrozenIndex() {
Settings.Builder settings = settings(IndexVersion.current());
settings.put(FrozenEngine.INDEX_FROZEN.getKey(), true);
IndexMetadata indexMetadata = IndexMetadata.builder("test").settings(settings).numberOfShards(1).numberOfReplicas(0).build();
ClusterState state = ClusterState.builder(ClusterState.EMPTY_STATE).metadata(Metadata.builder().put(indexMetadata, true)).build();
Map<String, List<DeprecationIssue>> issuesByIndex = checker.check(
state,
new DeprecationInfoAction.Request(TimeValue.THIRTY_SECONDS)
);
assertThat(
issuesByIndex.get("test"),
contains(
new DeprecationIssue(
DeprecationIssue.Level.WARNING,
"index [test] is a frozen index. The frozen indices feature is deprecated and will be removed in a future version",
"https://www.elastic.co/guide/en/elasticsearch/reference/master/frozen-indices.html",
"Frozen indices no longer offer any advantages. Consider cold or frozen tiers in place of frozen indices.",
false,
null
)
)
);
}
public void testCamelCaseDeprecation() {
String simpleMapping = "{\n\"_doc\": {"
+ "\"properties\" : {\n"

View file

@ -8,30 +8,19 @@ package org.elasticsearch.xpack.frozen;
import org.elasticsearch.action.ActionRequest;
import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.index.engine.frozen.FrozenEngine;
import org.elasticsearch.plugins.ActionPlugin;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.xpack.core.action.XPackUsageFeatureAction;
import org.elasticsearch.xpack.core.frozen.action.FreezeIndexAction;
import org.elasticsearch.xpack.frozen.action.TransportFreezeIndexAction;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
public class FrozenIndices extends Plugin implements ActionPlugin {
@Override
public List<Setting<?>> getSettings() {
return Arrays.asList(FrozenEngine.INDEX_FROZEN);
}
@Override
public List<ActionHandler<? extends ActionRequest, ? extends ActionResponse>> getActions() {
List<ActionHandler<? extends ActionRequest, ? extends ActionResponse>> actions = new ArrayList<>();
actions.add(new ActionHandler<>(XPackUsageFeatureAction.FROZEN_INDICES, FrozenIndicesUsageTransportAction.class));
actions.add(new ActionHandler<>(FreezeIndexAction.INSTANCE, TransportFreezeIndexAction.class));
return actions;
}
}

View file

@ -1,233 +0,0 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License
* 2.0; you may not use this file except in compliance with the Elastic License
* 2.0.
*/
package org.elasticsearch.xpack.frozen.action;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.ResourceNotFoundException;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.admin.indices.close.CloseIndexClusterStateUpdateRequest;
import org.elasticsearch.action.admin.indices.close.CloseIndexResponse;
import org.elasticsearch.action.admin.indices.open.OpenIndexClusterStateUpdateRequest;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.ActiveShardCount;
import org.elasticsearch.action.support.DestructiveOperations;
import org.elasticsearch.action.support.master.TransportMasterNodeAction;
import org.elasticsearch.cluster.AckedClusterStateUpdateTask;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateUpdateTask;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.block.ClusterBlocks;
import org.elasticsearch.cluster.metadata.IndexAbstraction;
import org.elasticsearch.cluster.metadata.IndexMetadata;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.Metadata;
import org.elasticsearch.cluster.metadata.MetadataIndexStateService;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.Priority;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.EsExecutors;
import org.elasticsearch.core.SuppressForbidden;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.engine.frozen.FrozenEngine;
import org.elasticsearch.injection.guice.Inject;
import org.elasticsearch.protocol.xpack.frozen.FreezeRequest;
import org.elasticsearch.protocol.xpack.frozen.FreezeResponse;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import org.elasticsearch.xpack.core.frozen.action.FreezeIndexAction;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.SortedMap;
public final class TransportFreezeIndexAction extends TransportMasterNodeAction<FreezeRequest, FreezeResponse> {
private static final Logger logger = LogManager.getLogger(TransportFreezeIndexAction.class);
private final DestructiveOperations destructiveOperations;
private final IndexNameExpressionResolver indexNameExpressionResolver;
private final MetadataIndexStateService indexStateService;
@Inject
public TransportFreezeIndexAction(
MetadataIndexStateService indexStateService,
TransportService transportService,
ClusterService clusterService,
ThreadPool threadPool,
ActionFilters actionFilters,
IndexNameExpressionResolver indexNameExpressionResolver,
DestructiveOperations destructiveOperations
) {
super(
FreezeIndexAction.NAME,
transportService,
clusterService,
threadPool,
actionFilters,
FreezeRequest::new,
FreezeResponse::new,
EsExecutors.DIRECT_EXECUTOR_SERVICE
);
this.indexStateService = indexStateService;
this.indexNameExpressionResolver = indexNameExpressionResolver;
this.destructiveOperations = destructiveOperations;
}
@Override
protected void doExecute(Task task, FreezeRequest request, ActionListener<FreezeResponse> listener) {
destructiveOperations.failDestructive(request.indices());
super.doExecute(task, request, listener);
}
private Index[] resolveIndices(FreezeRequest request, ClusterState state) {
List<Index> indices = new ArrayList<>();
for (Index index : indexNameExpressionResolver.concreteIndices(state, request)) {
IndexMetadata metadata = state.metadata().index(index);
Settings settings = metadata.getSettings();
// only unfreeze if we are frozen and only freeze if we are not frozen already.
// this prevents all indices that are already frozen that match a pattern to
// go through the cycles again.
if ((request.freeze() && FrozenEngine.INDEX_FROZEN.get(settings) == false)
|| (request.freeze() == false && FrozenEngine.INDEX_FROZEN.get(settings))) {
indices.add(index);
}
}
if (indices.isEmpty() && request.indicesOptions().allowNoIndices() == false) {
throw new ResourceNotFoundException("no index found to " + (request.freeze() ? "freeze" : "unfreeze"));
}
return indices.toArray(Index.EMPTY_ARRAY);
}
@Override
protected void masterOperation(Task task, FreezeRequest request, ClusterState state, ActionListener<FreezeResponse> listener) {
final Index[] concreteIndices = resolveIndices(request, state);
if (concreteIndices.length == 0) {
listener.onResponse(new FreezeResponse(true, true));
return;
}
final CloseIndexClusterStateUpdateRequest closeRequest = new CloseIndexClusterStateUpdateRequest(
request.masterNodeTimeout(),
request.ackTimeout(),
task.getId(),
ActiveShardCount.DEFAULT,
concreteIndices
);
indexStateService.closeIndices(closeRequest, new ActionListener<>() {
@Override
public void onResponse(final CloseIndexResponse response) {
if (response.isAcknowledged()) {
toggleFrozenSettings(concreteIndices, request, listener);
} else {
// TODO improve FreezeResponse so that it also reports failures from the close index API
listener.onResponse(new FreezeResponse(false, false));
}
}
@Override
public void onFailure(final Exception t) {
logger.debug(() -> "failed to close indices [" + Arrays.toString(concreteIndices) + "]", t);
listener.onFailure(t);
}
});
}
private void toggleFrozenSettings(
final Index[] concreteIndices,
final FreezeRequest request,
final ActionListener<FreezeResponse> listener
) {
submitUnbatchedTask(
"toggle-frozen-settings",
new AckedClusterStateUpdateTask(Priority.URGENT, request, listener.delegateFailure((delegate, acknowledgedResponse) -> {
OpenIndexClusterStateUpdateRequest updateRequest = new OpenIndexClusterStateUpdateRequest(
request.masterNodeTimeout(),
request.ackTimeout(),
request.waitForActiveShards(),
concreteIndices
);
indexStateService.openIndices(
updateRequest,
delegate.safeMap(
openIndexClusterStateUpdateResponse -> new FreezeResponse(
openIndexClusterStateUpdateResponse.isAcknowledged(),
openIndexClusterStateUpdateResponse.isShardsAcknowledged()
)
)
);
})) {
@Override
public ClusterState execute(ClusterState currentState) {
List<String> writeIndices = new ArrayList<>();
SortedMap<String, IndexAbstraction> lookup = currentState.metadata().getIndicesLookup();
for (Index index : concreteIndices) {
IndexAbstraction ia = lookup.get(index.getName());
if (ia != null && ia.getParentDataStream() != null && ia.getParentDataStream().getWriteIndex().equals(index)) {
writeIndices.add(index.getName());
}
}
if (writeIndices.size() > 0) {
throw new IllegalArgumentException(
"cannot freeze the following data stream write indices ["
+ Strings.collectionToCommaDelimitedString(writeIndices)
+ "]"
);
}
final Metadata.Builder builder = Metadata.builder(currentState.metadata());
ClusterBlocks.Builder blocks = ClusterBlocks.builder().blocks(currentState.blocks());
for (Index index : concreteIndices) {
final IndexMetadata indexMetadata = currentState.metadata().getIndexSafe(index);
if (indexMetadata.getState() != IndexMetadata.State.CLOSE) {
throw new IllegalStateException("index [" + index.getName() + "] is not closed");
}
final Settings.Builder settingsBuilder = Settings.builder().put(indexMetadata.getSettings());
if (request.freeze()) {
settingsBuilder.put(FrozenEngine.INDEX_FROZEN.getKey(), true);
settingsBuilder.put(IndexSettings.INDEX_SEARCH_THROTTLED.getKey(), true);
settingsBuilder.put("index.blocks.write", true);
blocks.addIndexBlock(index.getName(), IndexMetadata.INDEX_WRITE_BLOCK);
} else {
settingsBuilder.remove(FrozenEngine.INDEX_FROZEN.getKey());
settingsBuilder.remove(IndexSettings.INDEX_SEARCH_THROTTLED.getKey());
if (indexMetadata.isSearchableSnapshot() == false) {
settingsBuilder.remove("index.blocks.write");
blocks.removeIndexBlock(index.getName(), IndexMetadata.INDEX_WRITE_BLOCK);
}
}
builder.put(
IndexMetadata.builder(indexMetadata)
.settingsVersion(indexMetadata.getSettingsVersion() + 1)
.settings(settingsBuilder)
.build(),
true
);
}
return ClusterState.builder(currentState).blocks(blocks).metadata(builder).build();
}
}
);
}
@Override
protected ClusterBlockException checkBlock(FreezeRequest request, ClusterState state) {
return state.blocks()
.indicesBlockedException(ClusterBlockLevel.METADATA_WRITE, indexNameExpressionResolver.concreteIndexNames(state, request));
}
@SuppressForbidden(reason = "legacy usage of unbatched task") // TODO add support for batching here
private void submitUnbatchedTask(@SuppressWarnings("SameParameterValue") String source, ClusterStateUpdateTask task) {
clusterService.submitUnbatchedStateUpdateTask(source, task);
}
}

View file

@ -518,7 +518,6 @@ public class Constants {
"indices:admin/flush",
"indices:admin/flush[s]",
"indices:admin/forcemerge",
"indices:admin/freeze",
"indices:admin/get",
"indices:admin/analyze_disk_usage",
"indices:admin/ilm/explain",

View file

@ -1,6 +1,5 @@
dependencies {
javaRestTestImplementation project(path: xpackModule('ql:test-fixtures'))
clusterPlugins project(':x-pack:qa:freeze-plugin')
}
tasks.named("check").configure {dependsOn("javaRestTest") } // run these tests as part of the "check" task

View file

@ -43,7 +43,6 @@ public class SqlTestClusterWithRemote implements TestRule {
.setting("xpack.license.self_generated.type", "trial")
.setting("xpack.security.autoconfiguration.enabled", "false")
.user(USER_NAME, PASSWORD)
.plugin("freeze-plugin")
.build();
}
@ -58,7 +57,6 @@ public class SqlTestClusterWithRemote implements TestRule {
.setting("xpack.license.self_generated.type", "trial")
.setting("xpack.security.autoconfiguration.enabled", "false")
.user(USER_NAME, PASSWORD)
.plugin("freeze-plugin")
.build();
}

View file

@ -7,5 +7,4 @@ description = 'Run a subset of SQL tests against multiple nodes'
*/
dependencies {
clusterPlugins project(':x-pack:qa:freeze-plugin')
}

View file

@ -20,7 +20,6 @@ public class SqlTestCluster {
.setting("xpack.watcher.enabled", "false")
.setting("xpack.security.enabled", "false")
.setting("xpack.license.self_generated.type", "trial")
.plugin("freeze-plugin")
.build();
}
}

View file

@ -46,7 +46,6 @@ subprojects {
user username: "user1", password: 'x-pack-test-password', role: "user1"
user username: "user2", password: 'x-pack-test-password', role: "user2"
user username: "manage_user", password: 'x-pack-test-password', role: "manage_user"
plugin ':x-pack:qa:freeze-plugin'
}
File testArtifactsDir = project.file("$buildDir/testArtifacts")

View file

@ -2,5 +2,4 @@
apply plugin: 'elasticsearch.internal-test-artifact'
dependencies {
clusterPlugins project(':x-pack:qa:freeze-plugin')
}

View file

@ -12,7 +12,7 @@ import org.junit.ClassRule;
public class CliErrorsIT extends ErrorsTestCase {
@ClassRule
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster(false);
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster();
@Override
protected String getTestRestCluster() {

View file

@ -17,7 +17,7 @@ import static org.hamcrest.Matchers.startsWith;
public class CliExplainIT extends CliIntegrationTestCase {
@ClassRule
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster(false);
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster();
@Override
protected String getTestRestCluster() {

View file

@ -12,7 +12,7 @@ import org.junit.ClassRule;
public class CliFetchSizeIT extends FetchSizeTestCase {
@ClassRule
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster(false);
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster();
@Override
protected String getTestRestCluster() {

View file

@ -12,7 +12,7 @@ import org.junit.ClassRule;
public class CliLenientIT extends LenientTestCase {
@ClassRule
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster(false);
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster();
@Override
protected String getTestRestCluster() {

View file

@ -12,7 +12,7 @@ import org.junit.ClassRule;
public class CliPartialResultsIT extends PartialResultsTestCase {
@ClassRule
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster(false);
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster();
@Override
protected String getTestRestCluster() {

View file

@ -12,7 +12,7 @@ import org.junit.ClassRule;
public class CliSelectIT extends SelectTestCase {
@ClassRule
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster(false);
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster();
@Override
protected String getTestRestCluster() {

View file

@ -12,7 +12,7 @@ import org.junit.ClassRule;
public class CliShowIT extends ShowTestCase {
@ClassRule
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster(false);
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster();
@Override
protected String getTestRestCluster() {

View file

@ -52,7 +52,7 @@ import static org.hamcrest.collection.IsEmptyCollection.empty;
*/
public class ConsistentFunctionArgHandlingIT extends JdbcIntegrationTestCase {
@ClassRule
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster(false);
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster();
@Override
protected String getTestRestCluster() {

View file

@ -13,7 +13,7 @@ import org.junit.ClassRule;
public class CustomDateFormatIT extends CustomDateFormatTestCase {
@ClassRule
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster(false);
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster();
@Override
protected String getTestRestCluster() {

View file

@ -13,7 +13,7 @@ import org.junit.ClassRule;
public class FieldExtractorIT extends FieldExtractorTestCase {
@ClassRule
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster(false);
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster();
@Override
protected String getTestRestCluster() {

View file

@ -23,11 +23,11 @@ import static org.elasticsearch.xpack.ql.CsvSpecReader.specParser;
public class GeoJdbcCsvSpecIT extends GeoCsvSpecTestCase {
@ClassRule
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster(false);
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster();
@Override
protected void loadDataset(RestClient client) throws Exception {
DataLoader.loadDatasetIntoEs(client, false);
DataLoader.loadDatasetIntoEs(client);
}
@Override

View file

@ -15,11 +15,11 @@ import org.junit.ClassRule;
public class GeoJdbcSqlSpecIT extends GeoSqlSpecTestCase {
@ClassRule
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster(false);
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster();
@Override
protected void loadDataset(RestClient client) throws Exception {
DataLoader.loadDatasetIntoEs(client, false);
DataLoader.loadDatasetIntoEs(client);
}
@Override

View file

@ -22,11 +22,11 @@ import static org.elasticsearch.xpack.ql.CsvSpecReader.specParser;
public class JdbcCsvSpecIT extends CsvSpecTestCase {
@ClassRule
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster(false);
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster();
@Override
protected void loadDataset(RestClient client) throws Exception {
DataLoader.loadDatasetIntoEs(client, false);
DataLoader.loadDatasetIntoEs(client);
}
@Override

View file

@ -12,7 +12,7 @@ import org.junit.ClassRule;
public class JdbcDatabaseMetaDataIT extends DatabaseMetaDataTestCase {
@ClassRule
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster(false);
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster();
@Override
protected String getTestRestCluster() {

View file

@ -42,7 +42,7 @@ import static org.elasticsearch.xpack.sql.qa.jdbc.CsvTestUtils.executeCsvQuery;
*/
public class JdbcDocCsvSpecIT extends SpecBaseIntegrationTestCase {
@ClassRule
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster(false);
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster();
@Override
protected String getTestRestCluster() {
@ -58,7 +58,7 @@ public class JdbcDocCsvSpecIT extends SpecBaseIntegrationTestCase {
@Override
protected void loadDataset(RestClient client) throws Exception {
DataLoader.loadDocsDatasetIntoEs(client, false);
DataLoader.loadDocsDatasetIntoEs(client);
}
@ParametersFactory(shuffle = false, argumentFormatting = SqlSpecTestCase.PARAM_FORMATTING)

View file

@ -1,80 +0,0 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License
* 2.0; you may not use this file except in compliance with the Elastic License
* 2.0.
*/
package org.elasticsearch.xpack.sql.qa.single_node;
import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
import org.apache.logging.log4j.Logger;
import org.elasticsearch.client.RestClient;
import org.elasticsearch.test.cluster.ElasticsearchCluster;
import org.elasticsearch.xpack.sql.qa.jdbc.DataLoader;
import org.elasticsearch.xpack.sql.qa.jdbc.JdbcAssert;
import org.elasticsearch.xpack.sql.qa.jdbc.SpecBaseIntegrationTestCase;
import org.elasticsearch.xpack.sql.qa.jdbc.SqlSpecTestCase;
import org.junit.ClassRule;
import java.sql.Connection;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.util.List;
import static org.elasticsearch.xpack.ql.CsvSpecReader.CsvTestCase;
import static org.elasticsearch.xpack.ql.CsvSpecReader.specParser;
import static org.elasticsearch.xpack.ql.SpecReader.Parser;
import static org.elasticsearch.xpack.sql.qa.jdbc.CsvTestUtils.csvConnection;
import static org.elasticsearch.xpack.sql.qa.jdbc.CsvTestUtils.executeCsvQuery;
public class JdbcDocFrozenCsvSpecIT extends SpecBaseIntegrationTestCase {
@ClassRule
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster(true);
@Override
protected String getTestRestCluster() {
return cluster.getHttpAddresses();
}
private final CsvTestCase testCase;
@Override
protected String indexName() {
return "library";
}
@Override
protected void loadDataset(RestClient client) throws Exception {
DataLoader.loadDocsDatasetIntoEs(client, true);
}
@ParametersFactory(shuffle = false, argumentFormatting = SqlSpecTestCase.PARAM_FORMATTING)
public static List<Object[]> readScriptSpec() throws Exception {
Parser parser = specParser();
return readScriptSpec("/docs/docs-frozen.csv-spec", parser);
}
public JdbcDocFrozenCsvSpecIT(String fileName, String groupName, String testName, Integer lineNumber, CsvTestCase testCase) {
super(fileName, groupName, testName, lineNumber);
this.testCase = testCase;
}
@Override
protected void assertResults(ResultSet expected, ResultSet elastic) throws SQLException {
Logger log = logEsResultSet() ? logger : null;
JdbcAssert.assertResultSets(expected, elastic, log, true, true);
}
@Override
protected final void doTest() throws Throwable {
try (Connection csv = csvConnection(testCase); Connection es = esJdbc()) {
// pass the testName as table for debugging purposes (in case the underlying reader is missing)
ResultSet expected = executeCsvQuery(csv, testName);
ResultSet elasticResults = executeJdbcQuery(es, testCase.query);
assertResults(expected, elasticResults);
}
}
}

View file

@ -1,51 +0,0 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License
* 2.0; you may not use this file except in compliance with the Elastic License
* 2.0.
*/
package org.elasticsearch.xpack.sql.qa.single_node;
import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
import org.elasticsearch.test.cluster.ElasticsearchCluster;
import org.elasticsearch.xpack.sql.qa.jdbc.CsvSpecTestCase;
import org.junit.ClassRule;
import java.util.List;
import java.util.Properties;
import java.util.concurrent.TimeUnit;
import static org.elasticsearch.xpack.ql.CsvSpecReader.CsvTestCase;
import static org.elasticsearch.xpack.ql.CsvSpecReader.specParser;
public class JdbcFrozenCsvSpecIT extends CsvSpecTestCase {
@ClassRule
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster(true);
@Override
protected String getTestRestCluster() {
return cluster.getHttpAddresses();
}
@ParametersFactory(argumentFormatting = PARAM_FORMATTING)
public static List<Object[]> readScriptSpec() throws Exception {
return readScriptSpec("/slow/frozen.csv-spec", specParser());
}
@Override
protected Properties connectionProperties() {
Properties props = new Properties(super.connectionProperties());
String timeout = String.valueOf(TimeUnit.MINUTES.toMillis(5));
props.setProperty("connect.timeout", timeout);
props.setProperty("network.timeout", timeout);
props.setProperty("query.timeout", timeout);
props.setProperty("page.timeout", timeout);
return props;
}
public JdbcFrozenCsvSpecIT(String fileName, String groupName, String testName, Integer lineNumber, CsvTestCase testCase) {
super(fileName, groupName, testName, lineNumber, testCase);
}
}

View file

@ -25,7 +25,7 @@ import static org.hamcrest.Matchers.greaterThanOrEqualTo;
public class JdbcShardFailureIT extends JdbcIntegrationTestCase {
@ClassRule
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster(false);
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster();
private String nodeAddresses;

View file

@ -12,7 +12,7 @@ import org.junit.ClassRule;
public class JdbcShowTablesIT extends ShowTablesTestCase {
@ClassRule
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster(false);
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster();
@Override
protected String getTestRestCluster() {

View file

@ -14,11 +14,11 @@ import org.junit.ClassRule;
public class JdbcSqlSpecIT extends SqlSpecTestCase {
@ClassRule
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster(false);
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster();
@Override
protected void loadDataset(RestClient client) throws Exception {
DataLoader.loadDatasetIntoEs(client, false);
DataLoader.loadDatasetIntoEs(client);
}
@Override

View file

@ -20,7 +20,7 @@ import static org.elasticsearch.xpack.sql.qa.rest.RestSqlTestCase.SQL_QUERY_REST
public class RestSqlDeprecationIT extends BaseRestSqlTestCase {
@ClassRule
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster(false);
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster();
@Override
protected String getTestRestCluster() {

View file

@ -24,7 +24,7 @@ import static org.hamcrest.Matchers.containsString;
*/
public class RestSqlIT extends RestSqlTestCase {
@ClassRule
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster(false);
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster();
@Override
protected String getTestRestCluster() {

View file

@ -13,7 +13,7 @@ import org.junit.ClassRule;
public class RestSqlPaginationIT extends RestSqlPaginationTestCase {
@ClassRule
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster(false);
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster();
@Override
protected String getTestRestCluster() {

View file

@ -13,7 +13,7 @@ import org.junit.ClassRule;
public class RestSqlUsageIT extends RestSqlUsageTestCase {
@ClassRule
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster(false);
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster();
@Override
protected String getTestRestCluster() {

View file

@ -13,7 +13,7 @@ import org.junit.ClassRule;
public class SqlProtocolIT extends SqlProtocolTestCase {
@ClassRule
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster(false);
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster();
@Override
protected String getTestRestCluster() {

View file

@ -13,7 +13,7 @@ import org.elasticsearch.test.cluster.local.distribution.DistributionType;
public class SqlTestCluster {
public static String CLUSTER_NAME = "javaRestTest";
public static ElasticsearchCluster getCluster(boolean enableFreezing) {
public static ElasticsearchCluster getCluster() {
var settings = ElasticsearchCluster.local()
.distribution(DistributionType.DEFAULT)
.name(CLUSTER_NAME)
@ -22,10 +22,6 @@ public class SqlTestCluster {
.setting("xpack.security.enabled", "false")
.setting("xpack.license.self_generated.type", "trial");
if (enableFreezing) {
settings = settings.plugin("freeze-plugin");
}
return settings.build();
}
}

View file

@ -13,7 +13,7 @@ import org.junit.ClassRule;
public class SysColumnsIT extends SysColumnsTestCase {
@ClassRule
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster(false);
public static final ElasticsearchCluster cluster = SqlTestCluster.getCluster();
@Override
protected String getTestRestCluster() {

View file

@ -32,14 +32,14 @@ public class DataLoader {
public static void main(String[] args) throws Exception {
try (RestClient client = RestClient.builder(new HttpHost("localhost", 9200)).build()) {
loadEmpDatasetIntoEs(client, true);
loadDocsDatasetIntoEs(client, true);
loadEmpDatasetIntoEs(client);
loadDocsDatasetIntoEs(client);
LogManager.getLogger(DataLoader.class).info("Data loaded");
}
}
public static void loadDatasetIntoEs(RestClient client, boolean includeFrozenIndices) throws Exception {
loadEmpDatasetIntoEs(client, includeFrozenIndices);
public static void loadDatasetIntoEs(RestClient client) throws Exception {
loadEmpDatasetIntoEs(client);
}
public static void createEmptyIndex(RestClient client, String index) throws Exception {
@ -62,7 +62,7 @@ public class DataLoader {
client.performRequest(request);
}
private static void loadEmpDatasetIntoEs(RestClient client, boolean includeFrozenIndices) throws Exception {
private static void loadEmpDatasetIntoEs(RestClient client) throws Exception {
loadEmpDatasetIntoEs(client, "test_emp", "employees");
loadEmpDatasetWithExtraIntoEs(client, "test_emp_copy", "employees");
loadAppsDatasetIntoEs(client, "apps", "apps");
@ -71,10 +71,6 @@ public class DataLoader {
loadLogUnsignedLongIntoEs(client, "logs_unsigned_long", "logs_unsigned_long");
makeAlias(client, "test_alias", "test_emp", "test_emp_copy");
makeAlias(client, "test_alias_emp", "test_emp", "test_emp_copy");
if (includeFrozenIndices) {
loadEmpDatasetIntoEs(client, "frozen_emp", "employees");
freeze(client, "frozen_emp");
}
loadNoColsDatasetIntoEs(client, "empty_mapping");
}
@ -91,14 +87,10 @@ public class DataLoader {
client.performRequest(request);
}
public static void loadDocsDatasetIntoEs(RestClient client, boolean includeFrozenIndices) throws Exception {
public static void loadDocsDatasetIntoEs(RestClient client) throws Exception {
loadEmpDatasetIntoEs(client, "emp", "employees");
loadLibDatasetIntoEs(client, "library");
makeAlias(client, "employees", "emp");
if (includeFrozenIndices) {
loadLibDatasetIntoEs(client, "archive");
freeze(client, "archive");
}
}
public static void createString(String name, XContentBuilder builder) throws Exception {

View file

@ -60,7 +60,7 @@ public abstract class SpecBaseIntegrationTestCase extends JdbcIntegrationTestCas
}
protected void loadDataset(RestClient client) throws Exception {
DataLoader.loadDatasetIntoEs(client, true);
DataLoader.loadDatasetIntoEs(client);
}
@Override

View file

@ -46,21 +46,6 @@ my_remote_cluster|test_emp |TABLE |INDEX
my_remote_cluster|test_emp_copy |TABLE |INDEX
;
showTablesWithFrozen
SHOW TABLES CATALOG 'my_remote_cluster' INCLUDE FROZEN;
catalog | name | type | kind
-----------------+-------------------+---------------+---------------
my_remote_cluster|apps |TABLE |INDEX
my_remote_cluster|empty_mapping |TABLE |INDEX
my_remote_cluster|frozen_emp |TABLE |INDEX
my_remote_cluster|logs |TABLE |INDEX
my_remote_cluster|logs_nanos |TABLE |INDEX
my_remote_cluster|logs_unsigned_long |TABLE |INDEX
my_remote_cluster|test_emp |TABLE |INDEX
my_remote_cluster|test_emp_copy |TABLE |INDEX
;
showTablesSimpleLike
SHOW TABLES CATALOG 'my_remote_cluster' LIKE 'test_emp';

View file

@ -1,65 +0,0 @@
// To mute tests follow example in file: example.csv-spec
//
// Frozen indices tests
//
showTables
SHOW TABLES INCLUDE FROZEN;
catalog | name | type | kind
javaRestTest |apps |TABLE |INDEX
javaRestTest |empty_mapping |TABLE |INDEX
javaRestTest |frozen_emp |TABLE |FROZEN INDEX
javaRestTest |logs |TABLE |INDEX
javaRestTest |logs_nanos |TABLE |INDEX
javaRestTest |logs_unsigned_long |TABLE |INDEX
javaRestTest |test_alias |VIEW |ALIAS
javaRestTest |test_alias_emp |VIEW |ALIAS
javaRestTest |test_emp |TABLE |INDEX
javaRestTest |test_emp_copy |TABLE |INDEX
;
columnFromFrozen
SELECT gender FROM FROZEN frozen_emp ORDER BY gender LIMIT 5;
gender:s
F
F
F
F
F
;
percentileFrozen
SELECT gender, PERCENTILE(emp_no, 92.45) p1 FROM FROZEN frozen_emp GROUP BY gender;
gender:s | p1:d
null |10018.745
F |10096.336
M |10091.393
;
countFromFrozen
SELECT gender, COUNT(*) AS c FROM FROZEN frozen_emp GROUP BY gender;
gender:s | c:l
null |10
F |33
M |57
;
sum
SELECT SUM(salary) FROM FROZEN frozen_emp;
SUM(salary):l
---------------
4824855
;
kurtosisAndSkewnessNoGroup
SELECT KURTOSIS(emp_no) k, SKEWNESS(salary) s FROM FROZEN frozen_emp;
k:d | s:d
1.7997599759975997 | 0.2707722118423227
;

View file

@ -1,24 +0,0 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License
* 2.0; you may not use this file except in compliance with the Elastic License
* 2.0.
*/
apply plugin: 'elasticsearch.base-internal-es-plugin'
esplugin {
name = 'freeze-plugin'
description = 'Provides freeze-index endpoint for testing purposes only'
classname = 'org.elasticsearch.plugin.freeze.FreezeIndexPlugin'
extendedPlugins = ['x-pack-core']
}
dependencies {
compileOnly project(":server")
compileOnly project(path: xpackModule('core'))
}
tasks.named('test').configure { enabled = false }

View file

@ -1,93 +0,0 @@
/*
* Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
* or more contributor license agreements. Licensed under the Elastic License
* 2.0; you may not use this file except in compliance with the Elastic License
* 2.0.
*/
package org.elasticsearch.plugin.freeze;
import org.elasticsearch.action.support.ActiveShardCount;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.client.internal.node.NodeClient;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.settings.ClusterSettings;
import org.elasticsearch.common.settings.IndexScopedSettings;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.SettingsFilter;
import org.elasticsearch.features.NodeFeature;
import org.elasticsearch.plugins.ActionPlugin;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.protocol.xpack.frozen.FreezeRequest;
import org.elasticsearch.rest.BaseRestHandler;
import org.elasticsearch.rest.RestController;
import org.elasticsearch.rest.RestHandler;
import org.elasticsearch.rest.RestRequest;
import org.elasticsearch.rest.action.RestToXContentListener;
import org.elasticsearch.xpack.core.frozen.action.FreezeIndexAction;
import java.util.List;
import java.util.function.Predicate;
import java.util.function.Supplier;
import static org.elasticsearch.rest.RestRequest.Method.POST;
import static org.elasticsearch.rest.RestUtils.getAckTimeout;
import static org.elasticsearch.rest.RestUtils.getMasterNodeTimeout;
/**
* Restores the REST endpoint for freezing indices so that the JDBC tests can still freeze indices
* for testing purposes until frozen indices are no longer supported.
*/
public class FreezeIndexPlugin extends Plugin implements ActionPlugin {
@Override
public List<RestHandler> getRestHandlers(
Settings settings,
NamedWriteableRegistry namedWriteableRegistry,
RestController restController,
ClusterSettings clusterSettings,
IndexScopedSettings indexScopedSettings,
SettingsFilter settingsFilter,
IndexNameExpressionResolver indexNameExpressionResolver,
Supplier<DiscoveryNodes> nodesInCluster,
Predicate<NodeFeature> clusterSupportsFeature
) {
return List.of(new FreezeIndexRestEndpoint());
}
/**
* Used by the {@link FreezeIndexPlugin} above.
*/
static class FreezeIndexRestEndpoint extends BaseRestHandler {
@Override
public String getName() {
return "freeze-for-testing-only";
}
@Override
public List<Route> routes() {
return List.of(new Route(POST, "/{index}/_freeze"));
}
@Override
protected RestChannelConsumer prepareRequest(RestRequest request, NodeClient client) {
boolean freeze = request.path().endsWith("/_freeze");
FreezeRequest freezeRequest = new FreezeRequest(
getMasterNodeTimeout(request),
getAckTimeout(request),
Strings.splitStringByCommaToArray(request.param("index"))
);
freezeRequest.indicesOptions(IndicesOptions.fromRequest(request, freezeRequest.indicesOptions()));
String waitForActiveShards = request.param("wait_for_active_shards");
if (waitForActiveShards != null) {
freezeRequest.waitForActiveShards(ActiveShardCount.parseString(waitForActiveShards));
}
freezeRequest.setFreeze(freeze);
return channel -> client.execute(FreezeIndexAction.INSTANCE, freezeRequest, new RestToXContentListener<>(channel));
}
}
}