001/**
002 *
003 * Licensed to the Apache Software Foundation (ASF) under one
004 * or more contributor license agreements.  See the NOTICE file
005 * distributed with this work for additional information
006 * regarding copyright ownership.  The ASF licenses this file
007 * to you under the Apache License, Version 2.0 (the
008 * "License"); you may not use this file except in compliance
009 * with the License.  You may obtain a copy of the License at
010 *
011 *     http://www.apache.org/licenses/LICENSE-2.0
012 *
013 * Unless required by applicable law or agreed to in writing, software
014 * distributed under the License is distributed on an "AS IS" BASIS,
015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
016 * See the License for the specific language governing permissions and
017 * limitations under the License.
018 */
019package org.apache.hadoop.hbase.regionserver;
020
021import java.util.ArrayList;
022import java.util.Iterator;
023import java.util.List;
024import java.util.SortedSet;
025
026import org.apache.hadoop.hbase.Cell;
027import org.apache.hadoop.hbase.CellComparator;
028import org.apache.hadoop.hbase.io.TimeRange;
029import org.apache.yetus.audience.InterfaceAudience;
030import org.slf4j.Logger;
031
032/**
033 * The CompositeImmutableSegments is created as a collection of ImmutableSegments and supports
034 * the interface of a single ImmutableSegments.
035 * The CompositeImmutableSegments is planned to be used only as a snapshot,
036 * thus only relevant interfaces are supported
037 */
038@InterfaceAudience.Private
039public class CompositeImmutableSegment extends ImmutableSegment {
040
041  private final List<ImmutableSegment> segments;
042  private long keySize = 0;
043
044  public CompositeImmutableSegment(CellComparator comparator, List<ImmutableSegment> segments) {
045    super(comparator, segments);
046    this.segments = segments;
047    for (ImmutableSegment s : segments) {
048      this.timeRangeTracker.includeTimestamp(s.getTimeRangeTracker().getMax());
049      this.timeRangeTracker.includeTimestamp(s.getTimeRangeTracker().getMin());
050      this.keySize += s.getDataSize();
051    }
052  }
053
054  @Override
055  public List<Segment> getAllSegments() {
056    return new ArrayList<>(segments);
057  }
058
059  @Override
060  public int getNumOfSegments() {
061    return segments.size();
062  }
063
064  /**
065   * @return whether the segment has any cells
066   */
067  @Override
068  public boolean isEmpty() {
069    for (ImmutableSegment s : segments) {
070      if (!s.isEmpty()) return false;
071    }
072    return true;
073  }
074
075  /**
076   * @return number of cells in segment
077   */
078  @Override
079  public int getCellsCount() {
080    int result = 0;
081    for (ImmutableSegment s : segments) {
082      result += s.getCellsCount();
083    }
084    return result;
085  }
086
087  /**
088   * Closing a segment before it is being discarded
089   */
090  @Override
091  public void close() {
092    for (ImmutableSegment s : segments) {
093      s.close();
094    }
095  }
096
097  /**
098   * If the segment has a memory allocator the cell is being cloned to this space, and returned;
099   * otherwise the given cell is returned
100   * @return either the given cell or its clone
101   */
102  @Override
103  public Cell maybeCloneWithAllocator(Cell cell, boolean forceCloneOfBigCell) {
104    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
105  }
106
107  @Override
108  public boolean shouldSeek(TimeRange tr, long oldestUnexpiredTS){
109    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
110  }
111
112  /**
113   * Creates the scanner for the given read point
114   * @return a scanner for the given read point
115   */
116  @Override
117  public KeyValueScanner getScanner(long readPoint) {
118    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
119  }
120
121
122  @Override
123  public List<KeyValueScanner> getScanners(long readPoint) {
124    List<KeyValueScanner> list = new ArrayList<>(segments.size());
125    AbstractMemStore.addToScanners(segments, readPoint, list);
126    return list;
127  }
128
129  @Override
130  public boolean isTagsPresent() {
131    for (ImmutableSegment s : segments) {
132      if (s.isTagsPresent()) return true;
133    }
134    return false;
135  }
136
137  @Override
138  public void incScannerCount() {
139    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
140  }
141
142  @Override
143  public void decScannerCount() {
144    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
145  }
146
147  /**
148   * Setting the CellSet of the segment - used only for flat immutable segment for setting
149   * immutable CellSet after its creation in immutable segment constructor
150   * @return this object
151   */
152  @Override
153  protected CompositeImmutableSegment setCellSet(CellSet cellSetOld, CellSet cellSetNew) {
154    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
155  }
156
157
158  @Override
159  protected long indexEntrySize() {
160    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
161  }
162
163  @Override protected boolean canBeFlattened() {
164    return false;
165  }
166
167  /**
168   * @return Sum of all cell sizes.
169   */
170  @Override
171  public long getDataSize() {
172    return this.keySize;
173  }
174
175  /**
176   * @return The heap size of this segment.
177   */
178  @Override
179  public long getHeapSize() {
180    long result = 0;
181    for (ImmutableSegment s : segments) {
182      result += s.getHeapSize();
183    }
184    return result;
185  }
186
187  /**
188   * Updates the heap size counter of the segment by the given delta
189   */
190  @Override
191  public long incMemStoreSize(long delta, long heapOverhead, long offHeapOverhead, int cellsCount) {
192    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
193  }
194
195  @Override
196  public long getMinSequenceId() {
197    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
198  }
199
200  @Override
201  public TimeRangeTracker getTimeRangeTracker() {
202    return this.timeRangeTracker;
203  }
204
205  //*** Methods for SegmentsScanner
206  @Override
207  public Cell last() {
208    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
209  }
210
211  @Override
212  public Iterator<Cell> iterator() {
213    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
214  }
215
216  @Override
217  public SortedSet<Cell> headSet(Cell firstKeyOnRow) {
218    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
219  }
220
221  @Override
222  public int compare(Cell left, Cell right) {
223    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
224  }
225
226  @Override
227  public int compareRows(Cell left, Cell right) {
228    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
229  }
230
231  /**
232   * @return a set of all cells in the segment
233   */
234  @Override
235  protected CellSet getCellSet() {
236    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
237  }
238
239  @Override
240  protected void internalAdd(Cell cell, boolean mslabUsed, MemStoreSizing memstoreSizing,
241      boolean sizeAddedPreOperation) {
242    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
243  }
244
245  @Override
246  protected void updateMetaInfo(Cell cellToAdd, boolean succ, boolean mslabUsed,
247      MemStoreSizing memstoreSizing, boolean sizeAddedPreOperation) {
248    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
249  }
250
251  /**
252   * Returns a subset of the segment cell set, which starts with the given cell
253   * @param firstCell a cell in the segment
254   * @return a subset of the segment cell set, which starts with the given cell
255   */
256  @Override
257  protected SortedSet<Cell> tailSet(Cell firstCell) {
258    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
259  }
260
261  // Debug methods
262  /**
263   * Dumps all cells of the segment into the given log
264   */
265  @Override
266  void dump(Logger log) {
267    for (ImmutableSegment s : segments) {
268      s.dump(log);
269    }
270  }
271
272  @Override
273  public String toString() {
274    StringBuilder sb =
275        new StringBuilder("This is CompositeImmutableSegment and those are its segments:: ");
276    for (ImmutableSegment s : segments) {
277      sb.append(s.toString());
278    }
279    return sb.toString();
280  }
281
282  @Override
283  List<KeyValueScanner> getSnapshotScanners() {
284    List<KeyValueScanner> list = new ArrayList<>(this.segments.size());
285    for (ImmutableSegment segment: this.segments) {
286      list.add(new SnapshotSegmentScanner(segment));
287    }
288    return list;
289  }
290}