View Javadoc

1   /*
2    * Copyright 2011 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.util;
21  
22  import java.util.ArrayList;
23  import java.util.Collections;
24  import java.util.Iterator;
25  import java.util.List;
26  import java.util.SortedSet;
27  
28  import org.apache.hadoop.hbase.KeyValue;
29  import org.apache.hadoop.hbase.regionserver.AbstractKeyValueScanner;
30  import org.apache.hadoop.hbase.regionserver.KeyValueScanner;
31  
32  /**
33   * Utility scanner that wraps a sortable collection and serves
34   * as a KeyValueScanner.
35   */
36  public class CollectionBackedScanner extends AbstractKeyValueScanner {
37    final private Iterable<KeyValue> data;
38    final KeyValue.KVComparator comparator;
39    private Iterator<KeyValue> iter;
40    private KeyValue current;
41  
42    public CollectionBackedScanner(SortedSet<KeyValue> set) {
43      this(set, KeyValue.COMPARATOR);
44    }
45  
46    public CollectionBackedScanner(SortedSet<KeyValue> set,
47        KeyValue.KVComparator comparator) {
48      this.comparator = comparator;
49      data = set;
50      init();
51    }
52  
53    public CollectionBackedScanner(List<KeyValue> list) {
54      this(list, KeyValue.COMPARATOR);
55    }
56  
57    public CollectionBackedScanner(List<KeyValue> list,
58        KeyValue.KVComparator comparator) {
59      Collections.sort(list, comparator);
60      this.comparator = comparator;
61      data = list;
62      init();
63    }
64  
65    public CollectionBackedScanner(KeyValue.KVComparator comparator,
66        KeyValue... array) {
67      this.comparator = comparator;
68  
69      List<KeyValue> tmp = new ArrayList<KeyValue>(array.length);
70      for( int i = 0; i < array.length ; ++i) {
71        tmp.add(array[i]);
72      }
73      Collections.sort(tmp, comparator);
74      data = tmp;
75      init();
76    }
77  
78    private void init() {
79      iter = data.iterator();
80      if(iter.hasNext()){
81        current = iter.next();
82      }
83    }
84  
85    @Override
86    public KeyValue peek() {
87      return current;
88    }
89  
90    @Override
91    public KeyValue next() {
92      KeyValue oldCurrent = current;
93      if(iter.hasNext()){
94        current = iter.next();
95      } else {
96        current = null;
97      }
98      return oldCurrent;
99    }
100 
101   @Override
102   public boolean seek(KeyValue seekKv) {
103     // restart iterator
104     iter = data.iterator();
105     return reseek(seekKv);
106   }
107 
108   @Override
109   public boolean reseek(KeyValue seekKv) {
110     while(iter.hasNext()){
111       KeyValue next = iter.next();
112       int ret = comparator.compare(next, seekKv);
113       if(ret >= 0){
114         current = next;
115         return true;
116       }
117     }
118     return false;
119   }
120 
121   @Override
122   public long getSequenceID() {
123     return 0;
124   }
125 
126   @Override
127   public void close() {
128     // do nothing
129   }
130 }