You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by zm...@apache.org on 2016/02/27 00:58:23 UTC

aurora git commit: Do not modify list populated by MyBatis.

Repository: aurora
Updated Branches:
  refs/heads/master 7e32c4381 -> 8f98e6e8a


Do not modify list populated by MyBatis.

A ConcurrentModificationException can occur when multiple threads are sorting
the same task event list. This is possible because we have enabled MyBatis
caching with the readOnly flag which can use the same instance to populate
different `DbScheduledTask` instances.

Bugs closed: AURORA-1625

Reviewed at https://reviews.apache.org/r/44106/


Project: http://git-wip-us.apache.org/repos/asf/aurora/repo
Commit: http://git-wip-us.apache.org/repos/asf/aurora/commit/8f98e6e8
Tree: http://git-wip-us.apache.org/repos/asf/aurora/tree/8f98e6e8
Diff: http://git-wip-us.apache.org/repos/asf/aurora/diff/8f98e6e8

Branch: refs/heads/master
Commit: 8f98e6e8a50e59abfd4b80a0aa2696a6bed6daf9
Parents: 7e32c43
Author: Zameer Manji <zm...@apache.org>
Authored: Fri Feb 26 15:58:01 2016 -0800
Committer: Zameer Manji <zm...@apache.org>
Committed: Fri Feb 26 15:58:08 2016 -0800

----------------------------------------------------------------------
 .../storage/db/views/DbScheduledTask.java       |  6 ++--
 .../storage/AbstractTaskStoreTest.java          | 29 ++++++++++++++++++++
 2 files changed, 32 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/8f98e6e8/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbScheduledTask.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbScheduledTask.java b/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbScheduledTask.java
index 8d0dab0..6260923 100644
--- a/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbScheduledTask.java
+++ b/src/main/java/org/apache/aurora/scheduler/storage/db/views/DbScheduledTask.java
@@ -13,7 +13,6 @@
  */
 package org.apache.aurora.scheduler.storage.db.views;
 
-import java.util.Collections;
 import java.util.List;
 
 import com.google.common.collect.Ordering;
@@ -35,13 +34,14 @@ public final class DbScheduledTask {
   }
 
   public IScheduledTask toImmutable() {
-    Collections.sort(taskEvents, BY_TIMESTAMP);
     return IScheduledTask.build(
         new ScheduledTask()
             .setAssignedTask(assignedTask.toThrift())
             .setStatus(status)
             .setFailureCount(failureCount)
-            .setTaskEvents(taskEvents)
+            // Must be sorting a copy because taskEvents is populated by MyBatis and it might
+            // reuse the same instance.
+            .setTaskEvents(BY_TIMESTAMP.immutableSortedCopy(taskEvents))
             .setAncestorId(ancestorId));
   }
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/8f98e6e8/src/test/java/org/apache/aurora/scheduler/storage/AbstractTaskStoreTest.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/aurora/scheduler/storage/AbstractTaskStoreTest.java b/src/test/java/org/apache/aurora/scheduler/storage/AbstractTaskStoreTest.java
index dc5e2bd..e56fed2 100644
--- a/src/test/java/org/apache/aurora/scheduler/storage/AbstractTaskStoreTest.java
+++ b/src/test/java/org/apache/aurora/scheduler/storage/AbstractTaskStoreTest.java
@@ -13,11 +13,13 @@
  */
 package org.apache.aurora.scheduler.storage;
 
+import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 
 import com.google.common.base.Optional;
@@ -25,7 +27,12 @@ import com.google.common.collect.FluentIterable;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
+import com.google.common.util.concurrent.Futures;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import com.google.inject.Guice;
@@ -552,6 +559,28 @@ public abstract class AbstractTaskStoreTest extends TearDownTestCase {
     saveTasks(c);
   }
 
+  @Test
+  public void testConcurrentFetchTasks() throws Exception {
+    // Test for regression of AURORA-1625
+    ListeningExecutorService executor =
+        MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(5));
+    assertStoreContents();
+    saveTasks(TASK_A, TASK_B, TASK_C, TASK_D);
+
+    List<ListenableFuture<Integer>> futures = Lists.newArrayList();
+
+    for (int i = 0; i < 100; i++) {
+      futures.add(executor.submit(() -> Iterables.size(fetchTasks(Query.unscoped()))));
+    }
+
+    Future<List<Integer>> f = Futures.allAsList(futures);
+
+    executor.shutdown();
+    executor.awaitTermination(1, TimeUnit.MINUTES);
+
+    assertEquals(Iterables.getOnlyElement(ImmutableSet.copyOf(f.get())), (Integer) 4);
+  }
+
   private void assertStoreContents(IScheduledTask... tasks) {
     assertQueryResults(Query.unscoped(), tasks);
   }