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