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.util.ArrayList;
021import java.util.Collections;
022import java.util.Iterator;
023import java.util.List;
024
025import com.google.common.base.Preconditions;
026
027/**
028 * The difference between the current state and a previous state of a list.
029 * 
030 * Given a previous state of a set and a sequence of create, delete and modify
031 * operations such that the current state of the set can be obtained by applying
032 * the operations on the previous state, the following algorithm construct the
033 * difference between the current state and the previous state of the set.
034 * 
035 * <pre>
036 * Two lists are maintained in the algorithm:
037 * - c-list for newly created elements
038 * - d-list for the deleted elements
039 *
040 * Denote the state of an element by the following
041 *   (0, 0): neither in c-list nor d-list
042 *   (c, 0): in c-list but not in d-list
043 *   (0, d): in d-list but not in c-list
044 *   (c, d): in both c-list and d-list
045 *
046 * For each case below, ( , ) at the end shows the result state of the element.
047 *
048 * Case 1. Suppose the element i is NOT in the previous state.           (0, 0)
049 *   1.1. create i in current: add it to c-list                          (c, 0)
050 *   1.1.1. create i in current and then create: impossible
051 *   1.1.2. create i in current and then delete: remove it from c-list   (0, 0)
052 *   1.1.3. create i in current and then modify: replace it in c-list    (c', 0)
053 *
054 *   1.2. delete i from current: impossible
055 *
056 *   1.3. modify i in current: impossible
057 *
058 * Case 2. Suppose the element i is ALREADY in the previous state.       (0, 0)
059 *   2.1. create i in current: impossible
060 *
061 *   2.2. delete i from current: add it to d-list                        (0, d)
062 *   2.2.1. delete i from current and then create: add it to c-list      (c, d)
063 *   2.2.2. delete i from current and then delete: impossible
064 *   2.2.2. delete i from current and then modify: impossible
065 *
066 *   2.3. modify i in current: put it in both c-list and d-list          (c, d)
067 *   2.3.1. modify i in current and then create: impossible
068 *   2.3.2. modify i in current and then delete: remove it from c-list   (0, d)
069 *   2.3.3. modify i in current and then modify: replace it in c-list    (c', d)
070 * </pre>
071 *
072 * @param <K> The key type.
073 * @param <E> The element type, which must implement {@link Element} interface.
074 */
075public class Diff<K, E extends Diff.Element<K>> {
076  public static enum ListType {
077    CREATED, DELETED
078  }
079
080  /** An interface for the elements in a {@link Diff}. */
081  public static interface Element<K> extends Comparable<K> {
082    /** @return the key of this object. */
083    public K getKey();
084  }
085
086  /** An interface for passing a method in order to process elements. */
087  public static interface Processor<E> {
088    /** Process the given element. */
089    public void process(E element);
090  }
091
092  /** Containing exactly one element. */
093  public static class Container<E> {
094    private final E element;
095
096    private Container(E element) {
097      this.element = element;
098    }
099
100    /** @return the element. */
101    public E getElement() {
102      return element;
103    }
104  }
105  
106  /** 
107   * Undo information for some operations such as delete(E)
108   * and {@link Diff#modify(Element, Element)}.
109   */
110  public static class UndoInfo<E> {
111    private final int createdInsertionPoint;
112    private final E trashed;
113    private final Integer deletedInsertionPoint;
114    
115    private UndoInfo(final int createdInsertionPoint, final E trashed,
116        final Integer deletedInsertionPoint) {
117      this.createdInsertionPoint = createdInsertionPoint;
118      this.trashed = trashed;
119      this.deletedInsertionPoint = deletedInsertionPoint;
120    }
121    
122    public E getTrashedElement() {
123      return trashed;
124    }
125  }
126
127  private static final int DEFAULT_ARRAY_INITIAL_CAPACITY = 4;
128
129  /**
130   * Search the element from the list.
131   * @return -1 if the list is null; otherwise, return the insertion point
132   *    defined in {@link Collections#binarySearch(List, Object)}.
133   *    Note that, when the list is null, -1 is the correct insertion point.
134   */
135  protected static <K, E extends Comparable<K>> int search(
136      final List<E> elements, final K name) {
137    return elements == null? -1: Collections.binarySearch(elements, name);
138  }
139
140  private static <E> void remove(final List<E> elements, final int i,
141      final E expected) {
142    final E removed = elements.remove(-i - 1);
143    Preconditions.checkState(removed == expected,
144        "removed != expected=%s, removed=%s.", expected, removed);
145  }
146
147  /** c-list: element(s) created in current. */
148  private List<E> created;
149  /** d-list: element(s) deleted from current. */
150  private List<E> deleted;
151  
152  protected Diff() {}
153
154  protected Diff(final List<E> created, final List<E> deleted) {
155    this.created = created;
156    this.deleted = deleted;
157  }
158
159  /** @return the created list, which is never null. */
160  public List<E> getList(final ListType type) {
161    final List<E> list = type == ListType.CREATED? created: deleted;
162    return list == null? Collections.<E>emptyList(): list;
163  }
164
165  public int searchIndex(final ListType type, final K name) {
166    return search(getList(type), name);
167  }
168
169  /**
170   * @return null if the element is not found;
171   *         otherwise, return the element in the created/deleted list.
172   */
173  public E search(final ListType type, final K name) {
174    final List<E> list = getList(type); 
175    final int c = search(list, name);
176    return c < 0 ? null : list.get(c);
177  }
178  
179  /** @return true if no changes contained in the diff */
180  public boolean isEmpty() {
181    return (created == null || created.isEmpty())
182        && (deleted == null || deleted.isEmpty());
183  }
184  
185  /**
186   * Insert the given element to the created/deleted list.
187   * @param i the insertion point defined
188   *          in {@link Collections#binarySearch(List, Object)}
189   */
190  private void insert(final ListType type, final E element, final int i) {
191    List<E> list = type == ListType.CREATED? created: deleted; 
192    if (i >= 0) {
193      throw new AssertionError("Element already exists: element=" + element
194          + ", " + type + "=" + list);
195    }
196    if (list == null) {
197      list = new ArrayList<E>(DEFAULT_ARRAY_INITIAL_CAPACITY);
198      if (type == ListType.CREATED) {
199        created = list;
200      } else if (type == ListType.DELETED){
201        deleted = list;
202      }
203    }
204    list.add(-i - 1, element);
205  }
206
207  /**
208   * Create an element in current state.
209   * @return the c-list insertion point for undo.
210   */
211  public int create(final E element) {
212    final int c = search(created, element.getKey());
213    insert(ListType.CREATED, element, c);
214    return c;
215  }
216
217  /**
218   * Undo the previous create(E) operation. Note that the behavior is
219   * undefined if the previous operation is not create(E).
220   */
221  public void undoCreate(final E element, final int insertionPoint) {
222    remove(created, insertionPoint, element);
223  }
224
225  /**
226   * Delete an element from current state.
227   * @return the undo information.
228   */
229  public UndoInfo<E> delete(final E element) {
230    final int c = search(created, element.getKey());
231    E previous = null;
232    Integer d = null;
233    if (c >= 0) {
234      // remove a newly created element
235      previous = created.remove(c);
236    } else {
237      // not in c-list, it must be in previous
238      d = search(deleted, element.getKey());
239      insert(ListType.DELETED, element, d);
240    }
241    return new UndoInfo<E>(c, previous, d);
242  }
243  
244  /**
245   * Undo the previous delete(E) operation. Note that the behavior is
246   * undefined if the previous operation is not delete(E).
247   */
248  public void undoDelete(final E element, final UndoInfo<E> undoInfo) {
249    final int c = undoInfo.createdInsertionPoint;
250    if (c >= 0) {
251      created.add(c, undoInfo.trashed);
252    } else {
253      remove(deleted, undoInfo.deletedInsertionPoint, element);
254    }
255  }
256
257  /**
258   * Modify an element in current state.
259   * @return the undo information.
260   */
261  public UndoInfo<E> modify(final E oldElement, final E newElement) {
262    Preconditions.checkArgument(oldElement != newElement,
263        "They are the same object: oldElement == newElement = %s", newElement);
264    Preconditions.checkArgument(oldElement.compareTo(newElement.getKey()) == 0,
265        "The names do not match: oldElement=%s, newElement=%s",
266        oldElement, newElement);
267    final int c = search(created, newElement.getKey());
268    E previous = null;
269    Integer d = null;
270    if (c >= 0) {
271      // Case 1.1.3 and 2.3.3: element is already in c-list,
272      previous = created.set(c, newElement);
273      
274      // For previous != oldElement, set it to oldElement
275      previous = oldElement;
276    } else {
277      d = search(deleted, oldElement.getKey());
278      if (d < 0) {
279        // Case 2.3: neither in c-list nor d-list
280        insert(ListType.CREATED, newElement, c);
281        insert(ListType.DELETED, oldElement, d);
282      }
283    }
284    return new UndoInfo<E>(c, previous, d);
285  }
286
287  /**
288   * Undo the previous modify(E, E) operation. Note that the behavior
289   * is undefined if the previous operation is not modify(E, E).
290   */
291  public void undoModify(final E oldElement, final E newElement,
292      final UndoInfo<E> undoInfo) {
293    final int c = undoInfo.createdInsertionPoint;
294    if (c >= 0) {
295      created.set(c, undoInfo.trashed);
296    } else {
297      final int d = undoInfo.deletedInsertionPoint;
298      if (d < 0) {
299        remove(created, c, newElement);
300        remove(deleted, d, oldElement);
301      }
302    }
303  }
304
305  /**
306   * Find an element in the previous state.
307   * 
308   * @return null if the element cannot be determined in the previous state
309   *         since no change is recorded and it should be determined in the
310   *         current state; otherwise, return a {@link Container} containing the
311   *         element in the previous state. Note that the element can possibly
312   *         be null which means that the element is not found in the previous
313   *         state.
314   */
315  public Container<E> accessPrevious(final K name) {
316    return accessPrevious(name, created, deleted);
317  }
318
319  private static <K, E extends Diff.Element<K>> Container<E> accessPrevious(
320      final K name, final List<E> clist, final List<E> dlist) {
321    final int d = search(dlist, name);
322    if (d >= 0) {
323      // the element was in previous and was once deleted in current.
324      return new Container<E>(dlist.get(d));
325    } else {
326      final int c = search(clist, name);
327      // When c >= 0, the element in current is a newly created element.
328      return c < 0? null: new Container<E>(null);
329    }
330  }
331
332  /**
333   * Find an element in the current state.
334   * 
335   * @return null if the element cannot be determined in the current state since
336   *         no change is recorded and it should be determined in the previous
337   *         state; otherwise, return a {@link Container} containing the element in
338   *         the current state. Note that the element can possibly be null which
339   *         means that the element is not found in the current state.
340   */
341  public Container<E> accessCurrent(K name) {
342    return accessPrevious(name, deleted, created);
343  }
344
345  /**
346   * Apply this diff to previous state in order to obtain current state.
347   * @return the current state of the list.
348   */
349  public List<E> apply2Previous(final List<E> previous) {
350    return apply2Previous(previous,
351        getList(ListType.CREATED), getList(ListType.DELETED));
352  }
353
354  private static <K, E extends Diff.Element<K>> List<E> apply2Previous(
355      final List<E> previous, final List<E> clist, final List<E> dlist) {
356    // Assumptions:
357    // (A1) All lists are sorted.
358    // (A2) All elements in dlist must be in previous.
359    // (A3) All elements in clist must be not in tmp = previous - dlist.
360    final List<E> tmp = new ArrayList<E>(previous.size() - dlist.size());
361    {
362      // tmp = previous - dlist
363      final Iterator<E> i = previous.iterator();
364      for(E deleted : dlist) {
365        E e = i.next(); //since dlist is non-empty, e must exist by (A2).
366        int cmp = 0;
367        for(; (cmp = e.compareTo(deleted.getKey())) < 0; e = i.next()) {
368          tmp.add(e);
369        }
370        Preconditions.checkState(cmp == 0); // check (A2)
371      }
372      for(; i.hasNext(); ) {
373        tmp.add(i.next());
374      }
375    }
376
377    final List<E> current = new ArrayList<E>(tmp.size() + clist.size());
378    {
379      // current = tmp + clist
380      final Iterator<E> tmpIterator = tmp.iterator();
381      final Iterator<E> cIterator = clist.iterator();
382
383      E t = tmpIterator.hasNext()? tmpIterator.next(): null;
384      E c = cIterator.hasNext()? cIterator.next(): null;
385      for(; t != null || c != null; ) {
386        final int cmp = c == null? 1
387            : t == null? -1
388            : c.compareTo(t.getKey());
389
390        if (cmp < 0) {
391          current.add(c);
392          c = cIterator.hasNext()? cIterator.next(): null;
393        } else if (cmp > 0) {
394          current.add(t);
395          t = tmpIterator.hasNext()? tmpIterator.next(): null;
396        } else {
397          throw new AssertionError("Violated assumption (A3).");
398        }
399      }
400    }
401    return current;
402  }
403
404  /**
405   * Apply the reverse of this diff to current state in order
406   * to obtain the previous state.
407   * @return the previous state of the list.
408   */
409  public List<E> apply2Current(final List<E> current) {
410    return apply2Previous(current,
411        getList(ListType.DELETED), getList(ListType.CREATED));
412  }
413  
414  /**
415   * Combine this diff with a posterior diff.  We have the following cases:
416   * 
417   * <pre>
418   * 1. For (c, 0) in the posterior diff, check the element in this diff:
419   * 1.1 (c', 0)  in this diff: impossible
420   * 1.2 (0, d')  in this diff: put in c-list --> (c, d')
421   * 1.3 (c', d') in this diff: impossible
422   * 1.4 (0, 0)   in this diff: put in c-list --> (c, 0)
423   * This is the same logic as create(E).
424   * 
425   * 2. For (0, d) in the posterior diff,
426   * 2.1 (c', 0)  in this diff: remove from c-list --> (0, 0)
427   * 2.2 (0, d')  in this diff: impossible
428   * 2.3 (c', d') in this diff: remove from c-list --> (0, d')
429   * 2.4 (0, 0)   in this diff: put in d-list --> (0, d)
430   * This is the same logic as delete(E).
431   * 
432   * 3. For (c, d) in the posterior diff,
433   * 3.1 (c', 0)  in this diff: replace the element in c-list --> (c, 0)
434   * 3.2 (0, d')  in this diff: impossible
435   * 3.3 (c', d') in this diff: replace the element in c-list --> (c, d')
436   * 3.4 (0, 0)   in this diff: put in c-list and d-list --> (c, d)
437   * This is the same logic as modify(E, E).
438   * </pre>
439   * 
440   * @param posterior The posterior diff to combine with.
441   * @param deletedProcesser
442   *     process the deleted/overwritten elements in case 2.1, 2.3, 3.1 and 3.3.
443   */
444  public void combinePosterior(final Diff<K, E> posterior,
445      final Processor<E> deletedProcesser) {
446    final Iterator<E> createdIterator = posterior.getList(ListType.CREATED).iterator();
447    final Iterator<E> deletedIterator = posterior.getList(ListType.DELETED).iterator();
448
449    E c = createdIterator.hasNext()? createdIterator.next(): null;
450    E d = deletedIterator.hasNext()? deletedIterator.next(): null;
451
452    for(; c != null || d != null; ) {
453      final int cmp = c == null? 1
454          : d == null? -1
455          : c.compareTo(d.getKey());
456      if (cmp < 0) {
457        // case 1: only in c-list
458        create(c);
459        c = createdIterator.hasNext()? createdIterator.next(): null;
460      } else if (cmp > 0) {
461        // case 2: only in d-list
462        final UndoInfo<E> ui = delete(d);
463        if (deletedProcesser != null) {
464          deletedProcesser.process(ui.trashed);
465        }
466        d = deletedIterator.hasNext()? deletedIterator.next(): null;
467      } else {
468        // case 3: in both c-list and d-list 
469        final UndoInfo<E> ui = modify(d, c);
470        if (deletedProcesser != null) {
471          deletedProcesser.process(ui.trashed);
472        }
473        c = createdIterator.hasNext()? createdIterator.next(): null;
474        d = deletedIterator.hasNext()? deletedIterator.next(): null;
475      }
476    }
477  }
478
479  @Override
480  public String toString() {
481    return getClass().getSimpleName()
482        +  "{created=" + getList(ListType.CREATED)
483        + ", deleted=" + getList(ListType.DELETED) + "}";
484  }
485}