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