This is an automated email from the ASF dual-hosted git repository.
chesnay pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git
The following commit(s) were added to refs/heads/master by this push:
new 10f38e85d8d [FLINK-31972][tests] Remove powermock whitebox usages
10f38e85d8d is described below
commit 10f38e85d8d3ebda54e70b6354aec3e5b9806188
Author: Chesnay Schepler <[email protected]>
AuthorDate: Fri Apr 28 18:09:05 2023 +0200
[FLINK-31972][tests] Remove powermock whitebox usages
---
.../base/source/hybrid/HybridSourceReaderTest.java | 2 +-
.../hybrid/HybridSourceSplitEnumeratorTest.java | 10 ++++----
.../source/enumerator/KafkaEnumeratorTest.java | 7 +++---
.../confluent/CachedSchemaCoderProviderTest.java | 3 ++-
.../flink/formats/avro/AvroOutputFormatTest.java | 5 ++--
.../apache/flink/cep/operator/CEPOperatorTest.java | 6 ++---
.../operators/hash/CompactingHashTable.java | 11 ++++++++-
.../operators/hash/CompactingHashTableTest.java | 27 ++++++++--------------
.../main/java/org/apache/flink/mock/Whitebox.java | 5 ++--
9 files changed, 36 insertions(+), 40 deletions(-)
diff --git
a/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/hybrid/HybridSourceReaderTest.java
b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/hybrid/HybridSourceReaderTest.java
index 02231378a36..071e2147813 100644
---
a/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/hybrid/HybridSourceReaderTest.java
+++
b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/hybrid/HybridSourceReaderTest.java
@@ -314,7 +314,7 @@ public class HybridSourceReaderTest {
private static SourceReader<Integer, MockSourceSplit> currentReader(
HybridSourceReader<?> reader) {
- return (SourceReader) Whitebox.getInternalState(reader,
"currentReader");
+ return Whitebox.getInternalState(reader, "currentReader");
}
private static void assertAndClearSourceReaderFinishedEvent(
diff --git
a/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/hybrid/HybridSourceSplitEnumeratorTest.java
b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/hybrid/HybridSourceSplitEnumeratorTest.java
index e4f268dedef..fcde32811f4 100644
---
a/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/hybrid/HybridSourceSplitEnumeratorTest.java
+++
b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/hybrid/HybridSourceSplitEnumeratorTest.java
@@ -128,8 +128,7 @@ public class HybridSourceSplitEnumeratorTest {
Whitebox.setInternalState(enumerator, "currentEnumerator",
underlyingEnumeratorWrapper);
List<MockSourceSplit> mockSourceSplits =
- (List<MockSourceSplit>)
-
Whitebox.getInternalState(underlyingEnumeratorWrapper.enumerator, "splits");
+
Whitebox.getInternalState(underlyingEnumeratorWrapper.enumerator, "splits");
assertThat(mockSourceSplits).isEmpty();
// simulate reader reset to before switch by adding split of previous
source back
@@ -262,8 +261,7 @@ public class HybridSourceSplitEnumeratorTest {
private UnderlyingEnumeratorWrapper(MockSplitEnumerator enumerator) {
this.enumerator = enumerator;
- this.context =
- (SplitEnumeratorContext)
Whitebox.getInternalState(enumerator, "context");
+ this.context = Whitebox.getInternalState(enumerator, "context");
}
@Override
@@ -324,11 +322,11 @@ public class HybridSourceSplitEnumeratorTest {
}
private static int getCurrentSourceIndex(HybridSourceSplitEnumerator
enumerator) {
- return (int) Whitebox.getInternalState(enumerator,
"currentSourceIndex");
+ return Whitebox.getInternalState(enumerator, "currentSourceIndex");
}
private static MockSplitEnumerator getCurrentEnumerator(
HybridSourceSplitEnumerator enumerator) {
- return (MockSplitEnumerator) Whitebox.getInternalState(enumerator,
"currentEnumerator");
+ return Whitebox.getInternalState(enumerator, "currentEnumerator");
}
}
diff --git
a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaEnumeratorTest.java
b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaEnumeratorTest.java
index 8c39b547fb0..13363aa6648 100644
---
a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaEnumeratorTest.java
+++
b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaEnumeratorTest.java
@@ -296,12 +296,11 @@ public class KafkaEnumeratorTest {
properties)) {
enumerator.start();
- AdminClient adminClient =
- (AdminClient) Whitebox.getInternalState(enumerator,
"adminClient");
+ AdminClient adminClient = Whitebox.getInternalState(enumerator,
"adminClient");
assertThat(adminClient).isNotNull();
- String clientId = (String) Whitebox.getInternalState(adminClient,
"clientId");
+ String clientId = Whitebox.getInternalState(adminClient,
"clientId");
assertThat(clientId).isNotNull().startsWith(clientIdPrefix);
- assertThat(Whitebox.getInternalState(adminClient,
"defaultApiTimeoutMs"))
+ assertThat((Integer) Whitebox.getInternalState(adminClient,
"defaultApiTimeoutMs"))
.isEqualTo(defaultTimeoutMs);
assertThat(clientId).isNotNull().startsWith(clientIdPrefix);
diff --git
a/flink-formats/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/CachedSchemaCoderProviderTest.java
b/flink-formats/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/CachedSchemaCoderProviderTest.java
index cddbf134989..3e8f53cdf47 100644
---
a/flink-formats/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/CachedSchemaCoderProviderTest.java
+++
b/flink-formats/flink-avro-confluent-registry/src/test/java/org/apache/flink/formats/avro/registry/confluent/CachedSchemaCoderProviderTest.java
@@ -18,12 +18,13 @@
package org.apache.flink.formats.avro.registry.confluent;
+import org.apache.flink.mock.Whitebox;
+
import io.confluent.kafka.schemaregistry.client.CachedSchemaRegistryClient;
import io.confluent.kafka.schemaregistry.client.rest.RestService;
import
io.confluent.kafka.schemaregistry.client.security.basicauth.BasicAuthCredentialProvider;
import
io.confluent.kafka.schemaregistry.client.security.bearerauth.BearerAuthCredentialProvider;
import org.junit.jupiter.api.Test;
-import org.powermock.reflect.Whitebox;
import javax.net.ssl.SSLSocketFactory;
diff --git
a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroOutputFormatTest.java
b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroOutputFormatTest.java
index 1ce568c0d46..514e8ac1c1c 100644
---
a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroOutputFormatTest.java
+++
b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/AvroOutputFormatTest.java
@@ -120,9 +120,8 @@ class AvroOutputFormatTest {
@SuppressWarnings("unchecked")
final AvroOutputFormat<User> restored = (AvroOutputFormat<User>) o;
final AvroOutputFormat.Codec restoredCodec =
- (AvroOutputFormat.Codec)
Whitebox.getInternalState(restored, "codec");
- final Schema restoredSchema =
- (Schema) Whitebox.getInternalState(restored,
"userDefinedSchema");
+ Whitebox.getInternalState(restored, "codec");
+ final Schema restoredSchema = Whitebox.getInternalState(restored,
"userDefinedSchema");
assertThat(codec).isSameAs(restoredCodec);
assertThat(schema).isEqualTo(restoredSchema);
diff --git
a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java
b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java
index c04cb3e3da7..2f2f31eeeea 100644
---
a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java
+++
b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CEPOperatorTest.java
@@ -491,8 +491,7 @@ public class CEPOperatorTest extends TestLogger {
harness.open();
final ValueState nfaOperatorState =
- (ValueState)
- Whitebox.<ValueState>getInternalState(operator,
"computationStates");
+ Whitebox.getInternalState(operator, "computationStates");
final ValueState nfaOperatorStateSpy =
Mockito.spy(nfaOperatorState);
Whitebox.setInternalState(operator, "computationStates",
nfaOperatorStateSpy);
@@ -537,8 +536,7 @@ public class CEPOperatorTest extends TestLogger {
harness.open();
final ValueState nfaOperatorState =
- (ValueState)
- Whitebox.<ValueState>getInternalState(operator,
"computationStates");
+ Whitebox.getInternalState(operator, "computationStates");
final ValueState nfaOperatorStateSpy =
Mockito.spy(nfaOperatorState);
Whitebox.setInternalState(operator, "computationStates",
nfaOperatorStateSpy);
diff --git
a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/CompactingHashTable.java
b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/CompactingHashTable.java
index 41ed6c91d45..02dedb6c295 100644
---
a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/CompactingHashTable.java
+++
b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/hash/CompactingHashTable.java
@@ -18,6 +18,7 @@
package org.apache.flink.runtime.operators.hash;
+import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.typeutils.TypeComparator;
import org.apache.flink.api.common.typeutils.TypePairComparator;
import org.apache.flink.api.common.typeutils.TypeSerializer;
@@ -870,7 +871,8 @@ public class CompactingHashTable<T> extends
AbstractMutableHashTable<T> {
* @return true on success
* @throws IOException
*/
- private boolean resizeHashTable() throws IOException {
+ @VisibleForTesting
+ boolean resizeHashTable() throws IOException {
final int newNumBuckets = 2 * this.numBuckets;
final int bucketsPerSegment = this.bucketsPerSegmentMask + 1;
final int newNumSegments = (newNumBuckets + (bucketsPerSegment - 1)) /
bucketsPerSegment;
@@ -1106,6 +1108,13 @@ public class CompactingHashTable<T> extends
AbstractMutableHashTable<T> {
}
}
+ @VisibleForTesting
+ void compactPartitions() throws IOException {
+ for (int x = 0; x < partitions.size(); x++) {
+ compactPartition(x);
+ }
+ }
+
/**
* Compacts (garbage collects) partition with copy-compact strategy using
compaction partition
*
diff --git
a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/CompactingHashTableTest.java
b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/CompactingHashTableTest.java
index 7e7c9f05765..b7d29e61d9e 100644
---
a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/CompactingHashTableTest.java
+++
b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/hash/CompactingHashTableTest.java
@@ -31,9 +31,7 @@ import
org.apache.flink.runtime.operators.testutils.types.IntPair;
import org.apache.flink.util.MutableObjectIterator;
import org.junit.Test;
-import org.powermock.reflect.Whitebox;
-import java.lang.reflect.Field;
import java.util.ArrayList;
import java.util.BitSet;
import java.util.List;
@@ -289,7 +287,7 @@ public class CompactingHashTableTest extends
MutableHashTableTestBase {
// make sure there is enough memory for resize
memory.addAll(getMemory(ADDITIONAL_MEM));
- Boolean b = Whitebox.<Boolean>invokeMethod(table,
"resizeHashTable");
+ Boolean b = table.resizeHashTable();
assertTrue(b);
for (int i = 0; i < NUM_PAIRS; i++) {
@@ -337,7 +335,7 @@ public class CompactingHashTableTest extends
MutableHashTableTestBase {
// make sure there is enough memory for resize
memory.addAll(getMemory(ADDITIONAL_MEM));
- Boolean b = Whitebox.<Boolean>invokeMethod(table,
"resizeHashTable");
+ Boolean b = table.resizeHashTable();
assertTrue(b);
for (int i = 0; i < NUM_PAIRS; i++) {
@@ -349,7 +347,7 @@ public class CompactingHashTableTest extends
MutableHashTableTestBase {
// make sure there is enough memory for resize
memory.addAll(getMemory(ADDITIONAL_MEM));
- b = Whitebox.<Boolean>invokeMethod(table, "resizeHashTable");
+ b = table.resizeHashTable();
assertTrue(b);
for (int i = 0; i < NUM_PAIRS; i++) {
@@ -399,7 +397,7 @@ public class CompactingHashTableTest extends
MutableHashTableTestBase {
// make sure there is enough memory for resize
memory.addAll(getMemory(ADDITIONAL_MEM));
- Boolean b = Whitebox.<Boolean>invokeMethod(table,
"resizeHashTable");
+ Boolean b = table.resizeHashTable();
assertTrue(b);
for (int i = 0; i < NUM_PAIRS; i++) {
@@ -411,7 +409,7 @@ public class CompactingHashTableTest extends
MutableHashTableTestBase {
// make sure there is enough memory for resize
memory.addAll(getMemory(ADDITIONAL_MEM));
- b = Whitebox.<Boolean>invokeMethod(table, "resizeHashTable");
+ b = table.resizeHashTable();
assertTrue(b);
for (int i = 0; i < NUM_PAIRS; i++) {
@@ -423,7 +421,7 @@ public class CompactingHashTableTest extends
MutableHashTableTestBase {
// make sure there is enough memory for resize
memory.addAll(getMemory(2 * ADDITIONAL_MEM));
- b = Whitebox.<Boolean>invokeMethod(table, "resizeHashTable");
+ b = table.resizeHashTable();
assertTrue(b);
for (int i = 0; i < NUM_PAIRS; i++) {
@@ -473,7 +471,7 @@ public class CompactingHashTableTest extends
MutableHashTableTestBase {
// make sure there is enough memory for resize
memory.addAll(getMemory(ADDITIONAL_MEM));
- Boolean b = Whitebox.<Boolean>invokeMethod(table,
"resizeHashTable");
+ Boolean b = table.resizeHashTable();
assertTrue(b);
for (int i = 0; i < NUM_LISTS; i++) {
@@ -488,18 +486,11 @@ public class CompactingHashTableTest extends
MutableHashTableTestBase {
table.insertOrReplaceRecord(overwriteLists[i]);
}
- Field list = Whitebox.getField(CompactingHashTable.class,
"partitions");
- @SuppressWarnings("unchecked")
- ArrayList<InMemoryPartition<IntList>> partitions =
- (ArrayList<InMemoryPartition<IntList>>) list.get(table);
- int numPartitions = partitions.size();
- for (int i = 0; i < numPartitions; i++) {
- Whitebox.invokeMethod(table, "compactPartition", i);
- }
+ table.compactPartitions();
// make sure there is enough memory for resize
memory.addAll(getMemory(2 * ADDITIONAL_MEM));
- b = Whitebox.<Boolean>invokeMethod(table, "resizeHashTable");
+ b = table.resizeHashTable();
assertTrue(b);
for (int i = 0; i < NUM_LISTS; i++) {
diff --git
a/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/mock/Whitebox.java
b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/mock/Whitebox.java
index 11f1678c123..b4faded7b2a 100644
---
a/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/mock/Whitebox.java
+++
b/flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/mock/Whitebox.java
@@ -27,12 +27,13 @@ import java.lang.reflect.Field;
* the terms of the MIT License.
*/
public class Whitebox {
- public static Object getInternalState(Object target, String field) {
+ @SuppressWarnings("unchecked")
+ public static <T> T getInternalState(Object target, String field) {
Class<?> c = target.getClass();
try {
Field f = getFieldFromHierarchy(c, field);
f.setAccessible(true);
- return f.get(target);
+ return (T) f.get(target);
} catch (Exception e) {
throw new RuntimeException(
"Unable to set internal state on a private field. Please
report to mockito mailing list.",