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.io.IOException;
021import java.util.Arrays;
022import java.util.Map;
023import java.util.Optional;
024import java.util.OptionalLong;
025import java.util.TreeMap;
026
027import org.apache.hadoop.fs.Path;
028import org.apache.hadoop.hbase.Cell;
029import org.apache.hadoop.hbase.CellBuilderFactory;
030import org.apache.hadoop.hbase.CellBuilderType;
031import org.apache.hadoop.hbase.HBaseTestingUtility;
032import org.apache.hadoop.hbase.HDFSBlocksDistribution;
033import org.apache.hadoop.hbase.io.hfile.CacheConfig;
034import org.apache.hadoop.hbase.util.Bytes;
035import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
036import org.apache.yetus.audience.InterfaceAudience;
037
038/** A mock used so our tests don't deal with actual StoreFiles */
039@InterfaceAudience.Private
040public class MockHStoreFile extends HStoreFile {
041  long length = 0;
042  boolean isRef = false;
043  long ageInDisk;
044  long sequenceid;
045  private Map<byte[], byte[]> metadata = new TreeMap<>(Bytes.BYTES_COMPARATOR);
046  byte[] splitPoint = null;
047  TimeRangeTracker timeRangeTracker;
048  long entryCount;
049  boolean isMajor;
050  HDFSBlocksDistribution hdfsBlocksDistribution;
051  long modificationTime;
052  boolean compactedAway;
053
054  MockHStoreFile(HBaseTestingUtility testUtil, Path testPath,
055      long length, long ageInDisk, boolean isRef, long sequenceid) throws IOException {
056    super(testUtil.getTestFileSystem(), testPath, testUtil.getConfiguration(),
057        new CacheConfig(testUtil.getConfiguration()), BloomType.NONE, true);
058    this.length = length;
059    this.isRef = isRef;
060    this.ageInDisk = ageInDisk;
061    this.sequenceid = sequenceid;
062    this.isMajor = false;
063    hdfsBlocksDistribution = new HDFSBlocksDistribution();
064    hdfsBlocksDistribution.addHostsAndBlockWeight(
065      new String[] { RSRpcServices.getHostname(testUtil.getConfiguration(), false) }, 1);
066    modificationTime = EnvironmentEdgeManager.currentTime();
067  }
068
069  void setLength(long newLen) {
070    this.length = newLen;
071  }
072
073  @Override
074  public long getMaxSequenceId() {
075    return sequenceid;
076  }
077
078  @Override
079  public boolean isMajorCompactionResult() {
080    return isMajor;
081  }
082
083  public void setIsMajor(boolean isMajor) {
084    this.isMajor = isMajor;
085  }
086
087  @Override
088  public boolean isReference() {
089    return this.isRef;
090  }
091
092  @Override
093  public boolean isBulkLoadResult() {
094    return false;
095  }
096
097  @Override
098  public byte[] getMetadataValue(byte[] key) {
099    return this.metadata.get(key);
100  }
101
102  public void setMetadataValue(byte[] key, byte[] value) {
103    this.metadata.put(key, value);
104  }
105
106  void setTimeRangeTracker(TimeRangeTracker timeRangeTracker) {
107    this.timeRangeTracker = timeRangeTracker;
108  }
109
110  void setEntries(long entryCount) {
111    this.entryCount = entryCount;
112  }
113
114  @Override
115  public OptionalLong getMinimumTimestamp() {
116    return timeRangeTracker == null ? OptionalLong.empty()
117        : OptionalLong.of(timeRangeTracker.getMin());
118  }
119
120  @Override
121  public OptionalLong getMaximumTimestamp() {
122    return timeRangeTracker == null ? OptionalLong.empty()
123        : OptionalLong.of(timeRangeTracker.getMax());
124  }
125
126  @Override
127  public void markCompactedAway() {
128    this.compactedAway = true;
129  }
130
131  @Override
132  public boolean isCompactedAway() {
133    return compactedAway;
134  }
135
136  @Override
137  public long getModificationTimeStamp() {
138    return getModificationTimestamp();
139  }
140
141  @Override
142  public long getModificationTimestamp() {
143    return modificationTime;
144  }
145
146  @Override
147  public HDFSBlocksDistribution getHDFSBlockDistribution() {
148    return hdfsBlocksDistribution;
149  }
150
151  @Override
152  public void initReader() throws IOException {
153  }
154
155  @Override
156  public StoreFileScanner getPreadScanner(boolean cacheBlocks, long readPt, long scannerOrder,
157      boolean canOptimizeForNonNullColumn) {
158    return getReader().getStoreFileScanner(cacheBlocks, true, false, readPt, scannerOrder,
159      canOptimizeForNonNullColumn);
160  }
161
162  @Override
163  public StoreFileScanner getStreamScanner(boolean canUseDropBehind, boolean cacheBlocks,
164      boolean isCompaction, long readPt, long scannerOrder, boolean canOptimizeForNonNullColumn)
165      throws IOException {
166    return getReader().getStoreFileScanner(cacheBlocks, false, isCompaction, readPt, scannerOrder,
167      canOptimizeForNonNullColumn);
168  }
169
170  @Override
171  public StoreFileReader getReader() {
172    final long len = this.length;
173    final TimeRangeTracker timeRangeTracker = this.timeRangeTracker;
174    final long entries = this.entryCount;
175    return new StoreFileReader() {
176      @Override
177      public long length() {
178        return len;
179      }
180
181      @Override
182      public long getMaxTimestamp() {
183        return timeRange == null? Long.MAX_VALUE: timeRangeTracker.getMax();
184      }
185
186      @Override
187      public long getEntries() {
188        return entries;
189      }
190
191      @Override
192      public void close(boolean evictOnClose) throws IOException {
193        // no-op
194      }
195
196      @Override
197      public Optional<Cell> getLastKey() {
198        if (splitPoint != null) {
199          return Optional.of(CellBuilderFactory.create(CellBuilderType.DEEP_COPY)
200              .setType(Cell.Type.Put)
201              .setRow(Arrays.copyOf(splitPoint, splitPoint.length + 1)).build());
202        } else {
203          return Optional.empty();
204        }
205      }
206
207      @Override
208      public Optional<Cell> midKey() throws IOException {
209        if (splitPoint != null) {
210          return Optional.of(CellBuilderFactory.create(CellBuilderType.DEEP_COPY)
211              .setType(Cell.Type.Put).setRow(splitPoint).build());
212        } else {
213          return Optional.empty();
214        }
215      }
216
217      @Override
218      public Optional<Cell> getFirstKey() {
219        if (splitPoint != null) {
220          return Optional.of(CellBuilderFactory.create(CellBuilderType.DEEP_COPY)
221              .setType(Cell.Type.Put).setRow(splitPoint, 0, splitPoint.length - 1)
222              .build());
223        } else {
224          return Optional.empty();
225        }
226      }
227    };
228  }
229
230  @Override
231  public OptionalLong getBulkLoadTimestamp() {
232    // we always return false for isBulkLoadResult so we do not have a bulk load timestamp
233    return OptionalLong.empty();
234  }
235}