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        if (LOG.isDebugEnabled()) {
083          LOG.debug("recommended=" + recommended_length + ", actual=" + actual);
084        }
085    
086        entries = new LinkedElement[actual];
087        hash_mask = entries.length - 1;
088      }
089    
090      //compute actual length
091      private static int actualArrayLength(int recommended) {
092        if (recommended > MAX_ARRAY_LENGTH) {
093          return MAX_ARRAY_LENGTH;
094        } else if (recommended < MIN_ARRAY_LENGTH) {
095          return MIN_ARRAY_LENGTH;
096        } else {
097          final int a = Integer.highestOneBit(recommended);
098          return a == recommended? a: a << 1;
099        }
100      }
101    
102      @Override
103      public int size() {
104        return size;
105      }
106    
107      private int getIndex(final K key) {
108        return key.hashCode() & hash_mask;
109      }
110    
111      private E convert(final LinkedElement e){
112        @SuppressWarnings("unchecked")
113        final E r = (E)e;
114        return r;
115      }
116    
117      @Override
118      public E get(final K key) {
119        //validate key
120        if (key == null) {
121          throw new NullPointerException("key == null");
122        }
123    
124        //find element
125        final int index = getIndex(key);
126        for(LinkedElement e = entries[index]; e != null; e = e.getNext()) {
127          if (e.equals(key)) {
128            return convert(e);
129          }
130        }
131        //element not found
132        return null;
133      }
134    
135      @Override
136      public boolean contains(final K key) {
137        return get(key) != null;
138      }
139    
140      @Override
141      public E put(final E element) {
142        //validate element
143        if (element == null) {
144          throw new NullPointerException("Null element is not supported.");
145        }
146        if (!(element instanceof LinkedElement)) {
147          throw new HadoopIllegalArgumentException(
148              "!(element instanceof LinkedElement), element.getClass()="
149              + element.getClass());
150        }
151        final LinkedElement e = (LinkedElement)element;
152    
153        //find index
154        final int index = getIndex(element);
155    
156        //remove if it already exists
157        final E existing = remove(index, element);
158    
159        //insert the element to the head of the linked list
160        modification++;
161        size++;
162        e.setNext(entries[index]);
163        entries[index] = e;
164    
165        return existing;
166      }
167    
168      /**
169       * Remove the element corresponding to the key,
170       * given key.hashCode() == index.
171       *
172       * @return If such element exists, return it.
173       *         Otherwise, return null.
174       */
175      private E remove(final int index, final K key) {
176        if (entries[index] == null) {
177          return null;
178        } else if (entries[index].equals(key)) {
179          //remove the head of the linked list
180          modification++;
181          size--;
182          final LinkedElement e = entries[index];
183          entries[index] = e.getNext();
184          e.setNext(null);
185          return convert(e);
186        } else {
187          //head != null and key is not equal to head
188          //search the element
189          LinkedElement prev = entries[index];
190          for(LinkedElement curr = prev.getNext(); curr != null; ) {
191            if (curr.equals(key)) {
192              //found the element, remove it
193              modification++;
194              size--;
195              prev.setNext(curr.getNext());
196              curr.setNext(null);
197              return convert(curr);
198            } else {
199              prev = curr;
200              curr = curr.getNext();
201            }
202          }
203          //element not found
204          return null;
205        }
206      }
207    
208      @Override
209      public E remove(final K key) {
210        //validate key
211        if (key == null) {
212          throw new NullPointerException("key == null");
213        }
214        return remove(getIndex(key), key);
215      }
216    
217      @Override
218      public Iterator<E> iterator() {
219        return new SetIterator();
220      }
221    
222      @Override
223      public String toString() {
224        final StringBuilder b = new StringBuilder(getClass().getSimpleName());
225        b.append("(size=").append(size)
226         .append(String.format(", %08x", hash_mask))
227         .append(", modification=").append(modification)
228         .append(", entries.length=").append(entries.length)
229         .append(")");
230        return b.toString();
231      }
232    
233      /** Print detailed information of this object. */
234      public void printDetails(final PrintStream out) {
235        out.print(this + ", entries = [");
236        for(int i = 0; i < entries.length; i++) {
237          if (entries[i] != null) {
238            LinkedElement e = entries[i];
239            out.print("\n  " + i + ": " + e);
240            for(e = e.getNext(); e != null; e = e.getNext()) {
241              out.print(" -> " + e);
242            }
243          }
244        }
245        out.println("\n]");
246      }
247    
248      private class SetIterator implements Iterator<E> {
249        /** The starting modification for fail-fast. */
250        private final int startModification = modification;
251        /** The current index of the entry array. */
252        private int index = -1;
253        /** The next element to return. */
254        private LinkedElement next = nextNonemptyEntry();
255    
256        /** Find the next nonempty entry starting at (index + 1). */
257        private LinkedElement nextNonemptyEntry() {
258          for(index++; index < entries.length && entries[index] == null; index++);
259          return index < entries.length? entries[index]: null;
260        }
261    
262        @Override
263        public boolean hasNext() {
264          return next != null;
265        }
266    
267        @Override
268        public E next() {
269          if (modification != startModification) {
270            throw new ConcurrentModificationException("modification=" + modification
271                + " != startModification = " + startModification);
272          }
273    
274          final E e = convert(next);
275    
276          //find the next element
277          final LinkedElement n = next.getNext();
278          next = n != null? n: nextNonemptyEntry();
279    
280          return e;
281        }
282    
283        @Override
284        public void remove() {
285          throw new UnsupportedOperationException("Remove is not supported.");
286        }
287      }
288    }