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.ConcurrentModificationException;
022import java.util.Iterator;
023
024import org.apache.commons.logging.Log;
025import org.apache.commons.logging.LogFactory;
026import org.apache.hadoop.classification.InterfaceAudience;
027import 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
047public 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}