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.util.ArrayList;
021    import java.util.Collections;
022    import java.util.Iterator;
023    import java.util.List;
024    
025    import 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     */
075    public 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    }