This is an automated email from the ASF dual-hosted git repository. edimitrova pushed a commit to branch trunk in repository https://gitbox.apache.org/repos/asf/cassandra.git
The following commit(s) were added to refs/heads/trunk by this push: new 6c90950489 Fix tests post JDK-8210522 (rewrite reflection of "modifiers" field) patch by Ekaterina Dimitrova; reviewed by Michael Semb Wever for CASSANDRA-18181 6c90950489 is described below commit 6c90950489a42a38a6ddfa45f79c84a110dffbe7 Author: Ekaterina Dimitrova <ekaterina.dimitr...@datastax.com> AuthorDate: Thu Jan 19 20:30:43 2023 -0500 Fix tests post JDK-8210522 (rewrite reflection of "modifiers" field) patch by Ekaterina Dimitrova; reviewed by Michael Semb Wever for CASSANDRA-18181 --- src/java/org/apache/cassandra/net/Verb.java | 7 +-- .../org/apache/cassandra/transport/Message.java | 3 +- .../apache/cassandra/utils/ReflectionUtils.java | 61 ++++++++++++++++++++++ .../cassandra/transport/MessagePayloadTest.java | 3 +- 4 files changed, 69 insertions(+), 5 deletions(-) diff --git a/src/java/org/apache/cassandra/net/Verb.java b/src/java/org/apache/cassandra/net/Verb.java index d50a187fda..d52f14c7ac 100644 --- a/src/java/org/apache/cassandra/net/Verb.java +++ b/src/java/org/apache/cassandra/net/Verb.java @@ -96,6 +96,7 @@ import org.apache.cassandra.service.paxos.PrepareResponse; import org.apache.cassandra.service.paxos.v1.PrepareVerbHandler; import org.apache.cassandra.service.paxos.v1.ProposeVerbHandler; import org.apache.cassandra.streaming.ReplicationDoneVerbHandler; +import org.apache.cassandra.utils.ReflectionUtils; import org.apache.cassandra.utils.TimeUUID; import org.apache.cassandra.utils.UUIDSerializer; @@ -351,7 +352,7 @@ public enum Verb Supplier<? extends IVerbHandler<?>> original = this.handler; Field field = Verb.class.getDeclaredField("handler"); field.setAccessible(true); - Field modifiers = Field.class.getDeclaredField("modifiers"); + Field modifiers = ReflectionUtils.getModifiersField(); modifiers.setAccessible(true); modifiers.setInt(field, field.getModifiers() & ~Modifier.FINAL); field.set(this, handler); @@ -364,7 +365,7 @@ public enum Verb Supplier<? extends IVersionedAsymmetricSerializer<?, ?>> original = this.serializer; Field field = Verb.class.getDeclaredField("serializer"); field.setAccessible(true); - Field modifiers = Field.class.getDeclaredField("modifiers"); + Field modifiers = ReflectionUtils.getModifiersField(); modifiers.setAccessible(true); modifiers.setInt(field, field.getModifiers() & ~Modifier.FINAL); field.set(this, serializer); @@ -377,7 +378,7 @@ public enum Verb ToLongFunction<TimeUnit> original = this.expiration; Field field = Verb.class.getDeclaredField("expiration"); field.setAccessible(true); - Field modifiers = Field.class.getDeclaredField("modifiers"); + Field modifiers = ReflectionUtils.getModifiersField(); modifiers.setAccessible(true); modifiers.setInt(field, field.getModifiers() & ~Modifier.FINAL); field.set(this, expiration); diff --git a/src/java/org/apache/cassandra/transport/Message.java b/src/java/org/apache/cassandra/transport/Message.java index 2c91a76c3b..82015c1150 100644 --- a/src/java/org/apache/cassandra/transport/Message.java +++ b/src/java/org/apache/cassandra/transport/Message.java @@ -36,6 +36,7 @@ import org.apache.cassandra.service.StorageService; import org.apache.cassandra.tracing.Tracing; import org.apache.cassandra.transport.messages.*; import org.apache.cassandra.service.QueryState; +import org.apache.cassandra.utils.ReflectionUtils; import org.apache.cassandra.utils.TimeUUID; import static org.apache.cassandra.utils.TimeUUID.Generator.nextTimeUUID; @@ -132,7 +133,7 @@ public abstract class Message Codec<?> original = this.codec; Field field = Type.class.getDeclaredField("codec"); field.setAccessible(true); - Field modifiers = Field.class.getDeclaredField("modifiers"); + Field modifiers = ReflectionUtils.getModifiersField(); modifiers.setAccessible(true); modifiers.setInt(field, field.getModifiers() & ~Modifier.FINAL); field.set(this, codec); diff --git a/src/java/org/apache/cassandra/utils/ReflectionUtils.java b/src/java/org/apache/cassandra/utils/ReflectionUtils.java new file mode 100644 index 0000000000..801256d4ee --- /dev/null +++ b/src/java/org/apache/cassandra/utils/ReflectionUtils.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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. + */ + +package org.apache.cassandra.utils; + +import java.lang.reflect.Field; +import java.lang.reflect.Method; + +public class ReflectionUtils +{ + private ReflectionUtils() + { + + } + + public static Field getModifiersField() throws NoSuchFieldException + { + // below code works before Java 12 + try + { + return Field.class.getDeclaredField("modifiers"); + } + catch (NoSuchFieldException e) + { + // this is mitigation for JDK 17 (https://bugs.openjdk.org/browse/JDK-8210522) + try + { + Method getDeclaredFields0 = Class.class.getDeclaredMethod("getDeclaredFields0", boolean.class); + getDeclaredFields0.setAccessible(true); + Field[] fields = (Field[]) getDeclaredFields0.invoke(Field.class, false); + for (Field field : fields) + { + if ("modifiers".equals(field.getName())) + { + return field; + } + } + } + catch (ReflectiveOperationException ex) + { + e.addSuppressed(ex); + } + throw e; + } + } +} \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/transport/MessagePayloadTest.java b/test/unit/org/apache/cassandra/transport/MessagePayloadTest.java index f0b45c3a19..bd656d418b 100644 --- a/test/unit/org/apache/cassandra/transport/MessagePayloadTest.java +++ b/test/unit/org/apache/cassandra/transport/MessagePayloadTest.java @@ -47,6 +47,7 @@ import org.apache.cassandra.transport.messages.PrepareMessage; import org.apache.cassandra.transport.messages.QueryMessage; import org.apache.cassandra.transport.messages.ResultMessage; import org.apache.cassandra.utils.MD5Digest; +import org.apache.cassandra.utils.ReflectionUtils; import static org.apache.cassandra.utils.ByteBufferUtil.bytes; @@ -66,7 +67,7 @@ public class MessagePayloadTest extends CQLTester cqlQueryHandlerField = ClientState.class.getDeclaredField("cqlQueryHandler"); cqlQueryHandlerField.setAccessible(true); - Field modifiersField = Field.class.getDeclaredField("modifiers"); + Field modifiersField = ReflectionUtils.getModifiersField(); modifiersAccessible = modifiersField.isAccessible(); modifiersField.setAccessible(true); modifiersField.setInt(cqlQueryHandlerField, cqlQueryHandlerField.getModifiers() & ~Modifier.FINAL); --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org