zentol commented on a change in pull request #16064: URL: https://github.com/apache/flink/pull/16064#discussion_r644542905
########## File path: flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/testutils/junit/SharedObjects.java ########## @@ -0,0 +1,156 @@ +/* + * 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.flink.testutils.junit; + +import org.junit.rules.ExternalResource; + +import java.io.ObjectStreamException; +import java.io.Serializable; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * This rule allows objects to be used both in the main test case as well as in UDFs by using + * serializable {@link SharedReference}s. Usage: + * + * <pre><code> + * {@literal @Rule} + * public final SharedObjects sharedObjects = SharedObjects.create(); + * + * {@literal @Test} + * public void test() throws Exception { + * StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + * SerialRef<Queue<Long>> list = sharedObjects.add(new ConcurrentLinkedQueue<>()); + * int n = 10000; + * env.setParallelism(100); + * env.fromSequence(0, n).map(i -> list.get().add(i)); + * env.execute(); + * assertEquals(n + 1, list.get().size()); + * assertEquals( + * LongStream.rangeClosed(0, n).boxed().collect(Collectors.toList()), + * list.get().stream().sorted().collect(Collectors.toList())); + * } + * </code></pre> + * + * <p>Note that since the shared objects are accessed through multiple threads, they need to be thread-safe or accessed in a thread-safe manner. + */ +public class SharedObjects extends ExternalResource implements Serializable { + /** + * Instance-cache used to make a SharedObjects accessible for multiple threads. + */ + private static final Map<Integer, SharedObjects> INSTANCES = new ConcurrentHashMap<>(); + private static final AtomicInteger LAST_ID = new AtomicInteger(); + /** + * Identifier of the SharedObjects used to retrieve the original instance during deserialization. + */ + private final int id; + /** + * All registered objects for the current test case. The objects are purged upon completion. + */ + private final transient Map<SharedReference<?>, Object> objects = new ConcurrentHashMap<>(); + + private SharedObjects(int id) { + this.id = id; + } + + /** + * Creates a new instance. Usually that should be done inside a JUnit test class as an + * instance-field annotated with {@link org.junit.Rule}. + */ + public static SharedObjects create() { + return new SharedObjects(LAST_ID.getAndIncrement()); + } + + /** + * Adds a new object to this {@code SharedObjects}. Although not necessary, it is recommended + * to only access the object through the returned {@link SharedReference}. + */ + public <T> SharedReference<T> add(T object) { + SharedReference<T> tag = new DefaultTag<>(this, objects.size()); + objects.put(tag, object); + return tag; + } + + @Override + protected void before() throws Throwable { + INSTANCES.put(id, this); + } + + @Override + protected void after() { + objects.clear(); + INSTANCES.remove(id); + } + + private Object readResolve() throws ObjectStreamException { Review comment: can we put this somewhere more prominent, because when scanning the file this is easily missed yet is fundamental to the functionality. ########## File path: flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/testutils/junit/SharedObjects.java ########## @@ -0,0 +1,156 @@ +/* + * 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.flink.testutils.junit; + +import org.junit.rules.ExternalResource; + +import java.io.ObjectStreamException; +import java.io.Serializable; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * This rule allows objects to be used both in the main test case as well as in UDFs by using + * serializable {@link SharedReference}s. Usage: + * + * <pre><code> + * {@literal @Rule} + * public final SharedObjects sharedObjects = SharedObjects.create(); + * + * {@literal @Test} + * public void test() throws Exception { + * StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + * SerialRef<Queue<Long>> list = sharedObjects.add(new ConcurrentLinkedQueue<>()); + * int n = 10000; + * env.setParallelism(100); + * env.fromSequence(0, n).map(i -> list.get().add(i)); + * env.execute(); + * assertEquals(n + 1, list.get().size()); + * assertEquals( + * LongStream.rangeClosed(0, n).boxed().collect(Collectors.toList()), + * list.get().stream().sorted().collect(Collectors.toList())); + * } + * </code></pre> + * + * <p>Note that since the shared objects are accessed through multiple threads, they need to be thread-safe or accessed in a thread-safe manner. + */ +public class SharedObjects extends ExternalResource implements Serializable { + /** + * Instance-cache used to make a SharedObjects accessible for multiple threads. + */ + private static final Map<Integer, SharedObjects> INSTANCES = new ConcurrentHashMap<>(); + private static final AtomicInteger LAST_ID = new AtomicInteger(); + /** + * Identifier of the SharedObjects used to retrieve the original instance during deserialization. + */ + private final int id; + /** + * All registered objects for the current test case. The objects are purged upon completion. + */ + private final transient Map<SharedReference<?>, Object> objects = new ConcurrentHashMap<>(); + + private SharedObjects(int id) { + this.id = id; + } + + /** + * Creates a new instance. Usually that should be done inside a JUnit test class as an + * instance-field annotated with {@link org.junit.Rule}. + */ + public static SharedObjects create() { + return new SharedObjects(LAST_ID.getAndIncrement()); + } + + /** + * Adds a new object to this {@code SharedObjects}. Although not necessary, it is recommended + * to only access the object through the returned {@link SharedReference}. + */ + public <T> SharedReference<T> add(T object) { + SharedReference<T> tag = new DefaultTag<>(this, objects.size()); + objects.put(tag, object); + return tag; Review comment: this isn't thread-safe and can result in 2 objects with the same tag. ########## File path: flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/testutils/junit/SharedReference.java ########## @@ -0,0 +1,62 @@ +/* + * 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.flink.testutils.junit; + +import java.io.Serializable; +import java.util.function.Consumer; +import java.util.function.Function; + +/** + * Represents an object managed in a {@link SharedObjects}. The reference can serialized and will + * still point to the same instance after deserialization in the same JVM. The underlying object + * may change the state but this reference will never point to another object. Review comment: ```suggestion * Represents an object managed in a {@link SharedObjects}. The reference can be serialized and will * still point to the same instance after deserialization in the same JVM. The underlying object * may change the state but this reference will never point to another object. ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected]
