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