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.io.PrintStream;
021import java.util.ArrayList;
022import java.util.Collection;
023import java.util.ConcurrentModificationException;
024import java.util.Iterator;
025import java.util.LinkedList;
026import java.util.List;
027import java.util.NoSuchElementException;
028
029import org.apache.commons.logging.Log;
030import org.apache.commons.logging.LogFactory;
031
032/**
033 * A low memory linked hash set implementation, which uses an array for storing
034 * the elements and linked lists for collision resolution. This class does not
035 * support null element.
036 *
037 * This class is not thread safe.
038 *
039 */
040public class LightWeightHashSet<T> implements Collection<T> {
041  /**
042   * Elements of {@link LightWeightLinkedSet}.
043   */
044  static class LinkedElement<T> {
045    protected final T element;
046
047    // reference to the next entry within a bucket linked list
048    protected LinkedElement<T> next;
049
050    //hashCode of the element
051    protected final int hashCode;
052
053    public LinkedElement(T elem, int hash) {
054      this.element = elem;
055      this.next = null;
056      this.hashCode = hash;
057    }
058
059    @Override
060    public String toString() {
061      return element.toString();
062    }
063  }
064
065  protected static final float DEFAULT_MAX_LOAD_FACTOR = 0.75f;
066  protected static final float DEFAUT_MIN_LOAD_FACTOR = 0.2f;
067  protected static final int MINIMUM_CAPACITY = 16;
068
069  static final int MAXIMUM_CAPACITY = 1 << 30;
070  private static final Log LOG = LogFactory.getLog(LightWeightHashSet.class);
071
072  /**
073   * An internal array of entries, which are the rows of the hash table. The
074   * size must be a power of two.
075   */
076  protected LinkedElement<T>[] entries;
077  /** Size of the entry table. */
078  private int capacity;
079  /** The size of the set (not the entry array). */
080  protected int size = 0;
081  /** Hashmask used for determining the bucket index **/
082  private int hash_mask;
083  /** Capacity at initialization time **/
084  private final int initialCapacity;
085
086  /**
087   * Modification version for fail-fast.
088   *
089   * @see ConcurrentModificationException
090   */
091  protected int modification = 0;
092
093  private float maxLoadFactor;
094  private float minLoadFactor;
095  private final int expandMultiplier = 2;
096
097  private int expandThreshold;
098  private int shrinkThreshold;
099
100  /**
101   * @param initCapacity
102   *          Recommended size of the internal array.
103   * @param maxLoadFactor
104   *          used to determine when to expand the internal array
105   * @param minLoadFactor
106   *          used to determine when to shrink the internal array
107   */
108  @SuppressWarnings("unchecked")
109  public LightWeightHashSet(int initCapacity, float maxLoadFactor,
110      float minLoadFactor) {
111
112    if (maxLoadFactor <= 0 || maxLoadFactor > 1.0f)
113      throw new IllegalArgumentException("Illegal maxload factor: "
114          + maxLoadFactor);
115
116    if (minLoadFactor <= 0 || minLoadFactor > maxLoadFactor)
117      throw new IllegalArgumentException("Illegal minload factor: "
118          + minLoadFactor);
119
120    this.initialCapacity = computeCapacity(initCapacity);
121    this.capacity = this.initialCapacity;
122    this.hash_mask = capacity - 1;
123
124    this.maxLoadFactor = maxLoadFactor;
125    this.expandThreshold = (int) (capacity * maxLoadFactor);
126    this.minLoadFactor = minLoadFactor;
127    this.shrinkThreshold = (int) (capacity * minLoadFactor);
128
129    entries = new LinkedElement[capacity];
130    LOG.debug("initial capacity=" + initialCapacity + ", max load factor= "
131        + maxLoadFactor + ", min load factor= " + minLoadFactor);
132  }
133
134  public LightWeightHashSet() {
135    this(MINIMUM_CAPACITY, DEFAULT_MAX_LOAD_FACTOR, DEFAUT_MIN_LOAD_FACTOR);
136  }
137
138  public LightWeightHashSet(int minCapacity) {
139    this(minCapacity, DEFAULT_MAX_LOAD_FACTOR, DEFAUT_MIN_LOAD_FACTOR);
140  }
141
142  /**
143   * Check if the set is empty.
144   *
145   * @return true is set empty, false otherwise
146   */
147  @Override
148  public boolean isEmpty() {
149    return size == 0;
150  }
151
152  /**
153   * Return the current capacity (for testing).
154   */
155  public int getCapacity() {
156    return capacity;
157  }
158
159  /**
160   * Return the number of stored elements.
161   */
162  @Override
163  public int size() {
164    return size;
165  }
166
167  /**
168   * Get index in the internal table for a given hash.
169   */
170  protected int getIndex(int hashCode) {
171    return hashCode & hash_mask;
172  }
173
174  /**
175   * Check if the set contains given element
176   *
177   * @return true if element present, false otherwise.
178   */
179  @SuppressWarnings("unchecked")
180  @Override
181  public boolean contains(final Object key) {
182    return getElement((T)key) != null;
183  }
184  
185  /**
186   * Return the element in this set which is equal to
187   * the given key, if such an element exists.
188   * Otherwise returns null.
189   */
190  public T getElement(final T key) {
191    // validate key
192    if (key == null) {
193      throw new IllegalArgumentException("Null element is not supported.");
194    }
195    // find element
196    final int hashCode = key.hashCode();
197    final int index = getIndex(hashCode);
198    return getContainedElem(index, key, hashCode);
199  }
200
201  /**
202   * Check if the set contains given element at given index. If it
203   * does, return that element.
204   *
205   * @return the element, or null, if no element matches
206   */
207  protected T getContainedElem(int index, final T key, int hashCode) {
208    for (LinkedElement<T> e = entries[index]; e != null; e = e.next) {
209      // element found
210      if (hashCode == e.hashCode && e.element.equals(key)) {
211        return e.element;
212      }
213    }
214    // element not found
215    return null;
216  }
217
218  /**
219   * All all elements in the collection. Expand if necessary.
220   *
221   * @param toAdd - elements to add.
222   * @return true if the set has changed, false otherwise
223   */
224  @Override
225  public boolean addAll(Collection<? extends T> toAdd) {
226    boolean changed = false;
227    for (T elem : toAdd) {
228      changed |= addElem(elem);
229    }
230    expandIfNecessary();
231    return changed;
232  }
233
234  /**
235   * Add given element to the hash table. Expand table if necessary.
236   *
237   * @return true if the element was not present in the table, false otherwise
238   */
239  @Override
240  public boolean add(final T element) {
241    boolean added = addElem(element);
242    expandIfNecessary();
243    return added;
244  }
245
246  /**
247   * Add given element to the hash table
248   *
249   * @return true if the element was not present in the table, false otherwise
250   */
251  protected boolean addElem(final T element) {
252    // validate element
253    if (element == null) {
254      throw new IllegalArgumentException("Null element is not supported.");
255    }
256    // find hashCode & index
257    final int hashCode = element.hashCode();
258    final int index = getIndex(hashCode);
259    // return false if already present
260    if (getContainedElem(index, element, hashCode) != null) {
261      return false;
262    }
263
264    modification++;
265    size++;
266
267    // update bucket linked list
268    LinkedElement<T> le = new LinkedElement<T>(element, hashCode);
269    le.next = entries[index];
270    entries[index] = le;
271    return true;
272  }
273
274  /**
275   * Remove the element corresponding to the key.
276   *
277   * @return If such element exists, return true. Otherwise, return false.
278   */
279  @Override
280  @SuppressWarnings("unchecked")
281  public boolean remove(final Object key) {
282    // validate key
283    if (key == null) {
284      throw new IllegalArgumentException("Null element is not supported.");
285    }
286    LinkedElement<T> removed = removeElem((T) key);
287    shrinkIfNecessary();
288    return removed == null ? false : true;
289  }
290
291  /**
292   * Remove the element corresponding to the key, given key.hashCode() == index.
293   *
294   * @return If such element exists, return true. Otherwise, return false.
295   */
296  protected LinkedElement<T> removeElem(final T key) {
297    LinkedElement<T> found = null;
298    final int hashCode = key.hashCode();
299    final int index = getIndex(hashCode);
300    if (entries[index] == null) {
301      return null;
302    } else if (hashCode == entries[index].hashCode &&
303            entries[index].element.equals(key)) {
304      // remove the head of the bucket linked list
305      modification++;
306      size--;
307      found = entries[index];
308      entries[index] = found.next;
309    } else {
310      // head != null and key is not equal to head
311      // search the element
312      LinkedElement<T> prev = entries[index];
313      for (found = prev.next; found != null;) {
314        if (hashCode == found.hashCode &&
315                found.element.equals(key)) {
316          // found the element, remove it
317          modification++;
318          size--;
319          prev.next = found.next;
320          found.next = null;
321          break;
322        } else {
323          prev = found;
324          found = found.next;
325        }
326      }
327    }
328    return found;
329  }
330
331  /**
332   * Remove and return n elements from the hashtable.
333   * The order in which entries are removed is unspecified, and
334   * and may not correspond to the order in which they were inserted.
335   *
336   * @return first element
337   */
338  public List<T> pollN(int n) {
339    if (n >= size) {
340      return pollAll();
341    }
342    List<T> retList = new ArrayList<T>(n);
343    if (n == 0) {
344      return retList;
345    }
346    boolean done = false;
347    int currentBucketIndex = 0;
348
349    while (!done) {
350      LinkedElement<T> current = entries[currentBucketIndex];
351      while (current != null) {
352        retList.add(current.element);
353        current = current.next;
354        entries[currentBucketIndex] = current;
355        size--;
356        modification++;
357        if (--n == 0) {
358          done = true;
359          break;
360        }
361      }
362      currentBucketIndex++;
363    }
364    shrinkIfNecessary();
365    return retList;
366  }
367
368  /**
369   * Remove all elements from the set and return them. Clear the entries.
370   */
371  public List<T> pollAll() {
372    List<T> retList = new ArrayList<T>(size);
373    for (int i = 0; i < entries.length; i++) {
374      LinkedElement<T> current = entries[i];
375      while (current != null) {
376        retList.add(current.element);
377        current = current.next;
378      }
379    }
380    this.clear();
381    return retList;
382  }
383
384  /**
385   * Get array.length elements from the set, and put them into the array.
386   */
387  @SuppressWarnings("unchecked")
388  public T[] pollToArray(T[] array) {
389    int currentIndex = 0;
390    LinkedElement<T> current = null;
391
392    if (array.length == 0) {
393      return array;
394    }
395    if (array.length > size) {
396      array = (T[]) java.lang.reflect.Array.newInstance(array.getClass()
397          .getComponentType(), size);
398    }
399    // do fast polling if the entire set needs to be fetched
400    if (array.length == size) {
401      for (int i = 0; i < entries.length; i++) {
402        current = entries[i];
403        while (current != null) {
404          array[currentIndex++] = current.element;
405          current = current.next;
406        }
407      }
408      this.clear();
409      return array;
410    }
411
412    boolean done = false;
413    int currentBucketIndex = 0;
414
415    while (!done) {
416      current = entries[currentBucketIndex];
417      while (current != null) {
418        array[currentIndex++] = current.element;
419        current = current.next;
420        entries[currentBucketIndex] = current;
421        size--;
422        modification++;
423        if (currentIndex == array.length) {
424          done = true;
425          break;
426        }
427      }
428      currentBucketIndex++;
429    }
430    shrinkIfNecessary();
431    return array;
432  }
433
434  /**
435   * Compute capacity given initial capacity.
436   *
437   * @return final capacity, either MIN_CAPACITY, MAX_CAPACITY, or power of 2
438   *         closest to the requested capacity.
439   */
440  private int computeCapacity(int initial) {
441    if (initial < MINIMUM_CAPACITY) {
442      return MINIMUM_CAPACITY;
443    }
444    if (initial > MAXIMUM_CAPACITY) {
445      return MAXIMUM_CAPACITY;
446    }
447    int capacity = 1;
448    while (capacity < initial) {
449      capacity <<= 1;
450    }
451    return capacity;
452  }
453
454  /**
455   * Resize the internal table to given capacity.
456   */
457  @SuppressWarnings("unchecked")
458  private void resize(int cap) {
459    int newCapacity = computeCapacity(cap);
460    if (newCapacity == this.capacity) {
461      return;
462    }
463    this.capacity = newCapacity;
464    this.expandThreshold = (int) (capacity * maxLoadFactor);
465    this.shrinkThreshold = (int) (capacity * minLoadFactor);
466    this.hash_mask = capacity - 1;
467    LinkedElement<T>[] temp = entries;
468    entries = new LinkedElement[capacity];
469    for (int i = 0; i < temp.length; i++) {
470      LinkedElement<T> curr = temp[i];
471      while (curr != null) {
472        LinkedElement<T> next = curr.next;
473        int index = getIndex(curr.hashCode);
474        curr.next = entries[index];
475        entries[index] = curr;
476        curr = next;
477      }
478    }
479  }
480
481  /**
482   * Checks if we need to shrink, and shrinks if necessary.
483   */
484  protected void shrinkIfNecessary() {
485    if (size < this.shrinkThreshold && capacity > initialCapacity) {
486      resize(capacity / expandMultiplier);
487    }
488  }
489
490  /**
491   * Checks if we need to expand, and expands if necessary.
492   */
493  protected void expandIfNecessary() {
494    if (size > this.expandThreshold && capacity < MAXIMUM_CAPACITY) {
495      resize(capacity * expandMultiplier);
496    }
497  }
498
499  @Override
500  public Iterator<T> iterator() {
501    return new LinkedSetIterator();
502  }
503
504  @Override
505  public String toString() {
506    final StringBuilder b = new StringBuilder(getClass().getSimpleName());
507    b.append("(size=").append(size).append(", modification=")
508        .append(modification).append(", entries.length=")
509        .append(entries.length).append(")");
510    return b.toString();
511  }
512
513  /** Print detailed information of this object. */
514  public void printDetails(final PrintStream out) {
515    out.print(this + ", entries = [");
516    for (int i = 0; i < entries.length; i++) {
517      if (entries[i] != null) {
518        LinkedElement<T> e = entries[i];
519        out.print("\n  " + i + ": " + e);
520        for (e = e.next; e != null; e = e.next) {
521          out.print(" -> " + e);
522        }
523      }
524    }
525    out.println("\n]");
526  }
527
528  private class LinkedSetIterator implements Iterator<T> {
529    /** The starting modification for fail-fast. */
530    private final int startModification = modification;
531    /** The current index of the entry array. */
532    private int index = -1;
533    /** The next element to return. */
534    private LinkedElement<T> next = nextNonemptyEntry();
535
536    private LinkedElement<T> nextNonemptyEntry() {
537      for (index++; index < entries.length && entries[index] == null; index++);
538      return index < entries.length ? entries[index] : null;
539    }
540
541    @Override
542    public boolean hasNext() {
543      return next != null;
544    }
545
546    @Override
547    public T next() {
548      if (modification != startModification) {
549        throw new ConcurrentModificationException("modification="
550            + modification + " != startModification = " + startModification);
551      }
552      if (next == null) {
553        throw new NoSuchElementException();
554      }
555      final T e = next.element;
556      // find the next element
557      final LinkedElement<T> n = next.next;
558      next = n != null ? n : nextNonemptyEntry();
559      return e;
560    }
561
562    @Override
563    public void remove() {
564      throw new UnsupportedOperationException("Remove is not supported.");
565    }
566  }
567
568  /**
569   * Clear the set. Resize it to the original capacity.
570   */
571  @Override
572  @SuppressWarnings("unchecked")
573  public void clear() {
574    this.capacity = this.initialCapacity;
575    this.hash_mask = capacity - 1;
576
577    this.expandThreshold = (int) (capacity * maxLoadFactor);
578    this.shrinkThreshold = (int) (capacity * minLoadFactor);
579
580    entries = new LinkedElement[capacity];
581    size = 0;
582    modification++;
583  }
584
585  @Override
586  public Object[] toArray() {
587    Object[] result = new Object[size];
588    return toArray(result);
589  }
590
591  @Override
592  @SuppressWarnings("unchecked")
593  public <U> U[] toArray(U[] a) {
594    if (a == null) {
595      throw new NullPointerException("Input array can not be null");
596    }
597    if (a.length < size) {
598      a = (U[]) java.lang.reflect.Array.newInstance(a.getClass()
599          .getComponentType(), size);
600    }
601    int currentIndex = 0;
602    for (int i = 0; i < entries.length; i++) {
603      LinkedElement<T> current = entries[i];
604      while (current != null) {
605        a[currentIndex++] = (U) current.element;
606        current = current.next;
607      }
608    }
609    return a;
610  }
611
612  @Override
613  public boolean containsAll(Collection<?> c) {
614    Iterator<?> iter = c.iterator();
615    while (iter.hasNext()) {
616      if (!contains(iter.next())) {
617        return false;
618      }
619    }
620    return true;
621  }
622
623  @Override
624  public boolean removeAll(Collection<?> c) {
625    boolean changed = false;
626    Iterator<?> iter = c.iterator();
627    while (iter.hasNext()) {
628      changed |= remove(iter.next());
629    }
630    return changed;
631  }
632
633  @Override
634  public boolean retainAll(Collection<?> c) {
635    throw new UnsupportedOperationException("retainAll is not supported.");
636  }
637}