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.compactions;
020
021import static org.apache.hadoop.hbase.regionserver.Store.NO_PRIORITY;
022
023import java.util.Collection;
024import java.util.Collections;
025import java.util.stream.Collectors;
026
027import org.apache.hadoop.hbase.regionserver.HStoreFile;
028import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
029import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
030import org.apache.yetus.audience.InterfaceAudience;
031
032import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
033
034/**
035 * This class holds all logical details necessary to run a compaction.
036 */
037@InterfaceAudience.Private
038public class CompactionRequestImpl implements CompactionRequest {
039
040  // was this compaction promoted to an off-peak
041  private boolean isOffPeak = false;
042  private enum DisplayCompactionType { MINOR, ALL_FILES, MAJOR }
043  private DisplayCompactionType isMajor = DisplayCompactionType.MINOR;
044  private int priority = NO_PRIORITY;
045  private Collection<HStoreFile> filesToCompact;
046  private boolean isAfterSplit = false;
047
048  // CompactRequest object creation time.
049  private long selectionTime;
050  private String regionName = "";
051  private String storeName = "";
052  private long totalSize = -1L;
053  private CompactionLifeCycleTracker tracker = CompactionLifeCycleTracker.DUMMY;
054
055  public CompactionRequestImpl(Collection<HStoreFile> files) {
056    this.selectionTime = EnvironmentEdgeManager.currentTime();
057    this.filesToCompact = Preconditions.checkNotNull(files, "files for compaction can not null");
058    recalculateSize();
059  }
060
061  public void updateFiles(Collection<HStoreFile> files) {
062    this.filesToCompact = Preconditions.checkNotNull(files, "files for compaction can not null");
063    recalculateSize();
064  }
065
066  @Override
067  public Collection<HStoreFile> getFiles() {
068    return Collections.unmodifiableCollection(this.filesToCompact);
069  }
070
071  /**
072   * Sets the region/store name, for logging.
073   */
074  public void setDescription(String regionName, String storeName) {
075    this.regionName = regionName;
076    this.storeName = storeName;
077  }
078
079  /** Gets the total size of all StoreFiles in compaction */
080  @Override
081  public long getSize() {
082    return totalSize;
083  }
084
085  @Override
086  public boolean isAllFiles() {
087    return this.isMajor == DisplayCompactionType.MAJOR
088        || this.isMajor == DisplayCompactionType.ALL_FILES;
089  }
090
091  @Override
092  public boolean isMajor() {
093    return this.isMajor == DisplayCompactionType.MAJOR;
094  }
095
096  /** Gets the priority for the request */
097  @Override
098  public int getPriority() {
099    return priority;
100  }
101
102  /** Sets the priority for the request */
103  public void setPriority(int p) {
104    this.priority = p;
105  }
106
107  @Override
108  public boolean isOffPeak() {
109    return this.isOffPeak;
110  }
111
112  public void setOffPeak(boolean value) {
113    this.isOffPeak = value;
114  }
115
116  @Override
117  public long getSelectionTime() {
118    return this.selectionTime;
119  }
120
121  /**
122   * Specify if this compaction should be a major compaction based on the state of the store
123   * @param isMajor <tt>true</tt> if the system determines that this compaction should be a major
124   *          compaction
125   */
126  public void setIsMajor(boolean isMajor, boolean isAllFiles) {
127    assert isAllFiles || !isMajor;
128    this.isMajor = !isAllFiles ? DisplayCompactionType.MINOR
129        : (isMajor ? DisplayCompactionType.MAJOR : DisplayCompactionType.ALL_FILES);
130  }
131
132  public void setTracker(CompactionLifeCycleTracker tracker) {
133    this.tracker = tracker;
134  }
135
136  public CompactionLifeCycleTracker getTracker() {
137    return tracker;
138  }
139
140  public boolean isAfterSplit() {
141    return isAfterSplit;
142  }
143
144  public void setAfterSplit(boolean afterSplit) {
145    isAfterSplit = afterSplit;
146  }
147
148  @Override
149  public int hashCode() {
150    final int prime = 31;
151    int result = 1;
152    result = prime * result + ((filesToCompact == null) ? 0 : filesToCompact.hashCode());
153    result = prime * result + ((isMajor == null) ? 0 : isMajor.hashCode());
154    result = prime * result + (isOffPeak ? 1231 : 1237);
155    result = prime * result + priority;
156    result = prime * result + ((regionName == null) ? 0 : regionName.hashCode());
157    result = prime * result + (int) (selectionTime ^ (selectionTime >>> 32));
158    result = prime * result + ((storeName == null) ? 0 : storeName.hashCode());
159    result = prime * result + (int) (totalSize ^ (totalSize >>> 32));
160    result = prime * result + ((tracker == null) ? 0 : tracker.hashCode());
161    result = prime * result + (isAfterSplit ? 1231 : 1237);
162    return result;
163  }
164
165  @Override
166  public boolean equals(Object obj) {
167    if (this == obj) {
168      return true;
169    }
170    if (obj == null) {
171      return false;
172    }
173    if (getClass() != obj.getClass()) {
174      return false;
175    }
176    CompactionRequestImpl other = (CompactionRequestImpl) obj;
177    if (filesToCompact == null) {
178      if (other.filesToCompact != null) {
179        return false;
180      }
181    } else if (!filesToCompact.equals(other.filesToCompact)) {
182      return false;
183    }
184    if (isMajor != other.isMajor) {
185      return false;
186    }
187    if (isOffPeak != other.isOffPeak) {
188      return false;
189    }
190    if (priority != other.priority) {
191      return false;
192    }
193    if (regionName == null) {
194      if (other.regionName != null) {
195        return false;
196      }
197    } else if (!regionName.equals(other.regionName)) {
198      return false;
199    }
200    if (selectionTime != other.selectionTime) {
201      return false;
202    }
203    if (storeName == null) {
204      if (other.storeName != null) {
205        return false;
206      }
207    } else if (!storeName.equals(other.storeName)) {
208      return false;
209    }
210    if (totalSize != other.totalSize) {
211      return false;
212    }
213    if (isAfterSplit != other.isAfterSplit) {
214      return false;
215    }
216    if (tracker == null) {
217      if (other.tracker != null) {
218        return false;
219      }
220    } else if (!tracker.equals(other.tracker)) {
221      return false;
222    }
223    return true;
224  }
225
226  @Override
227  public String toString() {
228    String fsList = filesToCompact.stream().filter(f -> f.getReader() != null)
229        .map(f -> TraditionalBinaryPrefix.long2String(f.getReader().length(), "", 1))
230        .collect(Collectors.joining(", "));
231
232    return "regionName=" + regionName + ", storeName=" + storeName + ", fileCount=" +
233        this.getFiles().size() + ", fileSize=" +
234        TraditionalBinaryPrefix.long2String(totalSize, "", 1) +
235        ((fsList.isEmpty()) ? "" : " (" + fsList + ")") + ", priority=" + priority + ", time=" +
236        selectionTime;
237  }
238
239  /**
240   * Recalculate the size of the compaction based on current files.
241   */
242  private void recalculateSize() {
243    this.totalSize = filesToCompact.stream().map(HStoreFile::getReader)
244        .mapToLong(r -> r != null ? r.length() : 0L).sum();
245  }
246}