Back to home page

OSCL-LXR

 
 

    


0001 /*
0002  * Licensed to the Apache Software Foundation (ASF) under one or more
0003  * contributor license agreements.  See the NOTICE file distributed with
0004  * this work for additional information regarding copyright ownership.
0005  * The ASF licenses this file to You under the Apache License, Version 2.0
0006  * (the "License"); you may not use this file except in compliance with
0007  * the License.  You may obtain a copy of the License at
0008  *
0009  *    http://www.apache.org/licenses/LICENSE-2.0
0010  *
0011  * Unless required by applicable law or agreed to in writing, software
0012  * distributed under the License is distributed on an "AS IS" BASIS,
0013  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
0014  * See the License for the specific language governing permissions and
0015  * limitations under the License.
0016  */
0017 
0018 package org.apache.spark.util.kvstore;
0019 
0020 import com.google.common.base.Preconditions;
0021 
0022 import org.apache.spark.annotation.Private;
0023 
0024 /**
0025  * A configurable view that allows iterating over values in a {@link KVStore}.
0026  *
0027  * <p>
0028  * The different methods can be used to configure the behavior of the iterator. Calling the same
0029  * method multiple times is allowed; the most recent value will be used.
0030  * </p>
0031  *
0032  * <p>
0033  * The iterators returned by this view are of type {@link KVStoreIterator}; they auto-close
0034  * when used in a for loop that exhausts their contents, but when used manually, they need
0035  * to be closed explicitly unless all elements are read.
0036  * </p>
0037  */
0038 @Private
0039 public abstract class KVStoreView<T> implements Iterable<T> {
0040 
0041   boolean ascending = true;
0042   String index = KVIndex.NATURAL_INDEX_NAME;
0043   Object first = null;
0044   Object last = null;
0045   Object parent = null;
0046   long skip = 0L;
0047   long max = Long.MAX_VALUE;
0048 
0049   /**
0050    * Reverses the order of iteration. By default, iterates in ascending order.
0051    */
0052   public KVStoreView<T> reverse() {
0053     ascending = !ascending;
0054     return this;
0055   }
0056 
0057   /**
0058    * Iterates according to the given index.
0059    */
0060   public KVStoreView<T> index(String name) {
0061     this.index = Preconditions.checkNotNull(name);
0062     return this;
0063   }
0064 
0065   /**
0066    * Defines the value of the parent index when iterating over a child index. Only elements that
0067    * match the parent index's value will be included in the iteration.
0068    *
0069    * <p>
0070    * Required for iterating over child indices, will generate an error if iterating over a
0071    * parent-less index.
0072    * </p>
0073    */
0074   public KVStoreView<T> parent(Object value) {
0075     this.parent = value;
0076     return this;
0077   }
0078 
0079   /**
0080    * Iterates starting at the given value of the chosen index (inclusive).
0081    */
0082   public KVStoreView<T> first(Object value) {
0083     this.first = value;
0084     return this;
0085   }
0086 
0087   /**
0088    * Stops iteration at the given value of the chosen index (inclusive).
0089    */
0090   public KVStoreView<T> last(Object value) {
0091     this.last = value;
0092     return this;
0093   }
0094 
0095   /**
0096    * Stops iteration after a number of elements has been retrieved.
0097    */
0098   public KVStoreView<T> max(long max) {
0099     Preconditions.checkArgument(max > 0L, "max must be positive.");
0100     this.max = max;
0101     return this;
0102   }
0103 
0104   /**
0105    * Skips a number of elements at the start of iteration. Skipped elements are not accounted
0106    * when using {@link #max(long)}.
0107    */
0108   public KVStoreView<T> skip(long n) {
0109     this.skip = n;
0110     return this;
0111   }
0112 
0113   /**
0114    * Returns an iterator for the current configuration.
0115    */
0116   public KVStoreIterator<T> closeableIterator() throws Exception {
0117     return (KVStoreIterator<T>) iterator();
0118   }
0119 
0120 }