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    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}