001    /**
002     * Licensed to the Apache Software Foundation (ASF) under one
003     * or more contributor license agreements.  See the NOTICE file
004     * distributed with this work for additional information
005     * regarding copyright ownership.  The ASF licenses this file
006     * to you under the Apache License, Version 2.0 (the
007     * "License"); you may not use this file except in compliance
008     * with the License.  You may obtain a copy of the License at
009     *
010     *     http://www.apache.org/licenses/LICENSE-2.0
011     *
012     * Unless required by applicable law or agreed to in writing, software
013     * distributed under the License is distributed on an "AS IS" BASIS,
014     * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015     * See the License for the specific language governing permissions and
016     * limitations under the License.
017     */
018    package org.apache.hadoop.hdfs.util;
019    
020    import java.util.ConcurrentModificationException;
021    import java.util.Iterator;
022    import java.util.ArrayList;
023    import java.util.List;
024    import java.util.NoSuchElementException;
025    
026    /**
027     * A low memory linked hash set implementation, which uses an array for storing
028     * the elements and linked lists for collision resolution. In addition it stores
029     * elements in a linked list to ensure ordered traversal. This class does not
030     * support null element.
031     *
032     * This class is not thread safe.
033     *
034     */
035    public class LightWeightLinkedSet<T> extends LightWeightHashSet<T> {
036      /**
037       * Elements of {@link LightWeightLinkedSet}.
038       */
039      static class DoubleLinkedElement<T> extends LinkedElement<T> {
040        // references to elements within all-element linked list
041        private DoubleLinkedElement<T> before;
042        private DoubleLinkedElement<T> after;
043    
044        public DoubleLinkedElement(T elem, int hashCode) {
045          super(elem, hashCode);
046          this.before = null;
047          this.after = null;
048        }
049    
050        public String toString() {
051          return super.toString();
052        }
053      }
054    
055      private DoubleLinkedElement<T> head;
056      private DoubleLinkedElement<T> tail;
057    
058      /**
059       * @param initCapacity
060       *          Recommended size of the internal array.
061       * @param maxLoadFactor
062       *          used to determine when to expand the internal array
063       * @param minLoadFactor
064       *          used to determine when to shrink the internal array
065       */
066      public LightWeightLinkedSet(int initCapacity, float maxLoadFactor,
067          float minLoadFactor) {
068        super(initCapacity, maxLoadFactor, minLoadFactor);
069        head = null;
070        tail = null;
071      }
072    
073      public LightWeightLinkedSet() {
074        this(MINIMUM_CAPACITY, DEFAULT_MAX_LOAD_FACTOR, DEFAUT_MIN_LOAD_FACTOR);
075      }
076    
077      /**
078       * Add given element to the hash table
079       *
080       * @return true if the element was not present in the table, false otherwise
081       */
082      protected boolean addElem(final T element) {
083        // validate element
084        if (element == null) {
085          throw new IllegalArgumentException("Null element is not supported.");
086        }
087        // find hashCode & index
088        final int hashCode = element.hashCode();
089        final int index = getIndex(hashCode);
090        // return false if already present
091        if (containsElem(index, element, hashCode)) {
092          return false;
093        }
094    
095        modification++;
096        size++;
097    
098        // update bucket linked list
099        DoubleLinkedElement<T> le = new DoubleLinkedElement<T>(element, hashCode);
100        le.next = entries[index];
101        entries[index] = le;
102    
103        // insert to the end of the all-element linked list
104        le.after = null;
105        le.before = tail;
106        if (tail != null) {
107          tail.after = le;
108        }
109        tail = le;
110        if (head == null) {
111          head = le;
112        }
113        return true;
114      }
115    
116      /**
117       * Remove the element corresponding to the key, given key.hashCode() == index.
118       *
119       * @return Return the entry with the element if exists. Otherwise return null.
120       */
121      protected DoubleLinkedElement<T> removeElem(final T key) {
122        DoubleLinkedElement<T> found = (DoubleLinkedElement<T>) (super
123            .removeElem(key));
124        if (found == null) {
125          return null;
126        }
127    
128        // update linked list
129        if (found.after != null) {
130          found.after.before = found.before;
131        }
132        if (found.before != null) {
133          found.before.after = found.after;
134        }
135        if (head == found) {
136          head = head.after;
137        }
138        if (tail == found) {
139          tail = tail.before;
140        }
141        return found;
142      }
143    
144      /**
145       * Remove and return first element on the linked list of all elements.
146       *
147       * @return first element
148       */
149      public T pollFirst() {
150        if (head == null) {
151          return null;
152        }
153        T first = head.element;
154        this.remove(first);
155        return first;
156      }
157    
158      /**
159       * Remove and return n elements from the hashtable.
160       * The order in which entries are removed is corresponds 
161       * to the order in which they were inserted.
162       *
163       * @return first element
164       */
165      public List<T> pollN(int n) {
166        if (n >= size) {
167          // if we need to remove all elements then do fast polling
168          return pollAll();
169        }
170        List<T> retList = new ArrayList<T>(n);
171        while (n-- > 0 && head != null) {
172          T curr = head.element;
173          this.removeElem(curr);
174          retList.add(curr);
175        }
176        shrinkIfNecessary();
177        return retList;
178      }
179    
180      /**
181       * Remove all elements from the set and return them in order. Traverse the
182       * link list, don't worry about hashtable - faster version of the parent
183       * method.
184       */
185      public List<T> pollAll() {
186        List<T> retList = new ArrayList<T>(size);
187        while (head != null) {
188          retList.add(head.element);
189          head = head.after;
190        }
191        this.clear();
192        return retList;
193      }
194    
195      @Override
196      @SuppressWarnings("unchecked")
197      public <U> U[] toArray(U[] a) {
198        if (a == null) {
199          throw new NullPointerException("Input array can not be null");
200        }
201        if (a.length < size) {
202          a = (U[]) java.lang.reflect.Array.newInstance(a.getClass()
203              .getComponentType(), size);
204        }
205        int currentIndex = 0;
206        DoubleLinkedElement<T> current = head;
207        while (current != null) {
208          T curr = current.element;
209          a[currentIndex++] = (U) curr;
210          current = current.after;
211        }
212        return a;
213      }
214    
215      public Iterator<T> iterator() {
216        return new LinkedSetIterator();
217      }
218    
219      private class LinkedSetIterator implements Iterator<T> {
220        /** The starting modification for fail-fast. */
221        private final int startModification = modification;
222        /** The next element to return. */
223        private DoubleLinkedElement<T> next = head;
224    
225        @Override
226        public boolean hasNext() {
227          return next != null;
228        }
229    
230        @Override
231        public T next() {
232          if (modification != startModification) {
233            throw new ConcurrentModificationException("modification="
234                + modification + " != startModification = " + startModification);
235          }
236          if (next == null) {
237            throw new NoSuchElementException();
238          }
239          final T e = next.element;
240          // find the next element
241          next = next.after;
242          return e;
243        }
244    
245        @Override
246        public void remove() {
247          throw new UnsupportedOperationException("Remove is not supported.");
248        }
249      }
250    
251      /**
252       * Clear the set. Resize it to the original capacity.
253       */
254      public void clear() {
255        super.clear();
256        this.head = null;
257        this.tail = null;
258      }
259    }