You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by wa...@apache.org on 2013/10/15 02:19:49 UTC

svn commit: r1532153 - in /hadoop/common/branches/HDFS-4949/hadoop-common-project/hadoop-common/src: main/java/org/apache/hadoop/util/ test/java/org/apache/hadoop/util/

Author: wang
Date: Tue Oct 15 00:19:48 2013
New Revision: 1532153

URL: http://svn.apache.org/r1532153
Log:
HDFS-5359. Allow LightWeightGSet#Iterator to remove elements. (Contributed by Colin Patrick McCabe)

Added:
    hadoop/common/branches/HDFS-4949/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestLightWeightGSet.java   (with props)
Modified:
    hadoop/common/branches/HDFS-4949/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightCache.java
    hadoop/common/branches/HDFS-4949/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightGSet.java

Modified: hadoop/common/branches/HDFS-4949/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightCache.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightCache.java?rev=1532153&r1=1532152&r2=1532153&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightCache.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightCache.java Tue Oct 15 00:19:48 2013
@@ -18,6 +18,7 @@
 package org.apache.hadoop.util;
 
 import java.util.Comparator;
+import java.util.Iterator;
 import java.util.PriorityQueue;
 
 import org.apache.hadoop.HadoopIllegalArgumentException;
@@ -235,4 +236,28 @@ public class LightWeightCache<K, E exten
     }
     return removed;
   }
+
+  @Override
+  public Iterator<E> iterator() {
+    final Iterator<E> iter = super.iterator();
+    return new Iterator<E>() {
+      @Override
+      public boolean hasNext() {
+        return iter.hasNext();
+      }
+
+      @Override
+      public E next() {
+        return iter.next();
+      }
+
+      @Override
+      public void remove() {
+        // It would be tricky to support this because LightWeightCache#remove
+        // may evict multiple elements via evictExpiredEntries.
+        throw new UnsupportedOperationException("Remove via iterator is " +
+            "not supported for LightWeightCache");
+      }
+    };
+  }
 }

Modified: hadoop/common/branches/HDFS-4949/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightGSet.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightGSet.java?rev=1532153&r1=1532152&r2=1532153&view=diff
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightGSet.java (original)
+++ hadoop/common/branches/HDFS-4949/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightGSet.java Tue Oct 15 00:19:48 2013
@@ -246,10 +246,10 @@ public class LightWeightGSet<K, E extend
 
   private class SetIterator implements Iterator<E> {
     /** The starting modification for fail-fast. */
-    private final int startModification = modification;
+    private int iterModification = modification;
     /** The current index of the entry array. */
     private int index = -1;
-    /** The next element to return. */
+    private LinkedElement cur = null;
     private LinkedElement next = nextNonemptyEntry();
 
     /** Find the next nonempty entry starting at (index + 1). */
@@ -258,30 +258,51 @@ public class LightWeightGSet<K, E extend
       return index < entries.length? entries[index]: null;
     }
 
+    private void ensureNext() {
+      if (modification != iterModification) {
+        throw new ConcurrentModificationException("modification=" + modification
+            + " != iterModification = " + iterModification);
+      }
+      if (next != null) {
+        return;
+      }
+      if (cur == null) {
+        return;
+      }
+      next = cur.getNext();
+      if (next == null) {
+        next = nextNonemptyEntry();
+      }
+    }
+
     @Override
     public boolean hasNext() {
+      ensureNext();
       return next != null;
     }
 
     @Override
     public E next() {
-      if (modification != startModification) {
-        throw new ConcurrentModificationException("modification=" + modification
-            + " != startModification = " + startModification);
+      ensureNext();
+      if (next == null) {
+        throw new IllegalStateException("There are no more elements");
       }
-
-      final E e = convert(next);
-
-      //find the next element
-      final LinkedElement n = next.getNext();
-      next = n != null? n: nextNonemptyEntry();
-
-      return e;
+      cur = next;
+      next = null;
+      return convert(cur);
     }
 
+    @SuppressWarnings("unchecked")
     @Override
     public void remove() {
-      throw new UnsupportedOperationException("Remove is not supported.");
+      ensureNext();
+      if (cur == null) {
+        throw new IllegalStateException("There is no current element " +
+            "to remove");
+      }
+      LightWeightGSet.this.remove((K)cur);
+      iterModification++;
+      cur = null;
     }
   }
   

Added: hadoop/common/branches/HDFS-4949/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestLightWeightGSet.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/HDFS-4949/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestLightWeightGSet.java?rev=1532153&view=auto
==============================================================================
--- hadoop/common/branches/HDFS-4949/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestLightWeightGSet.java (added)
+++ hadoop/common/branches/HDFS-4949/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestLightWeightGSet.java Tue Oct 15 00:19:48 2013
@@ -0,0 +1,110 @@
+/**
+ * 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.hadoop.util;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.Random;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.util.LightWeightGSet.LinkedElement;
+import org.junit.Assert;
+import org.junit.Test;
+
+/** Testing {@link LightWeightGSet} */
+public class TestLightWeightGSet {
+  public static final Log LOG = LogFactory.getLog(TestLightWeightGSet.class);
+
+  private static ArrayList<Integer> getRandomList(int length, int randomSeed) {
+    Random random = new Random(randomSeed);
+    ArrayList<Integer> list = new ArrayList<Integer>(length);
+    for (int i = 0; i < length; i++) {
+      list.add(random.nextInt());
+    }
+    return list;
+  }
+  
+  private static class TestElement implements LightWeightGSet.LinkedElement {
+    private final int val;
+    private LinkedElement next;
+
+    TestElement(int val) {
+      this.val = val;
+      this.next = null;
+    }
+    
+    public int getVal() {
+      return val;
+    }
+
+    @Override
+    public void setNext(LinkedElement next) {
+      this.next = next;
+    }
+
+    @Override
+    public LinkedElement getNext() {
+      return next;
+    }
+  }
+
+  @Test(timeout=60000)
+  public void testRemoveAllViaIterator() {
+    ArrayList<Integer> list = getRandomList(100, 123);
+    LightWeightGSet<TestElement, TestElement> set =
+        new LightWeightGSet<TestElement, TestElement>(16);
+    for (Integer i : list) {
+      set.put(new TestElement(i));
+    }
+    for (Iterator<TestElement> iter = set.iterator();
+        iter.hasNext(); ) {
+      iter.next();
+      iter.remove();
+    }
+    Assert.assertEquals(0, set.size());
+  }
+
+  @Test(timeout=60000)
+  public void testRemoveSomeViaIterator() {
+    ArrayList<Integer> list = getRandomList(100, 123);
+    LightWeightGSet<TestElement, TestElement> set =
+        new LightWeightGSet<TestElement, TestElement>(16);
+    for (Integer i : list) {
+      set.put(new TestElement(i));
+    }
+    long sum = 0;
+    for (Iterator<TestElement> iter = set.iterator();
+        iter.hasNext(); ) {
+      sum += iter.next().getVal();
+    }
+    long mode = sum / set.size();
+    LOG.info("Removing all elements above " + mode);
+    for (Iterator<TestElement> iter = set.iterator();
+        iter.hasNext(); ) {
+      int item = iter.next().getVal();
+      if (item > mode) {
+        iter.remove();
+      }
+    }
+    for (Iterator<TestElement> iter = set.iterator();
+        iter.hasNext(); ) {
+      Assert.assertTrue(iter.next().getVal() <= mode);
+    }
+  }
+}

Propchange: hadoop/common/branches/HDFS-4949/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestLightWeightGSet.java
------------------------------------------------------------------------------
    svn:eol-style = native