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 */
018package org.apache.hadoop.hdfs.util;
019
020import java.util.ConcurrentModificationException;
021import java.util.Iterator;
022import java.util.ArrayList;
023import java.util.List;
024import 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 */
035public 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    @Override
051    public String toString() {
052      return super.toString();
053    }
054  }
055
056  private DoubleLinkedElement<T> head;
057  private DoubleLinkedElement<T> tail;
058
059  private LinkedSetIterator bookmark;
060
061  /**
062   * @param initCapacity
063   *          Recommended size of the internal array.
064   * @param maxLoadFactor
065   *          used to determine when to expand the internal array
066   * @param minLoadFactor
067   *          used to determine when to shrink the internal array
068   */
069  public LightWeightLinkedSet(int initCapacity, float maxLoadFactor,
070      float minLoadFactor) {
071    super(initCapacity, maxLoadFactor, minLoadFactor);
072    head = null;
073    tail = null;
074    bookmark = new LinkedSetIterator();
075  }
076
077  public LightWeightLinkedSet() {
078    this(MINIMUM_CAPACITY, DEFAULT_MAX_LOAD_FACTOR, DEFAUT_MIN_LOAD_FACTOR);
079  }
080
081  /**
082   * Add given element to the hash table
083   *
084   * @return true if the element was not present in the table, false otherwise
085   */
086  @Override
087  protected boolean addElem(final T element) {
088    // validate element
089    if (element == null) {
090      throw new IllegalArgumentException("Null element is not supported.");
091    }
092    // find hashCode & index
093    final int hashCode = element.hashCode();
094    final int index = getIndex(hashCode);
095    // return false if already present
096    if (getContainedElem(index, element, hashCode) != null) {
097      return false;
098    }
099
100    modification++;
101    size++;
102
103    // update bucket linked list
104    DoubleLinkedElement<T> le = new DoubleLinkedElement<T>(element, hashCode);
105    le.next = entries[index];
106    entries[index] = le;
107
108    // insert to the end of the all-element linked list
109    le.after = null;
110    le.before = tail;
111    if (tail != null) {
112      tail.after = le;
113    }
114    tail = le;
115    if (head == null) {
116      head = le;
117      bookmark.next = head;
118    }
119
120    // Update bookmark, if necessary.
121    if (bookmark.next == null) {
122      bookmark.next = le;
123    }
124    return true;
125  }
126
127  /**
128   * Remove the element corresponding to the key, given key.hashCode() == index.
129   *
130   * @return Return the entry with the element if exists. Otherwise return null.
131   */
132  @Override
133  protected DoubleLinkedElement<T> removeElem(final T key) {
134    DoubleLinkedElement<T> found = (DoubleLinkedElement<T>) (super
135        .removeElem(key));
136    if (found == null) {
137      return null;
138    }
139
140    // update linked list
141    if (found.after != null) {
142      found.after.before = found.before;
143    }
144    if (found.before != null) {
145      found.before.after = found.after;
146    }
147    if (head == found) {
148      head = head.after;
149    }
150    if (tail == found) {
151      tail = tail.before;
152    }
153
154    // Update bookmark, if necessary.
155    if (found == this.bookmark.next) {
156      this.bookmark.next = found.after;
157    }
158    return found;
159  }
160
161  /**
162   * Remove and return first element on the linked list of all elements.
163   *
164   * @return first element
165   */
166  public T pollFirst() {
167    if (head == null) {
168      return null;
169    }
170    T first = head.element;
171    this.remove(first);
172    return first;
173  }
174
175  /**
176   * Remove and return n elements from the hashtable.
177   * The order in which entries are removed is corresponds 
178   * to the order in which they were inserted.
179   *
180   * @return first element
181   */
182  @Override
183  public List<T> pollN(int n) {
184    if (n >= size) {
185      // if we need to remove all elements then do fast polling
186      return pollAll();
187    }
188    List<T> retList = new ArrayList<T>(n);
189    while (n-- > 0 && head != null) {
190      T curr = head.element;
191      this.removeElem(curr);
192      retList.add(curr);
193    }
194    shrinkIfNecessary();
195    return retList;
196  }
197
198  /**
199   * Remove all elements from the set and return them in order. Traverse the
200   * link list, don't worry about hashtable - faster version of the parent
201   * method.
202   */
203  @Override
204  public List<T> pollAll() {
205    List<T> retList = new ArrayList<T>(size);
206    while (head != null) {
207      retList.add(head.element);
208      head = head.after;
209    }
210    this.clear();
211    return retList;
212  }
213
214  @Override
215  @SuppressWarnings("unchecked")
216  public <U> U[] toArray(U[] a) {
217    if (a == null) {
218      throw new NullPointerException("Input array can not be null");
219    }
220    if (a.length < size) {
221      a = (U[]) java.lang.reflect.Array.newInstance(a.getClass()
222          .getComponentType(), size);
223    }
224    int currentIndex = 0;
225    DoubleLinkedElement<T> current = head;
226    while (current != null) {
227      T curr = current.element;
228      a[currentIndex++] = (U) curr;
229      current = current.after;
230    }
231    return a;
232  }
233
234  @Override
235  public Iterator<T> iterator() {
236    return new LinkedSetIterator();
237  }
238
239  private class LinkedSetIterator implements Iterator<T> {
240    /** The starting modification for fail-fast. */
241    private final int startModification = modification;
242    /** The next element to return. */
243    private DoubleLinkedElement<T> next = head;
244
245    @Override
246    public boolean hasNext() {
247      return next != null;
248    }
249
250    @Override
251    public T next() {
252      if (modification != startModification) {
253        throw new ConcurrentModificationException("modification="
254            + modification + " != startModification = " + startModification);
255      }
256      if (next == null) {
257        throw new NoSuchElementException();
258      }
259      final T e = next.element;
260      // find the next element
261      next = next.after;
262      return e;
263    }
264
265    @Override
266    public void remove() {
267      throw new UnsupportedOperationException("Remove is not supported.");
268    }
269  }
270
271  /**
272   * Clear the set. Resize it to the original capacity.
273   */
274  @Override
275  public void clear() {
276    super.clear();
277    this.head = null;
278    this.tail = null;
279    this.resetBookmark();
280  }
281
282  /**
283   * Returns a new iterator starting at the bookmarked element.
284   *
285   * @return the iterator to the bookmarked element.
286   */
287  public Iterator<T> getBookmark() {
288    LinkedSetIterator toRet = new LinkedSetIterator();
289    toRet.next = this.bookmark.next;
290    this.bookmark = toRet;
291    return toRet;
292  }
293
294  /**
295   * Resets the bookmark to the beginning of the list.
296   */
297  public void resetBookmark() {
298    this.bookmark.next = this.head;
299  }
300}