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