Browse Source

Moved some other "poison pills" to UpdateForV9 annotation (#102287)

Follow up of #101767
Move from exception and assertions using Version.CURRENT to @UpdateForV9 annotation
Lorenzo Dematté 1 year ago
parent
commit
bd4f29688d

+ 2 - 10
modules/ingest-attachment/src/main/java/org/elasticsearch/ingest/attachment/AttachmentProcessor.java

@@ -14,10 +14,10 @@ import org.apache.tika.metadata.Metadata;
 import org.apache.tika.metadata.Office;
 import org.apache.tika.metadata.TikaCoreProperties;
 import org.elasticsearch.ElasticsearchParseException;
-import org.elasticsearch.Version;
 import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.logging.DeprecationCategory;
 import org.elasticsearch.common.logging.DeprecationLogger;
+import org.elasticsearch.core.UpdateForV9;
 import org.elasticsearch.ingest.AbstractProcessor;
 import org.elasticsearch.ingest.IngestDocument;
 import org.elasticsearch.ingest.Processor;
@@ -226,15 +226,6 @@ public final class AttachmentProcessor extends AbstractProcessor {
 
         static final Set<Property> DEFAULT_PROPERTIES = EnumSet.allOf(Property.class);
 
-        static {
-            if (Version.CURRENT.major >= 9) {
-                throw new IllegalStateException(
-                    "[poison pill] update the [remove_binary] default to be 'true' assuming "
-                        + "enough time has passed. Deprecated in September 2022."
-                );
-            }
-        }
-
         @Override
         public AttachmentProcessor create(
             Map<String, Processor.Factory> registry,
@@ -249,6 +240,7 @@ public final class AttachmentProcessor extends AbstractProcessor {
             int indexedChars = readIntProperty(TYPE, processorTag, config, "indexed_chars", NUMBER_OF_CHARS_INDEXED);
             boolean ignoreMissing = readBooleanProperty(TYPE, processorTag, config, "ignore_missing", false);
             String indexedCharsField = readOptionalStringProperty(TYPE, processorTag, config, "indexed_chars_field");
+            @UpdateForV9 // update the [remove_binary] default to be 'true' assuming enough time has passed. Deprecated in September 2022.
             Boolean removeBinary = readOptionalBooleanProperty(TYPE, processorTag, config, "remove_binary");
             if (removeBinary == null) {
                 DEPRECATION_LOGGER.warn(

+ 2 - 2
server/src/main/java/org/elasticsearch/action/admin/cluster/migration/TransportGetFeatureUpgradeStatusAction.java

@@ -19,6 +19,7 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
 import org.elasticsearch.cluster.service.ClusterService;
 import org.elasticsearch.common.inject.Inject;
 import org.elasticsearch.common.util.concurrent.EsExecutors;
+import org.elasticsearch.core.UpdateForV9;
 import org.elasticsearch.index.IndexVersion;
 import org.elasticsearch.index.IndexVersions;
 import org.elasticsearch.indices.SystemIndices;
@@ -61,6 +62,7 @@ public class TransportGetFeatureUpgradeStatusAction extends TransportMasterNodeA
     PersistentTasksService persistentTasksService;
 
     @Inject
+    @UpdateForV9 // Once we begin working on 9.x, we need to update our migration classes
     public TransportGetFeatureUpgradeStatusAction(
         TransportService transportService,
         ThreadPool threadPool,
@@ -82,8 +84,6 @@ public class TransportGetFeatureUpgradeStatusAction extends TransportMasterNodeA
             EsExecutors.DIRECT_EXECUTOR_SERVICE
         );
 
-        assert Version.CURRENT.major == 8 : "Once we begin working on 9.x, we need to update our migration classes";
-
         this.systemIndices = systemIndices;
         this.persistentTasksService = persistentTasksService;
     }

+ 2 - 0
server/src/main/java/org/elasticsearch/transport/TransportInfo.java

@@ -17,6 +17,7 @@ import org.elasticsearch.common.transport.BoundTransportAddress;
 import org.elasticsearch.common.transport.TransportAddress;
 import org.elasticsearch.common.util.Maps;
 import org.elasticsearch.core.Nullable;
+import org.elasticsearch.core.UpdateForV9;
 import org.elasticsearch.node.ReportingService;
 import org.elasticsearch.xcontent.XContentBuilder;
 
@@ -30,6 +31,7 @@ public class TransportInfo implements ReportingService.Info {
     private static final DeprecationLogger deprecationLogger = DeprecationLogger.getLogger(TransportInfo.class);
 
     /** Whether to add hostname to publish host field when serializing. */
+    @UpdateForV9 // Remove es.transport.cname_in_publish_address property from TransportInfo in 9.0.0
     private static final boolean CNAME_IN_PUBLISH_ADDRESS = parseBoolean(
         System.getProperty("es.transport.cname_in_publish_address"),
         false

+ 2 - 1
server/src/main/java/org/elasticsearch/transport/TransportService.java

@@ -38,6 +38,7 @@ import org.elasticsearch.core.IOUtils;
 import org.elasticsearch.core.Nullable;
 import org.elasticsearch.core.Releasable;
 import org.elasticsearch.core.TimeValue;
+import org.elasticsearch.core.UpdateForV9;
 import org.elasticsearch.node.NodeClosedException;
 import org.elasticsearch.node.ReportingService;
 import org.elasticsearch.tasks.Task;
@@ -1673,11 +1674,11 @@ public class TransportService extends AbstractLifecycleComponent
 
     static {
         // Ensure that this property, introduced and immediately deprecated in 7.11, is not used in 8.x
+        @UpdateForV9 // we can remove this whole block in v9
         final String PERMIT_HANDSHAKES_FROM_INCOMPATIBLE_BUILDS_KEY = "es.unsafely_permit_handshake_from_incompatible_builds";
         if (System.getProperty(PERMIT_HANDSHAKES_FROM_INCOMPATIBLE_BUILDS_KEY) != null) {
             throw new IllegalArgumentException("system property [" + PERMIT_HANDSHAKES_FROM_INCOMPATIBLE_BUILDS_KEY + "] must not be set");
         }
-        assert Version.CURRENT.major == Version.V_7_0_0.major + 1; // we can remove this whole block in v9
     }
 
     private record UnregisterChildTransportResponseHandler<T extends TransportResponse>(

+ 8 - 4
server/src/main/java/org/elasticsearch/transport/TransportStats.java

@@ -9,7 +9,6 @@
 package org.elasticsearch.transport;
 
 import org.elasticsearch.TransportVersions;
-import org.elasticsearch.Version;
 import org.elasticsearch.common.collect.Iterators;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
@@ -18,6 +17,7 @@ import org.elasticsearch.common.network.HandlingTimeTracker;
 import org.elasticsearch.common.unit.ByteSizeValue;
 import org.elasticsearch.common.xcontent.ChunkedToXContent;
 import org.elasticsearch.core.TimeValue;
+import org.elasticsearch.core.UpdateForV9;
 import org.elasticsearch.xcontent.ToXContent;
 import org.elasticsearch.xcontent.XContentBuilder;
 
@@ -165,11 +165,14 @@ public class TransportStats implements Writeable, ChunkedToXContent {
         return transportActionStats;
     }
 
+    @UpdateForV9 // Review and simplify the if-else blocks containing this symbol once v9 is released
+    private static final boolean IMPOSSIBLE_IN_V9 = true;
+
     private boolean assertHistogramsConsistent() {
         assert inboundHandlingTimeBucketFrequencies.length == outboundHandlingTimeBucketFrequencies.length;
         if (inboundHandlingTimeBucketFrequencies.length == 0) {
             // Stats came from before v8.1
-            assert Version.CURRENT.major == Version.V_8_0_0.major;
+            assert IMPOSSIBLE_IN_V9;
         } else {
             assert inboundHandlingTimeBucketFrequencies.length == HandlingTimeTracker.BUCKET_COUNT;
         }
@@ -177,6 +180,7 @@ public class TransportStats implements Writeable, ChunkedToXContent {
     }
 
     @Override
+    @UpdateForV9 // review the "if" blocks checking for non-empty once we have
     public Iterator<? extends ToXContent> toXContentChunked(ToXContent.Params outerParams) {
         return Iterators.concat(Iterators.single((builder, params) -> {
             builder.startObject(Fields.TRANSPORT);
@@ -191,13 +195,13 @@ public class TransportStats implements Writeable, ChunkedToXContent {
                 histogramToXContent(builder, outboundHandlingTimeBucketFrequencies, Fields.OUTBOUND_HANDLING_TIME_HISTOGRAM);
             } else {
                 // Stats came from before v8.1
-                assert Version.CURRENT.major == Version.V_7_0_0.major + 1;
+                assert IMPOSSIBLE_IN_V9;
             }
             if (transportActionStats.isEmpty() == false) {
                 builder.startObject(Fields.ACTIONS);
             } else {
                 // Stats came from before v8.8
-                assert Version.CURRENT.major == Version.V_7_0_0.major + 1;
+                assert IMPOSSIBLE_IN_V9;
             }
             return builder;
         }),

+ 2 - 4
server/src/test/java/org/elasticsearch/action/admin/cluster/migration/TransportGetFeatureUpgradeStatusActionTests.java

@@ -8,11 +8,11 @@
 
 package org.elasticsearch.action.admin.cluster.migration;
 
-import org.elasticsearch.Version;
 import org.elasticsearch.cluster.ClusterState;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.cluster.metadata.Metadata;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.core.UpdateForV9;
 import org.elasticsearch.index.IndexVersion;
 import org.elasticsearch.indices.SystemIndexDescriptor;
 import org.elasticsearch.indices.SystemIndexDescriptorUtils;
@@ -84,9 +84,7 @@ public class TransportGetFeatureUpgradeStatusActionTests extends ESTestCase {
             .numberOfReplicas(0)
             .build();
 
-        // Once we start testing 9.x, we should update this test to use a 7.x "version created"
-        assert Version.CURRENT.major < 9;
-
+        @UpdateForV9 // Once we start testing 9.x, we should update this test to use a 7.x "version created"
         IndexMetadata indexMetadata2 = IndexMetadata.builder(".test-index-2")
             .settings(Settings.builder().put("index.version.created", TEST_OLD_VERSION).build())
             .numberOfShards(1)

+ 0 - 5
server/src/test/java/org/elasticsearch/transport/TransportInfoTests.java

@@ -8,7 +8,6 @@
 
 package org.elasticsearch.transport;
 
-import org.elasticsearch.Version;
 import org.elasticsearch.common.network.NetworkAddress;
 import org.elasticsearch.common.transport.BoundTransportAddress;
 import org.elasticsearch.common.transport.TransportAddress;
@@ -33,10 +32,6 @@ public class TransportInfoTests extends ESTestCase {
         return new TransportInfo(boundAddress, profiles, cnameInPublishAddressProperty);
     }
 
-    public void testDoNotForgetToRemoveProperty() {
-        assertTrue("Remove es.transport.cname_in_publish_address property from TransportInfo in 9.0.0", Version.CURRENT.major < 9);
-    }
-
     public void testCorrectlyDisplayPublishedCname() throws Exception {
         InetAddress address = InetAddress.getByName("localhost");
         int port = 9200;

+ 11 - 7
test/framework/src/main/java/org/elasticsearch/test/transport/MockTransportService.java

@@ -11,7 +11,6 @@ package org.elasticsearch.test.transport;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 import org.elasticsearch.TransportVersion;
-import org.elasticsearch.Version;
 import org.elasticsearch.action.ActionListener;
 import org.elasticsearch.cluster.ClusterModule;
 import org.elasticsearch.cluster.node.DiscoveryNode;
@@ -33,6 +32,7 @@ import org.elasticsearch.common.util.concurrent.RunOnce;
 import org.elasticsearch.core.IOUtils;
 import org.elasticsearch.core.Nullable;
 import org.elasticsearch.core.TimeValue;
+import org.elasticsearch.core.UpdateForV9;
 import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
 import org.elasticsearch.node.Node;
 import org.elasticsearch.plugins.Plugin;
@@ -541,12 +541,7 @@ public class MockTransportService extends TransportService {
                 request.writeTo(bStream);
                 final TransportRequest clonedRequest;
                 if (request instanceof BytesTransportRequest) {
-                    // Some request handlers read back a BytesTransportRequest
-                    // into a different class that cannot be re-serialized (i.e. JOIN_VALIDATE_ACTION_NAME),
-                    // in those cases we just copy the raw bytes back to a BytesTransportRequest.
-                    // This is only needed for the BwC for JOIN_VALIDATE_ACTION_NAME and can be removed in the next major
-                    assert Version.CURRENT.major == Version.V_7_17_0.major + 1;
-                    clonedRequest = new BytesTransportRequest(bStream.bytes().streamInput());
+                    clonedRequest = copyRawBytesForBwC(bStream);
                 } else {
                     RequestHandlerRegistry<?> reg = MockTransportService.this.getRequestHandler(action);
                     clonedRequest = reg.newRequest(bStream.bytes().streamInput());
@@ -576,6 +571,15 @@ public class MockTransportService extends TransportService {
                 }
             }
 
+            // Some request handlers read back a BytesTransportRequest
+            // into a different class that cannot be re-serialized (i.e. JOIN_VALIDATE_ACTION_NAME),
+            // in those cases we just copy the raw bytes back to a BytesTransportRequest.
+            // This is only needed for the BwC for JOIN_VALIDATE_ACTION_NAME and can be removed in the next major
+            @UpdateForV9
+            private static TransportRequest copyRawBytesForBwC(BytesStreamOutput bStream) throws IOException {
+                return new BytesTransportRequest(bStream.bytes().streamInput());
+            }
+
             @Override
             public void clearCallback() {
                 synchronized (this) {

+ 2 - 2
x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ilm/Phase.java

@@ -8,13 +8,13 @@ package org.elasticsearch.xpack.core.ilm;
 
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
-import org.elasticsearch.Version;
 import org.elasticsearch.common.Strings;
 import org.elasticsearch.common.io.stream.StreamInput;
 import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.common.io.stream.Writeable;
 import org.elasticsearch.common.util.Maps;
 import org.elasticsearch.core.TimeValue;
+import org.elasticsearch.core.UpdateForV9;
 import org.elasticsearch.xcontent.ConstructingObjectParser;
 import org.elasticsearch.xcontent.ContextParser;
 import org.elasticsearch.xcontent.ObjectParser.ValueType;
@@ -56,7 +56,7 @@ public class Phase implements ToXContentObject, Writeable {
             // when the phase is read from the cluster state during startup (even before negative timevalues were strictly
             // disallowed) so this is a hack to treat negative `min_age`s as 0 to prevent those errors.
             // They will be saved as `0` so this hack can be removed once we no longer have to read cluster states from 7.x.
-            assert Version.CURRENT.major < 9 : "remove this hack now that we don't have to read 7.x cluster states";
+            @UpdateForV9 // remove this hack now that we don't have to read 7.x cluster states
             final String timeValueString = p.text();
             if (timeValueString.startsWith("-")) {
                 logger.warn("phase has negative min_age value of [{}] - this will be treated as a min_age of 0", timeValueString);

+ 2 - 6
x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/allocation/decider/SearchableSnapshotEnableAllocationDecider.java

@@ -7,7 +7,6 @@
 
 package org.elasticsearch.xpack.searchablesnapshots.allocation.decider;
 
-import org.elasticsearch.Version;
 import org.elasticsearch.cluster.metadata.IndexMetadata;
 import org.elasticsearch.cluster.routing.RoutingNode;
 import org.elasticsearch.cluster.routing.ShardRouting;
@@ -18,6 +17,7 @@ import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDeci
 import org.elasticsearch.common.settings.ClusterSettings;
 import org.elasticsearch.common.settings.Setting;
 import org.elasticsearch.common.settings.Settings;
+import org.elasticsearch.core.UpdateForV9;
 
 public class SearchableSnapshotEnableAllocationDecider extends AllocationDecider {
 
@@ -28,6 +28,7 @@ public class SearchableSnapshotEnableAllocationDecider extends AllocationDecider
      * ongoing is determined by cluster.routing.allocation.enable=primaries. Notice that other values for that setting except "all" mean
      * that no searchable snapshots are allocated anyway.
      */
+    @UpdateForV9 // xpack.searchable.snapshot.allocate_on_rolling_restart was only temporary, remove it in the next major
     public static final Setting<Boolean> SEARCHABLE_SNAPSHOTS_ALLOCATE_ON_ROLLING_RESTART = Setting.boolSetting(
         "xpack.searchable.snapshot.allocate_on_rolling_restart",
         false,
@@ -36,11 +37,6 @@ public class SearchableSnapshotEnableAllocationDecider extends AllocationDecider
         Setting.Property.Deprecated
     );
 
-    static {
-        // TODO xpack.searchable.snapshot.allocate_on_rolling_restart was only temporary, remove it in the next major
-        assert Version.CURRENT.major == Version.V_7_17_0.major + 1;
-    }
-
     private volatile EnableAllocationDecider.Allocation enableAllocation;
     private volatile boolean allocateOnRollingRestart;