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.ConcurrentModificationException;
022    import java.util.Iterator;
023    
024    import org.apache.commons.logging.Log;
025    import org.apache.commons.logging.LogFactory;
026    import org.apache.hadoop.classification.InterfaceAudience;
027    import org.apache.hadoop.HadoopIllegalArgumentException;
028    
029    /**
030     * A low memory footprint {@link GSet} implementation,
031     * which uses an array for storing the elements
032     * and linked lists for collision resolution.
033     *
034     * No rehash will be performed.
035     * Therefore, the internal array will never be resized.
036     *
037     * This class does not support null element.
038     *
039     * This class is not thread safe.
040     *
041     * @param <K> Key type for looking up the elements
042     * @param <E> Element type, which must be
043     *       (1) a subclass of K, and
044     *       (2) implementing {@link LinkedElement} interface.
045     */
046    @InterfaceAudience.Private
047    public class LightWeightGSet<K, E extends K> implements GSet<K, E> {
048      /**
049       * Elements of {@link LightWeightGSet}.
050       */
051      public static interface LinkedElement {
052        /** Set the next element. */
053        public void setNext(LinkedElement next);
054    
055        /** Get the next element. */
056        public LinkedElement getNext();
057      }
058    
059      public static final Log LOG = LogFactory.getLog(GSet.class);
060      static final int MAX_ARRAY_LENGTH = 1 << 30; //prevent int overflow problem
061      static final int MIN_ARRAY_LENGTH = 1;
062    
063      /**
064       * An internal array of entries, which are the rows of the hash table.
065       * The size must be a power of two.
066       */
067      private final LinkedElement[] entries;
068      /** A mask for computing the array index from the hash value of an element. */
069      private final int hash_mask;
070      /** The size of the set (not the entry array). */
071      private int size = 0;
072      /** Modification version for fail-fast.
073       * @see ConcurrentModificationException
074       */
075      private volatile int modification = 0;
076    
077      /**
078       * @param recommended_length Recommended size of the internal array.
079       */
080      public LightWeightGSet(final int recommended_length) {
081        final int actual = actualArrayLength(recommended_length);
082        LOG.info("recommended=" + recommended_length + ", actual=" + actual);
083    
084        entries = new LinkedElement[actual];
085        hash_mask = entries.length - 1;
086      }
087    
088      //compute actual length
089      private static int actualArrayLength(int recommended) {
090        if (recommended > MAX_ARRAY_LENGTH) {
091          return MAX_ARRAY_LENGTH;
092        } else if (recommended < MIN_ARRAY_LENGTH) {
093          return MIN_ARRAY_LENGTH;
094        } else {
095          final int a = Integer.highestOneBit(recommended);
096          return a == recommended? a: a << 1;
097        }
098      }
099    
100      @Override
101      public int size() {
102        return size;
103      }
104    
105      private int getIndex(final K key) {
106        return key.hashCode() & hash_mask;
107      }
108    
109      private E convert(final LinkedElement e){
110        @SuppressWarnings("unchecked")
111        final E r = (E)e;
112        return r;
113      }
114    
115      @Override
116      public E get(final K key) {
117        //validate key
118        if (key == null) {
119          throw new NullPointerException("key == null");
120        }
121    
122        //find element
123        final int index = getIndex(key);
124        for(LinkedElement e = entries[index]; e != null; e = e.getNext()) {
125          if (e.equals(key)) {
126            return convert(e);
127          }
128        }
129        //element not found
130        return null;
131      }
132    
133      @Override
134      public boolean contains(final K key) {
135        return get(key) != null;
136      }
137    
138      @Override
139      public E put(final E element) {
140        //validate element
141        if (element == null) {
142          throw new NullPointerException("Null element is not supported.");
143        }
144        if (!(element instanceof LinkedElement)) {
145          throw new HadoopIllegalArgumentException(
146              "!(element instanceof LinkedElement), element.getClass()="
147              + element.getClass());
148        }
149        final LinkedElement e = (LinkedElement)element;
150    
151        //find index
152        final int index = getIndex(element);
153    
154        //remove if it already exists
155        final E existing = remove(index, element);
156    
157        //insert the element to the head of the linked list
158        modification++;
159        size++;
160        e.setNext(entries[index]);
161        entries[index] = e;
162    
163        return existing;
164      }
165    
166      /**
167       * Remove the element corresponding to the key,
168       * given key.hashCode() == index.
169       *
170       * @return If such element exists, return it.
171       *         Otherwise, return null.
172       */
173      private E remove(final int index, final K key) {
174        if (entries[index] == null) {
175          return null;
176        } else if (entries[index].equals(key)) {
177          //remove the head of the linked list
178          modification++;
179          size--;
180          final LinkedElement e = entries[index];
181          entries[index] = e.getNext();
182          e.setNext(null);
183          return convert(e);
184        } else {
185          //head != null and key is not equal to head
186          //search the element
187          LinkedElement prev = entries[index];
188          for(LinkedElement curr = prev.getNext(); curr != null; ) {
189            if (curr.equals(key)) {
190              //found the element, remove it
191              modification++;
192              size--;
193              prev.setNext(curr.getNext());
194              curr.setNext(null);
195              return convert(curr);
196            } else {
197              prev = curr;
198              curr = curr.getNext();
199            }
200          }
201          //element not found
202          return null;
203        }
204      }
205    
206      @Override
207      public E remove(final K key) {
208        //validate key
209        if (key == null) {
210          throw new NullPointerException("key == null");
211        }
212        return remove(getIndex(key), key);
213      }
214    
215      @Override
216      public Iterator<E> iterator() {
217        return new SetIterator();
218      }
219    
220      @Override
221      public String toString() {
222        final StringBuilder b = new StringBuilder(getClass().getSimpleName());
223        b.append("(size=").append(size)
224         .append(String.format(", %08x", hash_mask))
225         .append(", modification=").append(modification)
226         .append(", entries.length=").append(entries.length)
227         .append(")");
228        return b.toString();
229      }
230    
231      /** Print detailed information of this object. */
232      public void printDetails(final PrintStream out) {
233        out.print(this + ", entries = [");
234        for(int i = 0; i < entries.length; i++) {
235          if (entries[i] != null) {
236            LinkedElement e = entries[i];
237            out.print("\n  " + i + ": " + e);
238            for(e = e.getNext(); e != null; e = e.getNext()) {
239              out.print(" -> " + e);
240            }
241          }
242        }
243        out.println("\n]");
244      }
245    
246      private class SetIterator implements Iterator<E> {
247        /** The starting modification for fail-fast. */
248        private final int startModification = modification;
249        /** The current index of the entry array. */
250        private int index = -1;
251        /** The next element to return. */
252        private LinkedElement next = nextNonemptyEntry();
253    
254        /** Find the next nonempty entry starting at (index + 1). */
255        private LinkedElement nextNonemptyEntry() {
256          for(index++; index < entries.length && entries[index] == null; index++);
257          return index < entries.length? entries[index]: null;
258        }
259    
260        @Override
261        public boolean hasNext() {
262          return next != null;
263        }
264    
265        @Override
266        public E next() {
267          if (modification != startModification) {
268            throw new ConcurrentModificationException("modification=" + modification
269                + " != startModification = " + startModification);
270          }
271    
272          final E e = convert(next);
273    
274          //find the next element
275          final LinkedElement n = next.getNext();
276          next = n != null? n: nextNonemptyEntry();
277    
278          return e;
279        }
280    
281        @Override
282        public void remove() {
283          throw new UnsupportedOperationException("Remove is not supported.");
284        }
285      }
286    }