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     */
018    package org.apache.hadoop.hdfs.util;
019    
020    import java.io.PrintStream;
021    import java.util.ArrayList;
022    import java.util.Collection;
023    import java.util.ConcurrentModificationException;
024    import java.util.Iterator;
025    import java.util.LinkedList;
026    import java.util.List;
027    import java.util.NoSuchElementException;
028    
029    import org.apache.commons.logging.Log;
030    import 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     */
040    public 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 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    }