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