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