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