You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by se...@apache.org on 2015/09/16 16:07:05 UTC

flink git commit: [FLINK-2670] [tests] Add tests that validate memory release under concurrent modification exceptions

Repository: flink
Updated Branches:
  refs/heads/master 2cc5d98c6 -> 31f63178d


[FLINK-2670] [tests] Add tests that validate memory release under concurrent modification exceptions


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/31f63178
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/31f63178
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/31f63178

Branch: refs/heads/master
Commit: 31f63178df2f81682527ba3debf9027bde78149b
Parents: 2cc5d98
Author: Stephan Ewen <se...@apache.org>
Authored: Wed Sep 16 15:10:48 2015 +0200
Committer: Stephan Ewen <se...@apache.org>
Committed: Wed Sep 16 15:10:48 2015 +0200

----------------------------------------------------------------------
 .../MemoryManagerConcurrentModReleaseTest.java  | 152 +++++++++++++++++++
 1 file changed, 152 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/31f63178/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerConcurrentModReleaseTest.java
----------------------------------------------------------------------
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerConcurrentModReleaseTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerConcurrentModReleaseTest.java
new file mode 100644
index 0000000..d1063cc
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/memory/MemoryManagerConcurrentModReleaseTest.java
@@ -0,0 +1,152 @@
+/*
+ * 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.runtime.memory;
+
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemoryType;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.ConcurrentModificationException;
+import java.util.Iterator;
+
+import static org.junit.Assert.*;
+
+
+public class MemoryManagerConcurrentModReleaseTest {
+
+	@Test
+	public void testConcurrentModificationOnce() {
+		try {
+			final int numSegments = 10000;
+			final int segmentSize = 4096;
+
+			MemoryManager memMan = new MemoryManager(numSegments * segmentSize, 1, segmentSize, MemoryType.HEAP, true);
+
+			ArrayList<MemorySegment> segs = new ListWithConcModExceptionOnFirstAccess<>();
+			memMan.allocatePages(this, segs, numSegments);
+
+			memMan.release(segs);
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	@Test
+	public void testConcurrentModificationWhileReleasing() {
+		try {
+			final int numSegments = 10000;
+			final int segmentSize = 4096;
+			
+			MemoryManager memMan = new MemoryManager(numSegments * segmentSize, 1, segmentSize, MemoryType.HEAP, true);
+			
+			ArrayList<MemorySegment> segs = new ArrayList<>(numSegments);
+			memMan.allocatePages(this, segs, numSegments);
+			
+			// start a thread that performs concurrent modifications
+			Modifier mod = new Modifier(segs);
+			Thread modRunner = new Thread(mod);
+			modRunner.start();
+			
+			// give the thread some time to start working
+			Thread.sleep(500);
+			
+			try {
+				memMan.release(segs);
+			}
+			finally {
+				mod.cancel();
+			}
+			
+			modRunner.join();
+		}
+		catch (Exception e) {
+			e.printStackTrace();
+			fail(e.getMessage());
+		}
+	}
+	
+	private class Modifier implements Runnable {
+		
+		private final ArrayList<MemorySegment> toModify;
+		
+		private volatile boolean running = true;
+
+		
+		private Modifier(ArrayList<MemorySegment> toModify) {
+			this.toModify = toModify;
+		}
+
+		public void cancel() {
+			running = false;
+		}
+
+		@Override
+		public void run() {
+			while (running) {
+				try {
+					MemorySegment seg = toModify.remove(0);
+					toModify.add(seg);
+				}
+				catch (IndexOutOfBoundsException e) {
+					// may happen, just retry
+				}
+			}
+		}
+	}
+	
+	private class ListWithConcModExceptionOnFirstAccess<E> extends ArrayList<E> {
+
+		private static final long serialVersionUID = -1623249699823349781L;
+		
+		private boolean returnedIterator;
+		
+		@Override
+		public Iterator<E> iterator() {
+			if (returnedIterator) {
+				return super.iterator();
+			}
+			else {
+				returnedIterator = true;
+				return new ConcFailingIterator<>();
+			}
+		}
+	}
+	
+	
+	private class ConcFailingIterator<E> implements Iterator<E> {
+
+		@Override
+		public boolean hasNext() {
+			return true;
+		}
+
+		@Override
+		public E next() {
+			throw new ConcurrentModificationException();
+		}
+
+		@Override
+		public void remove() {
+			throw new UnsupportedOperationException();
+		}
+	}
+}