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   * Creates the scanner for the given read point, and a specific order in a list
125   * @return a scanner for the given read point
126   */
127  @Override
128  public KeyValueScanner getScanner(long readPoint, long order) {
129    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
130  }
131
132  @Override
133  public List<KeyValueScanner> getScanners(long readPoint, long order) {
134    List<KeyValueScanner> list = new ArrayList<>(segments.size());
135    AbstractMemStore.addToScanners(segments, readPoint, order, list);
136    return list;
137  }
138
139  @Override
140  public boolean isTagsPresent() {
141    for (ImmutableSegment s : segments) {
142      if (s.isTagsPresent()) return true;
143    }
144    return false;
145  }
146
147  @Override
148  public void incScannerCount() {
149    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
150  }
151
152  @Override
153  public void decScannerCount() {
154    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
155  }
156
157  /**
158   * Setting the CellSet of the segment - used only for flat immutable segment for setting
159   * immutable CellSet after its creation in immutable segment constructor
160   * @return this object
161   */
162  @Override
163  protected CompositeImmutableSegment setCellSet(CellSet cellSetOld, CellSet cellSetNew) {
164    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
165  }
166
167
168  @Override
169  protected long indexEntrySize() {
170    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
171  }
172
173  @Override protected boolean canBeFlattened() {
174    return false;
175  }
176
177  /**
178   * @return Sum of all cell sizes.
179   */
180  @Override
181  public long getDataSize() {
182    return this.keySize;
183  }
184
185  /**
186   * @return The heap size of this segment.
187   */
188  @Override
189  public long getHeapSize() {
190    long result = 0;
191    for (ImmutableSegment s : segments) {
192      result += s.getHeapSize();
193    }
194    return result;
195  }
196
197  /**
198   * Updates the heap size counter of the segment by the given delta
199   */
200  @Override
201  public long incMemStoreSize(long delta, long heapOverhead, long offHeapOverhead, int cellsCount) {
202    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
203  }
204
205  @Override
206  public long getMinSequenceId() {
207    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
208  }
209
210  @Override
211  public TimeRangeTracker getTimeRangeTracker() {
212    return this.timeRangeTracker;
213  }
214
215  //*** Methods for SegmentsScanner
216  @Override
217  public Cell last() {
218    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
219  }
220
221  @Override
222  public Iterator<Cell> iterator() {
223    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
224  }
225
226  @Override
227  public SortedSet<Cell> headSet(Cell firstKeyOnRow) {
228    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
229  }
230
231  @Override
232  public int compare(Cell left, Cell right) {
233    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
234  }
235
236  @Override
237  public int compareRows(Cell left, Cell right) {
238    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
239  }
240
241  /**
242   * @return a set of all cells in the segment
243   */
244  @Override
245  protected CellSet getCellSet() {
246    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
247  }
248
249  @Override
250  protected void internalAdd(Cell cell, boolean mslabUsed, MemStoreSizing memstoreSizing) {
251    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
252  }
253
254  @Override
255  protected void updateMetaInfo(Cell cellToAdd, boolean succ, boolean mslabUsed,
256      MemStoreSizing memstoreSizing) {
257    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
258  }
259
260  /**
261   * Returns a subset of the segment cell set, which starts with the given cell
262   * @param firstCell a cell in the segment
263   * @return a subset of the segment cell set, which starts with the given cell
264   */
265  @Override
266  protected SortedSet<Cell> tailSet(Cell firstCell) {
267    throw new IllegalStateException("Not supported by CompositeImmutableScanner");
268  }
269
270  // Debug methods
271  /**
272   * Dumps all cells of the segment into the given log
273   */
274  @Override
275  void dump(Logger log) {
276    for (ImmutableSegment s : segments) {
277      s.dump(log);
278    }
279  }
280
281  @Override
282  public String toString() {
283    StringBuilder sb =
284        new StringBuilder("This is CompositeImmutableSegment and those are its segments:: ");
285    for (ImmutableSegment s : segments) {
286      sb.append(s.toString());
287    }
288    return sb.toString();
289  }
290
291  @Override
292  List<KeyValueScanner> getSnapshotScanners() {
293    List<KeyValueScanner> list = new ArrayList<>(this.segments.size());
294    for (ImmutableSegment segment: this.segments) {
295      list.add(new SnapshotSegmentScanner(segment));
296    }
297    return list;
298  }
299}